You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by jkbradley <gi...@git.apache.org> on 2015/04/28 08:57:17 UTC

[GitHub] spark pull request: [SPARK-7176] [ml] Add validation functionality...

GitHub user jkbradley opened a pull request:

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

    [SPARK-7176] [ml] Add validation functionality to Param

    Main change: Added isValid field to Param.  Modified all usages to use isValid when relevant.  Added helper methods in ParamValidate.
    
    Also overrode Params.validate() in:
    * CrossValidator + model
    * Pipeline + model
    
    This PR is Scala + Java only.  Python will be in a follow-up PR.
    
    CC: @mengxr

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

    $ git pull https://github.com/jkbradley/spark enforce-validate

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

    https://github.com/apache/spark/pull/5740.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 #5740
    
----
commit dc2061f0a51bc80dac030c925024c39b26614b91
Author: Joseph K. Bradley <jo...@databricks.com>
Date:   2015-04-27T18:03:53Z

    merged with master.  enforcing Params.validate

commit 57b8ad12b4082bc330c82e56d86a6e8669e440d0
Author: Joseph K. Bradley <jo...@databricks.com>
Date:   2015-04-27T21:01:05Z

    Partly done with adding checks, but blocking on adding checking functionality to Param

commit 8e368c712f6e4d5567b1274ba3e770af17a06a59
Author: Joseph K. Bradley <jo...@databricks.com>
Date:   2015-04-28T02:58:26Z

    Still workin

commit d87278c211ea81f73a2a16f8e0ef282eefec0637
Author: Joseph K. Bradley <jo...@databricks.com>
Date:   2015-04-28T04:30:36Z

    still workin

commit 26d327c08f4ac53c5e1e6719164e4bf32c413c9d
Author: Joseph K. Bradley <jo...@databricks.com>
Date:   2015-04-28T05:15:40Z

    Maybe done

commit 39b036b2d622d4aa823db3fa24107a177ac6463a
Author: Joseph K. Bradley <jo...@databricks.com>
Date:   2015-04-28T05:40:16Z

    many cleanups

commit f02c3c3ac1066e841615d07f2e003357b2f863f3
Author: Joseph K. Bradley <jo...@databricks.com>
Date:   2015-04-28T05:54:43Z

    small cleanups

----


---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#discussion_r29273024
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala ---
    @@ -32,10 +32,14 @@ import org.apache.spark.sql.types.DataType
     class HashingTF extends UnaryTransformer[Iterable[_], Vector, HashingTF] {
     
       /**
    -   * number of features
    +   * Number of features.  Should be > 0.
    +   * (default = 2^18^)
        * @group param
        */
    -  val numFeatures = new IntParam(this, "numFeatures", "number of features")
    +  val numFeatures = new IntParam(this, "numFeatures", "number of features (> 0)",
    +    ParamValidate.gt[Int](0))
    --- End diff --
    
    `[Int]` is not necessary.


---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#issuecomment-97184042
  
    Btw, I made a few updates for the elastic net patch:
    * I changed "tol" to "convergenceTol"
    * I added some documentation


---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#discussion_r29307952
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala ---
    @@ -136,7 +147,7 @@ private[recommendation] trait ALSParams extends Params with HasMaxIter with HasR
     
       setDefault(rank -> 10, maxIter -> 10, regParam -> 0.1, numUserBlocks -> 10, numItemBlocks -> 10,
         implicitPrefs -> false, alpha -> 1.0, userCol -> "user", itemCol -> "item",
    -    ratingCol -> "rating", nonnegative -> false)
    +    ratingCol -> "rating", nonnegative -> false, checkpointInterval -> 10)
    --- End diff --
    
    Yes, I changed it to 10 in spark.ml. We may checkpoint too often with 3.


---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#discussion_r29273016
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala ---
    @@ -86,6 +86,13 @@ class Pipeline extends Estimator[PipelineModel] {
       def setStages(value: Array[PipelineStage]): this.type = { set(stages, value); this }
       def getStages: Array[PipelineStage] = getOrDefault(stages)
     
    +  override def validate(paramMap: ParamMap): Unit = {
    +    val map = extractParamMap(paramMap)
    +    getStages.foreach {
    +      case pStage: Params => pStage.validate(map)
    +    }
    --- End diff --
    
    Should add `case _ =>`.


---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#issuecomment-97565883
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/31301/
    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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#discussion_r29281239
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala ---
    @@ -89,16 +94,19 @@ private[recommendation] trait ALSParams extends Params with HasMaxIter with HasR
       def getImplicitPrefs: Boolean = getOrDefault(implicitPrefs)
     
       /**
    -   * Param for the alpha parameter in the implicit preference formulation.
    +   * Param for the alpha parameter in the implicit preference formulation (>= 0).
    +   * Default: 1.0
        * @group param
        */
    -  val alpha = new DoubleParam(this, "alpha", "alpha for implicit preference")
    +  val alpha = new DoubleParam(this, "alpha", "alpha for implicit preference",
    --- End diff --
    
    @mengxr  Please confirm this isValid is 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: [SPARK-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#issuecomment-97601884
  
    @jkbradley Could you merge the master branch? The changes to LinearRegression documentation may conflict with #5767. If there are documentation updates, we can do that in a separate PR.


---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#issuecomment-97173182
  
    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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#issuecomment-97565853
  
      [Test build #31301 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/31301/consoleFull) for   PR 5740 at commit [`f148330`](https://github.com/apache/spark/commit/f148330b3ebe6bdb73b80612add8bf9f67f60d7b).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class Param[T] (val parent: Params, val name: String, val doc: String, val isValid: T => Boolean)`
      * `class DoubleParam(parent: Params, name: String, doc: String, isValid: Double => Boolean)`
      * `class IntParam(parent: Params, name: String, doc: String, isValid: Int => Boolean)`
      * `class FloatParam(parent: Params, name: String, doc: String, isValid: Float => Boolean)`
      * `class LongParam(parent: Params, name: String, doc: String, isValid: Long => Boolean)`
      * `class BooleanParam(parent: Params, name: String, doc: String) // No need for isValid`
      * `case class ParamPair[T](param: Param[T], value: T) `
    
     * This patch **adds the following new dependencies:**
       * `spark-unsafe_2.10-1.4.0-SNAPSHOT.jar`



---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#issuecomment-96951332
  
      [Test build #31126 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/31126/consoleFull) for   PR 5740 at commit [`f02c3c3`](https://github.com/apache/spark/commit/f02c3c3ac1066e841615d07f2e003357b2f863f3).
     * This patch **fails RAT tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class Param[T] (val parent: Params, val name: String, val doc: String, val isValid: T => Boolean)`
      * `class DoubleParam(parent: Params, name: String, doc: String, isValid: Double => Boolean)`
      * `class IntParam(parent: Params, name: String, doc: String, isValid: Int => Boolean)`
      * `class FloatParam(parent: Params, name: String, doc: String, isValid: Float => Boolean)`
      * `class LongParam(parent: Params, name: String, doc: String, isValid: Long => Boolean)`
      * `class BooleanParam(parent: Params, name: String, doc: String) // No need for isValid`
      * `case class ParamPair[T](param: Param[T], value: T) `
    
     * This patch does not change any dependencies.


---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#issuecomment-97246353
  
    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-7176] [ml] Add validation functionality...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

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


---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#issuecomment-97525087
  
    Merged build started.


---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#discussion_r29281272
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala ---
    @@ -136,7 +147,7 @@ private[recommendation] trait ALSParams extends Params with HasMaxIter with HasR
     
       setDefault(rank -> 10, maxIter -> 10, regParam -> 0.1, numUserBlocks -> 10, numItemBlocks -> 10,
         implicitPrefs -> false, alpha -> 1.0, userCol -> "user", itemCol -> "item",
    -    ratingCol -> "rating", nonnegative -> false)
    +    ratingCol -> "rating", nonnegative -> false, checkpointInterval -> 10)
    --- End diff --
    
    @mengxr  Defaults for ALS differ between ml and mllib.  Please confirm values in ml.


---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#issuecomment-97619151
  
      [Test build #31343 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/31343/consoleFull) for   PR 5740 at commit [`ad9c6c1`](https://github.com/apache/spark/commit/ad9c6c19834efac4f593dc874d6321e28a5cf95a).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `final class Word2Vec extends Estimator[Word2VecModel] with Word2VecBase `
      * `class Param[T] (val parent: Params, val name: String, val doc: String, val isValid: T => Boolean)`
      * `class DoubleParam(parent: Params, name: String, doc: String, isValid: Double => Boolean)`
      * `class IntParam(parent: Params, name: String, doc: String, isValid: Int => Boolean)`
      * `class FloatParam(parent: Params, name: String, doc: String, isValid: Float => Boolean)`
      * `class LongParam(parent: Params, name: String, doc: String, isValid: Long => Boolean)`
      * `class BooleanParam(parent: Params, name: String, doc: String) // No need for isValid`
      * `case class ParamPair[T](param: Param[T], value: T) `
    
     * This patch does not change any dependencies.


---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#discussion_r29273033
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala ---
    @@ -38,9 +38,12 @@ class PolynomialExpansion extends UnaryTransformer[Vector, Vector, PolynomialExp
     
       /**
        * The polynomial degree to expand, which should be larger than 1.
    +   * Default: 2
        * @group param
        */
    -  val degree = new IntParam(this, "degree", "the polynomial degree to expand")
    +  val degree = new IntParam(this, "degree", "the polynomial degree to expand",
    +    ParamValidate.gt[Int](2))
    --- End diff --
    
    `gt(1)`. Users may set degree to `1` to turn off polynomial expansion in tuning.


---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#discussion_r29273107
  
    --- Diff: python/pyspark/mllib/linalg.py ---
    @@ -39,7 +39,8 @@
         IntegerType, ByteType
     
     
    -__all__ = ['Vector', 'DenseVector', 'SparseVector', 'Vectors', 'DenseMatrix', 'Matrices']
    +__all__ = ['Vector', 'DenseVector', 'SparseVector', 'Vectors',
    +           'DenseMatrix', 'Matrices', 'SparseMatrix']
    --- End diff --
    
    This should be in a different PR.


---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#issuecomment-97246349
  
      [Test build #31178 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/31178/consoleFull) for   PR 5740 at commit [`e54d79e`](https://github.com/apache/spark/commit/e54d79e1f679d5b7739dc4bf82b0303099dae466).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class Param[T] (val parent: Params, val name: String, val doc: String, val isValid: T => Boolean)`
      * `class DoubleParam(parent: Params, name: String, doc: String, isValid: Double => Boolean)`
      * `class IntParam(parent: Params, name: String, doc: String, isValid: Int => Boolean)`
      * `class FloatParam(parent: Params, name: String, doc: String, isValid: Float => Boolean)`
      * `class LongParam(parent: Params, name: String, doc: String, isValid: Long => Boolean)`
      * `class BooleanParam(parent: Params, name: String, doc: String) // No need for isValid`
      * `case class ParamPair[T](param: Param[T], value: T) `
      * `trait LDAOptimizer`
      * `class EMLDAOptimizer extends LDAOptimizer`
      * `class OffsetRange(object):`
      * `class TopicAndPartition(object):`
      * `class Broker(object):`
      * `trait ExpectsInputTypes `
      * `abstract class BinaryMathExpression(f: (Double, Double) => Double, name: String) `
      * `case class Pow(left: Expression, right: Expression) extends BinaryMathExpression(math.pow, "POWER")`
      * `case class Hypot(`
      * `case class Atan2(`
      * `abstract class MathematicalExpression(name: String)`
      * `abstract class MathematicalExpressionForDouble(f: Double => Double, name: String)`
      * `abstract class MathematicalExpressionForInt(f: Int => Int, name: String)`
      * `abstract class MathematicalExpressionForFloat(f: Float => Float, name: String)`
      * `abstract class MathematicalExpressionForLong(f: Long => Long, name: String)`
      * `case class Sin(child: Expression) extends MathematicalExpressionForDouble(math.sin, "SIN")`
      * `case class Asin(child: Expression) extends MathematicalExpressionForDouble(math.asin, "ASIN")`
      * `case class Sinh(child: Expression) extends MathematicalExpressionForDouble(math.sinh, "SINH")`
      * `case class Cos(child: Expression) extends MathematicalExpressionForDouble(math.cos, "COS")`
      * `case class Acos(child: Expression) extends MathematicalExpressionForDouble(math.acos, "ACOS")`
      * `case class Cosh(child: Expression) extends MathematicalExpressionForDouble(math.cosh, "COSH")`
      * `case class Tan(child: Expression) extends MathematicalExpressionForDouble(math.tan, "TAN")`
      * `case class Atan(child: Expression) extends MathematicalExpressionForDouble(math.atan, "ATAN")`
      * `case class Tanh(child: Expression) extends MathematicalExpressionForDouble(math.tanh, "TANH")`
      * `case class Ceil(child: Expression) extends MathematicalExpressionForDouble(math.ceil, "CEIL")`
      * `case class Floor(child: Expression) extends MathematicalExpressionForDouble(math.floor, "FLOOR")`
      * `case class Rint(child: Expression) extends MathematicalExpressionForDouble(math.rint, "ROUND")`
      * `case class Cbrt(child: Expression) extends MathematicalExpressionForDouble(math.cbrt, "CBRT")`
      * `case class Signum(child: Expression) extends MathematicalExpressionForDouble(math.signum, "SIGNUM")`
      * `case class ISignum(child: Expression) extends MathematicalExpressionForInt(math.signum, "ISIGNUM")`
      * `case class FSignum(child: Expression) extends MathematicalExpressionForFloat(math.signum, "FSIGNUM")`
      * `case class LSignum(child: Expression) extends MathematicalExpressionForLong(math.signum, "LSIGNUM")`
      * `case class ToDegrees(child: Expression) `
      * `case class ToRadians(child: Expression) `
      * `case class Log(child: Expression) extends MathematicalExpressionForDouble(math.log, "LOG")`
      * `case class Log10(child: Expression) extends MathematicalExpressionForDouble(math.log10, "LOG10")`
      * `case class Log1p(child: Expression) extends MathematicalExpressionForDouble(math.log1p, "LOG1P")`
      * `case class Exp(child: Expression) extends MathematicalExpressionForDouble(math.exp, "EXP")`
      * `case class Expm1(child: Expression) extends MathematicalExpressionForDouble(math.expm1, "EXPM1")`
    
     * This patch does not change any dependencies.


---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#issuecomment-97207211
  
     Merged build triggered.


---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#issuecomment-97273089
  
      [Test build #730 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/730/consoleFull) for   PR 5740 at commit [`e54d79e`](https://github.com/apache/spark/commit/e54d79e1f679d5b7739dc4bf82b0303099dae466).


---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#issuecomment-97565879
  
    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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#discussion_r29220742
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/param/params.scala ---
    @@ -305,6 +439,8 @@ private[spark] object Params {
       }
     }
     
    +abstract class JavaParams extends Params
    --- End diff --
    
    I'll add doc 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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#issuecomment-97173169
  
      [Test build #31148 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/31148/consoleFull) for   PR 5740 at commit [`9cf4dc5`](https://github.com/apache/spark/commit/9cf4dc51a563e118fcf47c6c6f46283ea161432f).
     * This patch **fails Spark unit tests**.
     * This patch **does not merge cleanly**.
     * This patch adds the following public classes _(experimental)_:
      * `class Param[T] (val parent: Params, val name: String, val doc: String, val isValid: T => Boolean)`
      * `class DoubleParam(parent: Params, name: String, doc: String, isValid: Double => Boolean)`
      * `class IntParam(parent: Params, name: String, doc: String, isValid: Int => Boolean)`
      * `class FloatParam(parent: Params, name: String, doc: String, isValid: Float => Boolean)`
      * `class LongParam(parent: Params, name: String, doc: String, isValid: Long => Boolean)`
      * `class BooleanParam(parent: Params, name: String, doc: String) // No need for isValid`
      * `case class ParamPair[T](param: Param[T], value: T) `
    
     * This patch **removes the following dependencies:**
       * `RoaringBitmap-0.4.5.jar`
       * `activation-1.1.jar`
       * `akka-actor_2.10-2.3.4-spark.jar`
       * `akka-remote_2.10-2.3.4-spark.jar`
       * `akka-slf4j_2.10-2.3.4-spark.jar`
       * `aopalliance-1.0.jar`
       * `arpack_combined_all-0.1.jar`
       * `avro-1.7.7.jar`
       * `breeze-macros_2.10-0.11.2.jar`
       * `breeze_2.10-0.11.2.jar`
       * `chill-java-0.5.0.jar`
       * `chill_2.10-0.5.0.jar`
       * `commons-beanutils-1.7.0.jar`
       * `commons-beanutils-core-1.8.0.jar`
       * `commons-cli-1.2.jar`
       * `commons-codec-1.10.jar`
       * `commons-collections-3.2.1.jar`
       * `commons-compress-1.4.1.jar`
       * `commons-configuration-1.6.jar`
       * `commons-digester-1.8.jar`
       * `commons-httpclient-3.1.jar`
       * `commons-io-2.1.jar`
       * `commons-lang-2.5.jar`
       * `commons-lang3-3.3.2.jar`
       * `commons-math-2.1.jar`
       * `commons-math3-3.4.1.jar`
       * `commons-net-2.2.jar`
       * `compress-lzf-1.0.0.jar`
       * `config-1.2.1.jar`
       * `core-1.1.2.jar`
       * `curator-client-2.4.0.jar`
       * `curator-framework-2.4.0.jar`
       * `curator-recipes-2.4.0.jar`
       * `gmbal-api-only-3.0.0-b023.jar`
       * `grizzly-framework-2.1.2.jar`
       * `grizzly-http-2.1.2.jar`
       * `grizzly-http-server-2.1.2.jar`
       * `grizzly-http-servlet-2.1.2.jar`
       * `grizzly-rcm-2.1.2.jar`
       * `groovy-all-2.3.7.jar`
       * `guava-14.0.1.jar`
       * `guice-3.0.jar`
       * `hadoop-annotations-2.2.0.jar`
       * `hadoop-auth-2.2.0.jar`
       * `hadoop-client-2.2.0.jar`
       * `hadoop-common-2.2.0.jar`
       * `hadoop-hdfs-2.2.0.jar`
       * `hadoop-mapreduce-client-app-2.2.0.jar`
       * `hadoop-mapreduce-client-common-2.2.0.jar`
       * `hadoop-mapreduce-client-core-2.2.0.jar`
       * `hadoop-mapreduce-client-jobclient-2.2.0.jar`
       * `hadoop-mapreduce-client-shuffle-2.2.0.jar`
       * `hadoop-yarn-api-2.2.0.jar`
       * `hadoop-yarn-client-2.2.0.jar`
       * `hadoop-yarn-common-2.2.0.jar`
       * `hadoop-yarn-server-common-2.2.0.jar`
       * `ivy-2.4.0.jar`
       * `jackson-annotations-2.4.0.jar`
       * `jackson-core-2.4.4.jar`
       * `jackson-core-asl-1.8.8.jar`
       * `jackson-databind-2.4.4.jar`
       * `jackson-jaxrs-1.8.8.jar`
       * `jackson-mapper-asl-1.8.8.jar`
       * `jackson-module-scala_2.10-2.4.4.jar`
       * `jackson-xc-1.8.8.jar`
       * `jansi-1.4.jar`
       * `javax.inject-1.jar`
       * `javax.servlet-3.0.0.v201112011016.jar`
       * `javax.servlet-3.1.jar`
       * `javax.servlet-api-3.0.1.jar`
       * `jaxb-api-2.2.2.jar`
       * `jaxb-impl-2.2.3-1.jar`
       * `jcl-over-slf4j-1.7.10.jar`
       * `jersey-client-1.9.jar`
       * `jersey-core-1.9.jar`
       * `jersey-grizzly2-1.9.jar`
       * `jersey-guice-1.9.jar`
       * `jersey-json-1.9.jar`
       * `jersey-server-1.9.jar`
       * `jersey-test-framework-core-1.9.jar`
       * `jersey-test-framework-grizzly2-1.9.jar`
       * `jets3t-0.7.1.jar`
       * `jettison-1.1.jar`
       * `jetty-util-6.1.26.jar`
       * `jline-0.9.94.jar`
       * `jline-2.10.4.jar`
       * `jodd-core-3.6.3.jar`
       * `json4s-ast_2.10-3.2.10.jar`
       * `json4s-core_2.10-3.2.10.jar`
       * `json4s-jackson_2.10-3.2.10.jar`
       * `jsr305-1.3.9.jar`
       * `jtransforms-2.4.0.jar`
       * `jul-to-slf4j-1.7.10.jar`
       * `kryo-2.21.jar`
       * `log4j-1.2.17.jar`
       * `lz4-1.2.0.jar`
       * `management-api-3.0.0-b012.jar`
       * `mesos-0.21.0-shaded-protobuf.jar`
       * `metrics-core-3.1.0.jar`
       * `metrics-graphite-3.1.0.jar`
       * `metrics-json-3.1.0.jar`
       * `metrics-jvm-3.1.0.jar`
       * `minlog-1.2.jar`
       * `netty-3.8.0.Final.jar`
       * `netty-all-4.0.23.Final.jar`
       * `objenesis-1.2.jar`
       * `opencsv-2.3.jar`
       * `oro-2.0.8.jar`
       * `paranamer-2.6.jar`
       * `parquet-column-1.6.0rc3.jar`
       * `parquet-common-1.6.0rc3.jar`
       * `parquet-encoding-1.6.0rc3.jar`
       * `parquet-format-2.2.0-rc1.jar`
       * `parquet-generator-1.6.0rc3.jar`
       * `parquet-hadoop-1.6.0rc3.jar`
       * `parquet-jackson-1.6.0rc3.jar`
       * `protobuf-java-2.4.1.jar`
       * `protobuf-java-2.5.0-spark.jar`
       * `py4j-0.8.2.1.jar`
       * `pyrolite-2.0.1.jar`
       * `quasiquotes_2.10-2.0.1.jar`
       * `reflectasm-1.07-shaded.jar`
       * `scala-compiler-2.10.4.jar`
       * `scala-library-2.10.4.jar`
       * `scala-reflect-2.10.4.jar`
       * `scalap-2.10.4.jar`
       * `scalatest_2.10-2.2.1.jar`
       * `slf4j-api-1.7.10.jar`
       * `slf4j-log4j12-1.7.10.jar`
       * `snappy-java-1.1.1.7.jar`
       * `spark-bagel_2.10-1.4.0-SNAPSHOT.jar`
       * `spark-catalyst_2.10-1.4.0-SNAPSHOT.jar`
       * `spark-core_2.10-1.4.0-SNAPSHOT.jar`
       * `spark-graphx_2.10-1.4.0-SNAPSHOT.jar`
       * `spark-launcher_2.10-1.4.0-SNAPSHOT.jar`
       * `spark-mllib_2.10-1.4.0-SNAPSHOT.jar`
       * `spark-network-common_2.10-1.4.0-SNAPSHOT.jar`
       * `spark-network-shuffle_2.10-1.4.0-SNAPSHOT.jar`
       * `spark-repl_2.10-1.4.0-SNAPSHOT.jar`
       * `spark-sql_2.10-1.4.0-SNAPSHOT.jar`
       * `spark-streaming_2.10-1.4.0-SNAPSHOT.jar`
       * `spire-macros_2.10-0.7.4.jar`
       * `spire_2.10-0.7.4.jar`
       * `stax-api-1.0.1.jar`
       * `stream-2.7.0.jar`
       * `tachyon-0.6.4.jar`
       * `tachyon-client-0.6.4.jar`
       * `uncommons-maths-1.2.2a.jar`
       * `unused-1.0.0.jar`
       * `xmlenc-0.52.jar`
       * `xz-1.0.jar`
       * `zookeeper-3.4.5.jar`



---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#issuecomment-97607138
  
     Merged build triggered.


---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#discussion_r29273820
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/param/params.scala ---
    @@ -65,38 +90,129 @@ class Param[T] (val parent: Params, val name: String, val doc: String) extends S
       }
     }
     
    +/**
    + * Factory methods for common validation functions for [[Param.isValid]].
    + * The numerical methods only support Int, Long, Float, and Double.
    + */
    +object ParamValidate {
    --- End diff --
    
    We might need `sealed abstract class ParamValidate` to make it show up correctly in Java. For the name, I recommend `ParamValidators` instead of `ParamValidate`.


---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#issuecomment-97525210
  
      [Test build #31301 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/31301/consoleFull) for   PR 5740 at commit [`f148330`](https://github.com/apache/spark/commit/f148330b3ebe6bdb73b80612add8bf9f67f60d7b).


---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#issuecomment-97619999
  
    LGTM. Merged into master. 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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#discussion_r29273087
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala ---
    @@ -93,6 +95,13 @@ class CrossValidator extends Estimator[CrossValidatorModel] with CrossValidatorP
       /** @group setParam */
       def setNumFolds(value: Int): this.type = set(numFolds, value)
     
    +  override def validate(paramMap: ParamMap): Unit = {
    +    val map = extractParamMap(paramMap)
    +    getEstimatorParamMaps.foreach { eMap =>
    +      getEstimator.validate(map ++ eMap)
    --- End diff --
    
    Should be `getEstimator.validate(eMap ++ paramMap)`. Estimator doesn't need CV params.


---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#issuecomment-97619164
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/31343/
    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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#issuecomment-97207300
  
      [Test build #31178 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/31178/consoleFull) for   PR 5740 at commit [`e54d79e`](https://github.com/apache/spark/commit/e54d79e1f679d5b7739dc4bf82b0303099dae466).


---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#issuecomment-97173185
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/31148/
    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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#issuecomment-97246356
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/31178/
    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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#issuecomment-97142862
  
      [Test build #31148 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/31148/consoleFull) for   PR 5740 at commit [`9cf4dc5`](https://github.com/apache/spark/commit/9cf4dc51a563e118fcf47c6c6f46283ea161432f).


---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#issuecomment-96951321
  
      [Test build #31126 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/31126/consoleFull) for   PR 5740 at commit [`f02c3c3`](https://github.com/apache/spark/commit/f02c3c3ac1066e841615d07f2e003357b2f863f3).


---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#issuecomment-97525023
  
     Merged build triggered.


---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#issuecomment-97515319
  
    @mengxr  I'm fine taking tol out of this PR.  I missed the discussion on the elastic net PR.  Using "tol" goes against usual Spark naming conventions, which aim to make names easily understood.  I'll remove it though.


---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#issuecomment-97207127
  
    @mengxr  Hopefully that fixed everything.  I'm sending separate PRs for those other 2 small, unrelated changes


---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#issuecomment-97607364
  
      [Test build #31343 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/31343/consoleFull) for   PR 5740 at commit [`ad9c6c1`](https://github.com/apache/spark/commit/ad9c6c19834efac4f593dc874d6321e28a5cf95a).


---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#discussion_r29307951
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala ---
    @@ -89,16 +94,19 @@ private[recommendation] trait ALSParams extends Params with HasMaxIter with HasR
       def getImplicitPrefs: Boolean = getOrDefault(implicitPrefs)
     
       /**
    -   * Param for the alpha parameter in the implicit preference formulation.
    +   * Param for the alpha parameter in the implicit preference formulation (>= 0).
    +   * Default: 1.0
        * @group param
        */
    -  val alpha = new DoubleParam(this, "alpha", "alpha for implicit preference")
    +  val alpha = new DoubleParam(this, "alpha", "alpha for implicit preference",
    --- End diff --
    
    Yes.


---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#discussion_r29280831
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/param/params.scala ---
    @@ -65,38 +90,129 @@ class Param[T] (val parent: Params, val name: String, val doc: String) extends S
       }
     }
     
    +/**
    + * Factory methods for common validation functions for [[Param.isValid]].
    + * The numerical methods only support Int, Long, Float, and Double.
    + */
    +object ParamValidate {
    --- End diff --
    
    Hm, it was working before...I'll fix it and also check out the Java docs


---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#issuecomment-97207238
  
    Merged build started.


---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#issuecomment-97523389
  
    updated


---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#issuecomment-97291808
  
    @jkbradley Let's try to keep this PR minimal. We discussed `tol` vs. `convergenceTol` in the elastic net PR and `tol` seems to be sufficient, which is also used in scikit-learn and MATLAB as the param name for convergence tolerance.


---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#discussion_r29273104
  
    --- Diff: project/SparkBuild.scala ---
    @@ -467,7 +467,7 @@ object Unidoc {
             "mllib.evaluation", "mllib.feature", "mllib.random", "mllib.stat.correlation",
             "mllib.stat.test", "mllib.tree.impl", "mllib.tree.loss",
             "ml", "ml.attribute", "ml.classification", "ml.evaluation", "ml.feature", "ml.param",
    -        "ml.tuning"
    +        "ml.recommendation", "ml.regression", "ml.tuning"
    --- End diff --
    
    This should be in a different PR.


---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#issuecomment-97607150
  
    Hopefully that did 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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#issuecomment-97607160
  
    Merged build started.


---
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-7176] [ml] Add validation functionality...

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

    https://github.com/apache/spark/pull/5740#issuecomment-97619162
  
    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