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 2017/12/29 03:42:22 UTC

[GitHub] spark pull request #20111: [SPARK-22883][ML][TEST] Streaming tests for spark...

GitHub user jkbradley opened a pull request:

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

    [SPARK-22883][ML][TEST] Streaming tests for spark.ml.feature, from A to H

    ## What changes were proposed in this pull request?
    
    Adds structured streaming tests using testTransformer for these suites:
    * BinarizerSuite
    * BucketedRandomProjectionLSHSuite
    * BucketizerSuite
    * ChiSqSelectorSuite
    * CountVectorizerSuite
    * DCTSuite.scala
    * ElementwiseProductSuite
    * FeatureHasherSuite
    * HashingTFSuite
    
    ## How was this patch tested?
    
    It tests itself because it is a bunch of tests!

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

    $ git pull https://github.com/jkbradley/spark SPARK-22883-streaming-featureAM

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

    https://github.com/apache/spark/pull/20111.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 #20111
    
----
commit 12b3dcf13f90ea00c2a12ec186a5f3277e812095
Author: Joseph K. Bradley <jo...@...>
Date:   2017-12-29T03:31:17Z

    added streaming tests for first quarter of spark.ml.feature

----


---

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


[GitHub] spark issue #20111: [SPARK-22883][ML][TEST] Streaming tests for spark.ml.fea...

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

    https://github.com/apache/spark/pull/20111
  
    Merging with master


---

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


[GitHub] spark issue #20111: [SPARK-22883][ML][TEST] Streaming tests for spark.ml.fea...

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

    https://github.com/apache/spark/pull/20111
  
    **[Test build #85494 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85494/testReport)** for PR 20111 at commit [`12b3dcf`](https://github.com/apache/spark/commit/12b3dcf13f90ea00c2a12ec186a5f3277e812095).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class BinarizerSuite extends MLTest with DefaultReadWriteTest `
      * `class BucketedRandomProjectionLSHSuite extends MLTest with DefaultReadWriteTest `
      * `class BucketizerSuite extends MLTest with DefaultReadWriteTest `
      * `class ChiSqSelectorSuite extends MLTest with DefaultReadWriteTest `
      * `class CountVectorizerSuite extends MLTest with DefaultReadWriteTest `
      * `class DCTSuite extends MLTest with DefaultReadWriteTest `
      * `class ElementwiseProductSuite extends MLTest with DefaultReadWriteTest `
      * `class FeatureHasherSuite extends MLTest with DefaultReadWriteTest `
      * `class HashingTFSuite extends MLTest with DefaultReadWriteTest `


---

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


[GitHub] spark issue #20111: [SPARK-22883][ML][TEST] Streaming tests for spark.ml.fea...

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

    https://github.com/apache/spark/pull/20111
  
    LGTM except a tiny issue. :)


---

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


[GitHub] spark pull request #20111: [SPARK-22883][ML][TEST] Streaming tests for spark...

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

    https://github.com/apache/spark/pull/20111#discussion_r159106208
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/BucketedRandomProjectionLSHSuite.scala ---
    @@ -98,6 +97,21 @@ class BucketedRandomProjectionLSHSuite
         MLTestingUtils.checkCopyAndUids(brp, brpModel)
       }
     
    +  test("BucketedRandomProjectionLSH: streaming transform") {
    +    val brp = new BucketedRandomProjectionLSH()
    +      .setNumHashTables(2)
    +      .setInputCol("keys")
    +      .setOutputCol("values")
    +      .setBucketLength(1.0)
    +      .setSeed(12345)
    +    val brpModel = brp.fit(dataset)
    +
    +    testTransformer[Tuple1[Vector]](dataset.toDF(), brpModel, "keys", "values") {
    +      case Row(_: Vector, values: Seq[_]) =>
    --- End diff --
    
    I don't think we have to.  The main thing here is to make sure that the transform really does happen.  Other tests check validity of the values.


---

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


[GitHub] spark issue #20111: [SPARK-22883][ML][TEST] Streaming tests for spark.ml.fea...

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

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


---

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


[GitHub] spark pull request #20111: [SPARK-22883][ML][TEST] Streaming tests for spark...

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

    https://github.com/apache/spark/pull/20111#discussion_r159022766
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/ElementwiseProductSuite.scala ---
    @@ -17,13 +17,31 @@
     
     package org.apache.spark.ml.feature
     
    -import org.apache.spark.SparkFunSuite
    -import org.apache.spark.ml.linalg.Vectors
    -import org.apache.spark.ml.util.DefaultReadWriteTest
    -import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.ml.linalg.{Vector, Vectors}
    +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest}
    +import org.apache.spark.ml.util.TestingUtils._
    +import org.apache.spark.sql.Row
     
    -class ElementwiseProductSuite
    -  extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +class ElementwiseProductSuite extends MLTest with DefaultReadWriteTest {
    +
    +  import testImplicits._
    +
    +  test("streaming transform") {
    --- End diff --
    
    No existing unit test to use


---

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


[GitHub] spark issue #20111: [SPARK-22883][ML][TEST] Streaming tests for spark.ml.fea...

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

    https://github.com/apache/spark/pull/20111
  
    **[Test build #85789 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85789/testReport)** for PR 20111 at commit [`12b3dcf`](https://github.com/apache/spark/commit/12b3dcf13f90ea00c2a12ec186a5f3277e812095).


---

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


[GitHub] spark issue #20111: [SPARK-22883][ML][TEST] Streaming tests for spark.ml.fea...

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

    https://github.com/apache/spark/pull/20111
  
    Updated!  Thanks @WeichenXu123 -- I'll merge this once tests pass.


---

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


[GitHub] spark issue #20111: [SPARK-22883][ML][TEST] Streaming tests for spark.ml.fea...

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

    https://github.com/apache/spark/pull/20111
  
    **[Test build #85494 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85494/testReport)** for PR 20111 at commit [`12b3dcf`](https://github.com/apache/spark/commit/12b3dcf13f90ea00c2a12ec186a5f3277e812095).


---

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


[GitHub] spark issue #20111: [SPARK-22883][ML][TEST] Streaming tests for spark.ml.fea...

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

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


---

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


[GitHub] spark issue #20111: [SPARK-22883][ML][TEST] Streaming tests for spark.ml.fea...

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

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


---

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


[GitHub] spark issue #20111: [SPARK-22883][ML][TEST] Streaming tests for spark.ml.fea...

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

    https://github.com/apache/spark/pull/20111
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/1207/
    Test PASSed.


---

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


[GitHub] spark pull request #20111: [SPARK-22883][ML][TEST] Streaming tests for spark...

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

    https://github.com/apache/spark/pull/20111#discussion_r159027879
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/HashingTFSuite.scala ---
    @@ -37,21 +36,28 @@ class HashingTFSuite extends SparkFunSuite with MLlibTestSparkContext with Defau
       }
     
       test("hashingTF") {
    --- End diff --
    
    ditto: rearranged to do validity check per-row


---

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


[GitHub] spark issue #20111: [SPARK-22883][ML][TEST] Streaming tests for spark.ml.fea...

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

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


---

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


[GitHub] spark pull request #20111: [SPARK-22883][ML][TEST] Streaming tests for spark...

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

    https://github.com/apache/spark/pull/20111#discussion_r159022657
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/BucketedRandomProjectionLSHSuite.scala ---
    @@ -98,6 +97,21 @@ class BucketedRandomProjectionLSHSuite
         MLTestingUtils.checkCopyAndUids(brp, brpModel)
       }
     
    +  test("BucketedRandomProjectionLSH: streaming transform") {
    --- End diff --
    
    No existing test to use


---

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


[GitHub] spark pull request #20111: [SPARK-22883][ML][TEST] Streaming tests for spark...

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

    https://github.com/apache/spark/pull/20111#discussion_r159027858
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/FeatureHasherSuite.scala ---
    @@ -51,31 +48,31 @@ class FeatureHasherSuite extends SparkFunSuite
       }
     
       test("feature hashing") {
    --- End diff --
    
    Rearranged this test so it checks each row independently.


---

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


[GitHub] spark issue #20111: [SPARK-22883][ML][TEST] Streaming tests for spark.ml.fea...

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

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


---

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


[GitHub] spark pull request #20111: [SPARK-22883][ML][TEST] Streaming tests for spark...

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

    https://github.com/apache/spark/pull/20111#discussion_r159116537
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/BucketedRandomProjectionLSHSuite.scala ---
    @@ -98,6 +97,21 @@ class BucketedRandomProjectionLSHSuite
         MLTestingUtils.checkCopyAndUids(brp, brpModel)
       }
     
    +  test("BucketedRandomProjectionLSH: streaming transform") {
    +    val brp = new BucketedRandomProjectionLSH()
    +      .setNumHashTables(2)
    +      .setInputCol("keys")
    +      .setOutputCol("values")
    +      .setBucketLength(1.0)
    +      .setSeed(12345)
    +    val brpModel = brp.fit(dataset)
    +
    +    testTransformer[Tuple1[Vector]](dataset.toDF(), brpModel, "keys", "values") {
    +      case Row(_: Vector, values: Seq[_]) =>
    --- End diff --
    
    OK. so I prefer to use simpler code:
    ```
    testTransformer[Tuple1[Vector]](dataset.toDF(), brpModel, "values") {
        case Row(values: Seq[_]) =>
    ...
    ```


---

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


[GitHub] spark pull request #20111: [SPARK-22883][ML][TEST] Streaming tests for spark...

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

    https://github.com/apache/spark/pull/20111#discussion_r159022677
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala ---
    @@ -163,18 +162,19 @@ class ChiSqSelectorSuite extends SparkFunSuite with MLlibTestSparkContext
             assert(expected.selectedFeatures === actual.selectedFeatures)
           }
       }
    -}
     
    -object ChiSqSelectorSuite {
    -
    -  private def testSelector(selector: ChiSqSelector, dataset: Dataset[_]): ChiSqSelectorModel = {
    -    val selectorModel = selector.fit(dataset)
    -    selectorModel.transform(dataset).select("filtered", "topFeature").collect()
    -      .foreach { case Row(vec1: Vector, vec2: Vector) =>
    +  private def testSelector(selector: ChiSqSelector, data: Dataset[_]): ChiSqSelectorModel = {
    --- End diff --
    
    Moved from object to class b/c this needed testTransformer from the MLTest mix-in


---

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


[GitHub] spark issue #20111: [SPARK-22883][ML][TEST] Streaming tests for spark.ml.fea...

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

    https://github.com/apache/spark/pull/20111
  
    **[Test build #87857 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87857/testReport)** for PR 20111 at commit [`448668d`](https://github.com/apache/spark/commit/448668d73269d2be5ed8e54c33753691767dd90f).


---

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


[GitHub] spark pull request #20111: [SPARK-22883][ML][TEST] Streaming tests for spark...

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

    https://github.com/apache/spark/pull/20111#discussion_r159022777
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/FeatureHasherSuite.scala ---
    @@ -17,26 +17,23 @@
     
     package org.apache.spark.ml.feature
     
    -import org.apache.spark.SparkFunSuite
     import org.apache.spark.ml.attribute.AttributeGroup
     import org.apache.spark.ml.linalg.{Vector, Vectors}
     import org.apache.spark.ml.param.ParamsSuite
    -import org.apache.spark.ml.util.DefaultReadWriteTest
    +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest}
     import org.apache.spark.ml.util.TestingUtils._
    -import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.sql.Row
     import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
     import org.apache.spark.sql.functions.col
     import org.apache.spark.sql.types._
     
    -class FeatureHasherSuite extends SparkFunSuite
    -  with MLlibTestSparkContext
    -  with DefaultReadWriteTest {
    +class FeatureHasherSuite extends MLTest with DefaultReadWriteTest {
     
       import testImplicits._
     
       import HashingTFSuite.murmur3FeatureIdx
     
    -  implicit private val vectorEncoder = ExpressionEncoder[Vector]()
    +  implicit private val vectorEncoder: ExpressionEncoder[Vector] = ExpressionEncoder[Vector]()
    --- End diff --
    
    scala style


---

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


[GitHub] spark issue #20111: [SPARK-22883][ML][TEST] Streaming tests for spark.ml.fea...

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

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


---

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


[GitHub] spark issue #20111: [SPARK-22883][ML][TEST] Streaming tests for spark.ml.fea...

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

    https://github.com/apache/spark/pull/20111
  
    **[Test build #85789 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85789/testReport)** for PR 20111 at commit [`12b3dcf`](https://github.com/apache/spark/commit/12b3dcf13f90ea00c2a12ec186a5f3277e812095).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class BinarizerSuite extends MLTest with DefaultReadWriteTest `
      * `class BucketedRandomProjectionLSHSuite extends MLTest with DefaultReadWriteTest `
      * `class BucketizerSuite extends MLTest with DefaultReadWriteTest `
      * `class ChiSqSelectorSuite extends MLTest with DefaultReadWriteTest `
      * `class CountVectorizerSuite extends MLTest with DefaultReadWriteTest `
      * `class DCTSuite extends MLTest with DefaultReadWriteTest `
      * `class ElementwiseProductSuite extends MLTest with DefaultReadWriteTest `
      * `class FeatureHasherSuite extends MLTest with DefaultReadWriteTest `
      * `class HashingTFSuite extends MLTest with DefaultReadWriteTest `


---

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


[GitHub] spark pull request #20111: [SPARK-22883][ML][TEST] Streaming tests for spark...

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

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


---

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


[GitHub] spark issue #20111: [SPARK-22883][ML][TEST] Streaming tests for spark.ml.fea...

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

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


---

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


[GitHub] spark pull request #20111: [SPARK-22883][ML][TEST] Streaming tests for spark...

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

    https://github.com/apache/spark/pull/20111#discussion_r159048079
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/BucketedRandomProjectionLSHSuite.scala ---
    @@ -98,6 +97,21 @@ class BucketedRandomProjectionLSHSuite
         MLTestingUtils.checkCopyAndUids(brp, brpModel)
       }
     
    +  test("BucketedRandomProjectionLSH: streaming transform") {
    +    val brp = new BucketedRandomProjectionLSH()
    +      .setNumHashTables(2)
    +      .setInputCol("keys")
    +      .setOutputCol("values")
    +      .setBucketLength(1.0)
    +      .setSeed(12345)
    +    val brpModel = brp.fit(dataset)
    +
    +    testTransformer[Tuple1[Vector]](dataset.toDF(), brpModel, "keys", "values") {
    +      case Row(_: Vector, values: Seq[_]) =>
    --- End diff --
    
    So here do not need to select "keys" column ?


---

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


[GitHub] spark issue #20111: [SPARK-22883][ML][TEST] Streaming tests for spark.ml.fea...

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

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


---

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