You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by zhengruifeng <gi...@git.apache.org> on 2017/12/12 11:41:48 UTC

[GitHub] spark pull request #19950: [SPARK-22450][Core][MLLib][FollowUp] safely regis...

GitHub user zhengruifeng opened a pull request:

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

    [SPARK-22450][Core][MLLib][FollowUp] safely register class for mllib - LabeledPoint/VectorWithNorm/TreePoint

    ## What changes were proposed in this pull request?
    register following classes in Kryo:
    `org.apache.spark.mllib.regression.LabeledPoint`
    `org.apache.spark.mllib.clustering.VectorWithNorm`
    `org.apache.spark.ml.feature.LabeledPoint`
    `org.apache.spark.ml.tree.impl.TreePoint`
    
    `org.apache.spark.ml.tree.impl.BaggedPoint` seems also need to be registered, but I don't know how to do it in this safe way.
    @WeichenXu123 @cloud-fan 
    
    ## How was this patch tested?
    added tests


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

    $ git pull https://github.com/zhengruifeng/spark labeled_kryo

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

    https://github.com/apache/spark/pull/19950.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 #19950
    
----
commit 5e36d1a4c941220d6277c6f6525ba4f5bddfe428
Author: Zheng RuiFeng <ru...@foxmail.com>
Date:   2017-12-11T09:23:20Z

    create pr

commit 612963633e8c05224e6955a10a0ddd3da0a91581
Author: Zheng RuiFeng <ru...@foxmail.com>
Date:   2017-12-12T11:07:03Z

    update

commit 94362ccb3e983b2be6aad41063847346294146e1
Author: Zheng RuiFeng <ru...@foxmail.com>
Date:   2017-12-12T11:28:23Z

    update

commit 0e825c5256a4ffc50f45dfacd4c8f03e350db7d8
Author: Zheng RuiFeng <ru...@foxmail.com>
Date:   2017-12-12T11:35:29Z

    update 2

----


---

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


[GitHub] spark issue #19950: [SPARK-22450][Core][MLLib][FollowUp] safely register cla...

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

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


---

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


[GitHub] spark issue #19950: [SPARK-22450][Core][MLLib][FollowUp] safely register cla...

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

    https://github.com/apache/spark/pull/19950
  
    @cloud-fan Does it works like: If A and B are any class which is registered, then Type Tuple2[A, B] will be automatically registered for kyro ?


---

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


[GitHub] spark issue #19950: [SPARK-22450][Core][MLLib][FollowUp] safely register cla...

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

    https://github.com/apache/spark/pull/19950
  
    @WeichenXu123  I am not very sure, but it seems that `Kryo` will automatic ser/deser `Tuple2[A, B]` type if both `A` and `B` have been registered:
    
    ```
    scala> import org.apache.spark.SparkConf
    import org.apache.spark.SparkConf
    
    scala> import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors}
    import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors}
    
    scala> import org.apache.spark.serializer.KryoSerializer
    import org.apache.spark.serializer.KryoSerializer
    
    scala> val conf = new SparkConf(false)
    conf: org.apache.spark.SparkConf = org.apache.spark.SparkConf@71b0289b
    
    scala> conf.set("spark.kryo.registrationRequired", "true")
    res0: org.apache.spark.SparkConf = org.apache.spark.SparkConf@71b0289b
    
    scala> val ser = new KryoSerializer(conf).newInstance()
    ser: org.apache.spark.serializer.SerializerInstance = org.apache.spark.serializer.KryoSerializerInstance@33430fc
    
    scala> class X (val values: Array[Double])
    defined class X
    
    scala> val x = new X(Array(1.0,2.0))
    x: X = X@69d58731
    
    scala> val x2 = ser.deserialize[X](ser.serialize(x))
    java.lang.IllegalArgumentException: Class is not registered: X
    Note: To register this class use: kryo.register(X.class);
      at com.esotericsoftware.kryo.Kryo.getRegistration(Kryo.java:488)
      at com.twitter.chill.KryoBase.getRegistration(KryoBase.scala:52)
      at com.esotericsoftware.kryo.util.DefaultClassResolver.writeClass(DefaultClassResolver.java:97)
      at com.esotericsoftware.kryo.Kryo.writeClass(Kryo.java:517)
      at com.esotericsoftware.kryo.Kryo.writeClassAndObject(Kryo.java:622)
      at org.apache.spark.serializer.KryoSerializerInstance.serialize(KryoSerializer.scala:346)
      ... 49 elided
    
    scala> val t1 = (1.0, Vectors.dense(Array(1.0, 2.0)))
    t1: (Double, org.apache.spark.mllib.linalg.Vector) = (1.0,[1.0,2.0])
    
    scala> val t2 = ser.deserialize[(Double, Vector)](ser.serialize(t1))
    t2: (Double, org.apache.spark.mllib.linalg.Vector) = (1.0,[1.0,2.0])
    ```



---

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


[GitHub] spark pull request #19950: [SPARK-22450][Core][MLLib][FollowUp] safely regis...

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

    https://github.com/apache/spark/pull/19950#discussion_r157922929
  
    --- Diff: core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala ---
    @@ -187,14 +187,18 @@ class KryoSerializer(conf: SparkConf)
           "org.apache.spark.mllib.linalg.Matrix",
           "org.apache.spark.mllib.linalg.DenseMatrix",
           "org.apache.spark.mllib.linalg.SparseMatrix",
    +      "org.apache.spark.mllib.regression.LabeledPoint",
    +      "org.apache.spark.mllib.clustering.VectorWithNorm",
           "org.apache.spark.ml.linalg.Vector",
           "org.apache.spark.ml.linalg.DenseVector",
           "org.apache.spark.ml.linalg.SparseVector",
           "org.apache.spark.ml.linalg.Matrix",
           "org.apache.spark.ml.linalg.DenseMatrix",
           "org.apache.spark.ml.linalg.SparseMatrix",
           "org.apache.spark.ml.feature.Instance",
    -      "org.apache.spark.ml.feature.OffsetInstance"
    +      "org.apache.spark.ml.feature.OffsetInstance",
    +      "org.apache.spark.ml.feature.LabeledPoint",
    +      "org.apache.spark.ml.tree.impl.TreePoint"
    --- End diff --
    
    I recommend to write these items in alphabet order, so we can easily check whether it miss some item and easier to add more items in the future.


---

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


[GitHub] spark issue #19950: [SPARK-22450][Core][MLLib][FollowUp] safely register cla...

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

    https://github.com/apache/spark/pull/19950
  
    **[Test build #85085 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85085/testReport)** for PR 19950 at commit [`6f51096`](https://github.com/apache/spark/commit/6f51096060645abeb0e9b9568bca6dbb6213b16a).


---

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


[GitHub] spark issue #19950: [SPARK-22450][Core][MLLib][FollowUp] safely register cla...

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

    https://github.com/apache/spark/pull/19950
  
    **[Test build #85147 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85147/testReport)** for PR 19950 at commit [`3d1d3db`](https://github.com/apache/spark/commit/3d1d3dba1e78eb9a082fa1f6a2aa2437a9ed99a6).
     * 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 #19950: [SPARK-22450][Core][MLLib][FollowUp] safely register cla...

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

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


---

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


[GitHub] spark issue #19950: [SPARK-22450][Core][MLLib][FollowUp] safely register cla...

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

    https://github.com/apache/spark/pull/19950
  
    And, these items added cannot cover the case in `MultilayerPeceptron`. Look at `FeedForwardTrainer.train`, the persisted stacked `trainData`, the format is `RDD[(Double, mllib.Vector)]`. The registered classes here do not cover this.



---

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


[GitHub] spark issue #19950: [SPARK-22450][Core][MLLib][FollowUp] safely register cla...

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

    https://github.com/apache/spark/pull/19950
  
    **[Test build #84765 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84765/testReport)** for PR 19950 at commit [`0e825c5`](https://github.com/apache/spark/commit/0e825c5256a4ffc50f45dfacd4c8f03e350db7d8).


---

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


[GitHub] spark pull request #19950: [SPARK-22450][Core][MLLib][FollowUp] safely regis...

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

    https://github.com/apache/spark/pull/19950#discussion_r157343175
  
    --- Diff: mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala ---
    @@ -311,6 +312,23 @@ class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext {
         assert(returnModel.clusterCenters(1) === initialModel.clusterCenters(1))
       }
     
    +  test("Kryo class register") {
    +    val conf = new SparkConf(false)
    +    conf.set("spark.kryo.registrationRequired", "true")
    +
    +    val ser = new KryoSerializer(conf).newInstance()
    +
    +    def check(v: VectorWithNorm): Unit = {
    --- End diff --
    
    Likewise these seem like things you can just write in a loop over several objects to test


---

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


[GitHub] spark issue #19950: [SPARK-22450][Core][MLLib][FollowUp] safely register cla...

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

    https://github.com/apache/spark/pull/19950
  
    **[Test build #85147 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85147/testReport)** for PR 19950 at commit [`3d1d3db`](https://github.com/apache/spark/commit/3d1d3dba1e78eb9a082fa1f6a2aa2437a9ed99a6).


---

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


[GitHub] spark issue #19950: [SPARK-22450][Core][MLLib][FollowUp] safely register cla...

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

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


---

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


[GitHub] spark issue #19950: [SPARK-22450][Core][MLLib][FollowUp] safely register cla...

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

    https://github.com/apache/spark/pull/19950
  
    **[Test build #84828 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84828/testReport)** for PR 19950 at commit [`024d835`](https://github.com/apache/spark/commit/024d835d4ed00f384b2f221c36c3edc656031a65).
     * 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 #19950: [SPARK-22450][Core][MLLib][FollowUp] safely register cla...

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

    https://github.com/apache/spark/pull/19950
  
    **[Test build #84824 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84824/testReport)** for PR 19950 at commit [`183868c`](https://github.com/apache/spark/commit/183868cd2a572470c512e92b212b3bc775af562f).


---

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


[GitHub] spark pull request #19950: [SPARK-22450][Core][MLLib][FollowUp] safely regis...

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

    https://github.com/apache/spark/pull/19950#discussion_r157343164
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreePointSuite.scala ---
    @@ -0,0 +1,39 @@
    +/*
    + * 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.tree.impl
    +
    +import org.apache.spark.{SparkConf, SparkFunSuite}
    +import org.apache.spark.serializer.KryoSerializer
    +
    +class TreePointSuite extends SparkFunSuite{
    +  test("Kryo class register") {
    +    val conf = new SparkConf(false)
    +    conf.set("spark.kryo.registrationRequired", "true")
    +
    +    val ser = new KryoSerializer(conf).newInstance()
    +
    +    def check(p: TreePoint): Unit = {
    --- End diff --
    
    Is there much value in defining this method?


---

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


[GitHub] spark issue #19950: [SPARK-22450][Core][MLLib][FollowUp] safely register cla...

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

    https://github.com/apache/spark/pull/19950
  
    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 #19950: [SPARK-22450][Core][MLLib][FollowUp] safely register cla...

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

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


---

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


[GitHub] spark issue #19950: [SPARK-22450][Core][MLLib][FollowUp] safely register cla...

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

    https://github.com/apache/spark/pull/19950
  
    **[Test build #85148 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85148/testReport)** for PR 19950 at commit [`604cb7d`](https://github.com/apache/spark/commit/604cb7df1d892010316c964250977c8692799666).
     * 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 #19950: [SPARK-22450][Core][MLLib][FollowUp] safely register cla...

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

    https://github.com/apache/spark/pull/19950
  
    **[Test build #84765 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84765/testReport)** for PR 19950 at commit [`0e825c5`](https://github.com/apache/spark/commit/0e825c5256a4ffc50f45dfacd4c8f03e350db7d8).
     * This patch **fails Spark unit 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 #19950: [SPARK-22450][Core][MLLib][FollowUp] safely register cla...

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

    https://github.com/apache/spark/pull/19950
  
    **[Test build #85085 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85085/testReport)** for PR 19950 at commit [`6f51096`](https://github.com/apache/spark/commit/6f51096060645abeb0e9b9568bca6dbb6213b16a).
     * 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 pull request #19950: [SPARK-22450][Core][MLLib][FollowUp] safely regis...

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

    https://github.com/apache/spark/pull/19950#discussion_r157343154
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/LabeledPointSuite.scala ---
    @@ -0,0 +1,42 @@
    +/*
    + * 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 scala.reflect.ClassTag
    +
    +import org.apache.spark.{SparkConf, SparkFunSuite}
    +import org.apache.spark.ml.linalg.Vectors
    +import org.apache.spark.serializer.KryoSerializer
    +
    +class LabeledPointSuite extends SparkFunSuite{
    --- End diff --
    
    Nit: space before brace


---

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


[GitHub] spark issue #19950: [SPARK-22450][Core][MLLib][FollowUp] safely register cla...

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

    https://github.com/apache/spark/pull/19950
  
    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 #19950: [SPARK-22450][Core][MLLib][FollowUp] safely regis...

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

    https://github.com/apache/spark/pull/19950#discussion_r157653695
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreePointSuite.scala ---
    @@ -0,0 +1,39 @@
    +/*
    + * 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.tree.impl
    +
    +import org.apache.spark.{SparkConf, SparkFunSuite}
    +import org.apache.spark.serializer.KryoSerializer
    +
    +class TreePointSuite extends SparkFunSuite{
    +  test("Kryo class register") {
    +    val conf = new SparkConf(false)
    +    conf.set("spark.kryo.registrationRequired", "true")
    +
    +    val ser = new KryoSerializer(conf).newInstance()
    +
    +    def check(p: TreePoint): Unit = {
    --- End diff --
    
    I also think there is not much value to do this, although current testsuites are all like this.


---

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


[GitHub] spark issue #19950: [SPARK-22450][Core][MLLib][FollowUp] safely register cla...

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

    https://github.com/apache/spark/pull/19950
  
    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 #19950: [SPARK-22450][Core][MLLib][FollowUp] safely register cla...

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

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


---

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


[GitHub] spark issue #19950: [SPARK-22450][Core][MLLib][FollowUp] safely register cla...

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

    https://github.com/apache/spark/pull/19950
  
    **[Test build #85148 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85148/testReport)** for PR 19950 at commit [`604cb7d`](https://github.com/apache/spark/commit/604cb7df1d892010316c964250977c8692799666).


---

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


[GitHub] spark issue #19950: [SPARK-22450][Core][MLLib][FollowUp] safely register cla...

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

    https://github.com/apache/spark/pull/19950
  
    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 #19950: [SPARK-22450][Core][MLLib][FollowUp] safely regis...

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

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


---

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


[GitHub] spark issue #19950: [SPARK-22450][Core][MLLib][FollowUp] safely register cla...

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

    https://github.com/apache/spark/pull/19950
  
    Since `VectorWithNorm` and `TreePoint` do not override method `equals`, we can not directly using `===` to compare objects.
    `LabeledPoint` is a case class, which method `equals` is automaticly supplied


---

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


[GitHub] spark issue #19950: [SPARK-22450][Core][MLLib][FollowUp] safely register cla...

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

    https://github.com/apache/spark/pull/19950
  
    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 #19950: [SPARK-22450][Core][MLLib][FollowUp] safely register cla...

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

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


---

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


[GitHub] spark issue #19950: [SPARK-22450][Core][MLLib][FollowUp] safely register cla...

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

    https://github.com/apache/spark/pull/19950
  
    **[Test build #85084 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85084/testReport)** for PR 19950 at commit [`daba630`](https://github.com/apache/spark/commit/daba630343cb1d7f3ad137a75aaffb2a29b99cb6).


---

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


[GitHub] spark issue #19950: [SPARK-22450][Core][MLLib][FollowUp] safely register cla...

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

    https://github.com/apache/spark/pull/19950
  
    **[Test build #84824 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84824/testReport)** for PR 19950 at commit [`183868c`](https://github.com/apache/spark/commit/183868cd2a572470c512e92b212b3bc775af562f).
     * This patch **fails Spark unit 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 #19950: [SPARK-22450][Core][MLLib][FollowUp] safely register cla...

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

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


---

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


[GitHub] spark issue #19950: [SPARK-22450][Core][MLLib][FollowUp] safely register cla...

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

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


---

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


[GitHub] spark issue #19950: [SPARK-22450][Core][MLLib][FollowUp] safely register cla...

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

    https://github.com/apache/spark/pull/19950
  
    Merged to master


---

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


[GitHub] spark issue #19950: [SPARK-22450][Core][MLLib][FollowUp] safely register cla...

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

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


---

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


[GitHub] spark issue #19950: [SPARK-22450][Core][MLLib][FollowUp] safely register cla...

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

    https://github.com/apache/spark/pull/19950
  
    **[Test build #84828 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84828/testReport)** for PR 19950 at commit [`024d835`](https://github.com/apache/spark/commit/024d835d4ed00f384b2f221c36c3edc656031a65).


---

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


[GitHub] spark issue #19950: [SPARK-22450][Core][MLLib][FollowUp] safely register cla...

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

    https://github.com/apache/spark/pull/19950
  
    **[Test build #85084 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85084/testReport)** for PR 19950 at commit [`daba630`](https://github.com/apache/spark/commit/daba630343cb1d7f3ad137a75aaffb2a29b99cb6).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class LabeledPointSuite extends SparkFunSuite `
      * `class TreePointSuite extends SparkFunSuite `


---

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