You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by phatak-dev <gi...@git.apache.org> on 2017/05/13 06:35:28 UTC

[GitHub] spark pull request #17972: [SPARK-20723][ML]Add intermediate storage level t...

GitHub user phatak-dev opened a pull request:

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

    [SPARK-20723][ML]Add intermediate storage level to tree based classifiers

    ## What changes were proposed in this pull request?
    
    Currently Random Forest implementation caches the intermediate data using MEMORY_AND_DISK storage level. This creates issues in low memory scenarios. So we should expose an expert param intermediateStorageLevel which allows user to customise the storage level. This is similar to als options specified in below jira
    
    https://issues.apache.org/jira/browse/SPARK-14412
    
    ## How was this patch tested?
    unit test cases.

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

    $ git pull https://github.com/phatak-dev/spark randomforestfix

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

    https://github.com/apache/spark/pull/17972.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 #17972
    
----
commit 7b555f01841ea8a9a4fc48656636328cb8f254e9
Author: madhu <ph...@gmail.com>
Date:   2017-05-13T06:30:11Z

    Add intermediate storage level to tree based classifiers

----


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

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


[GitHub] spark issue #17972: [SPARK-20723][ML]Add intermediate storage level to tree ...

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

    https://github.com/apache/spark/pull/17972
  
    I have updated the code to have the parameter in sharedParamsCodeGen.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #17972: [SPARK-20723][ML]Add intermediate storage level t...

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

    https://github.com/apache/spark/pull/17972#discussion_r118479347
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala ---
    @@ -406,4 +409,21 @@ private[ml] trait HasAggregationDepth extends Params {
       /** @group expertGetParam */
       final def getAggregationDepth: Int = $(aggregationDepth)
     }
    +
    +/**
    + * Trait for shared param intermediateStorageLevel (default: "MEMORY_AND_DISK").
    + */
    +private[ml] trait HasIntermediateStorageLevel extends Params {
    +
    +  /**
    +   * Param for Param for StorageLevelfor intermediate datasets.
    +   * @group expertParam
    +   */
    +  final val intermediateStorageLevel: Param[String] = new Param[String](this, "intermediateStorageLevel", "Param for StorageLevelfor intermediate datasets", (s: String) => Try(StorageLevel.fromString(s)).isSuccess && s != "NONE")
    --- End diff --
    
    should be a space: `StorageLevelfor` -> `StorageLevel for`


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

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


[GitHub] spark issue #17972: [SPARK-20723][ML]Add intermediate storage level to tree ...

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

    https://github.com/apache/spark/pull/17972
  
    @MLnick i am working on fixes you recommended. Meantime jenkins build doesn't seems to be 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 issue #17972: [SPARK-20723][ML]Add intermediate storage level to tree ...

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

    https://github.com/apache/spark/pull/17972
  
    There are around 33 binary incompatibility issues as I made this a shared param. Some of them from build logs are below
    ```scala
    [error]  * abstract synthetic method org$apache$spark$ml$param$shared$HasIntermediateStorageLevel$_setter_$intermediateStorageLevel_=(org.apache.spark.ml.param.Param)Unit in trait org.apache.spark.ml.param.shared.HasIntermediateStorageLevel is inherited by class ALSParams in current version.
    [error]    filter with: ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasIntermediateStorageLevel.org$apache$spark$ml$param$shared$HasIntermediateStorageLevel$_setter_$intermediateStorageLevel_=")
    [error]  * method getIntermediateStorageLevel()java.lang.String in class org.apache.spark.ml.recommendation.ALS is declared final in current version
    [error]    filter with: ProblemFilters.exclude[FinalMethodProblem]("org.apache.spark.ml.recommendation.ALS.getIntermediateStorageLevel")
    [error]  * method intermediateStorageLevel()org.apache.spark.ml.param.Param in class org.apache.spark.ml.recommendation.ALS is declared final in current version
    [error]    filter with: ProblemFilters.exclude[FinalMethodProblem]("org.apache.spark.ml.recommendation.ALS.intermediateStorageLevel")
    [error]  * abstract synthetic method org$apache$spark$ml$param$shared$HasIntermediateStorageLevel$_setter_$intermediateStorageLevel_=(org.apache.spark.ml.param.Param)Unit in trait org.apache.spark.ml.param.shared.HasIntermediateStorageLevel is inherited by class RandomForestRegressorParams in current version.
    ```
    Should I add them to the exclude file?


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

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


[GitHub] spark issue #17972: [SPARK-20723][ML]Add intermediate storage level to tree ...

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

    https://github.com/apache/spark/pull/17972
  
    **[Test build #77498 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77498/testReport)** for PR 17972 at commit [`4e48f5c`](https://github.com/apache/spark/commit/4e48f5c42d746ac0ab48c30e87d781e47bab4820).


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

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


[GitHub] spark issue #17972: [SPARK-20723][ML]Add intermediate storage level to tree ...

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

    https://github.com/apache/spark/pull/17972
  
    @WeichenXu123 resolved merge conflicts. Can you initiate jenkins build?


---

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


[GitHub] spark issue #17972: [SPARK-20723][ML]Add intermediate storage level to tree ...

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

    https://github.com/apache/spark/pull/17972
  
    If other algos are going to use this param, we should put in in shared params - i.e. update `SharedParamsCodeGen` and have the relevant estimators inherit from the trait.


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

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


[GitHub] spark issue #17972: [SPARK-20723][ML]Add intermediate storage level to tree ...

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

    https://github.com/apache/spark/pull/17972
  
    Ok to test


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

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


[GitHub] spark issue #17972: [SPARK-20723][ML]Add intermediate storage level to tree ...

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

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


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

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


[GitHub] spark issue #17972: [SPARK-20723][ML]Add intermediate storage level to tree ...

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

    https://github.com/apache/spark/pull/17972
  
    jenkins test this please


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

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


[GitHub] spark issue #17972: [SPARK-20723][ML]Add intermediate storage level to tree ...

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

    https://github.com/apache/spark/pull/17972
  
    Yes let's make it shared if possible.


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

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


[GitHub] spark issue #17972: [SPARK-20723][ML]Add intermediate storage level to tree ...

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

    https://github.com/apache/spark/pull/17972
  
    Can one of the admins verify this patch?


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

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


[GitHub] spark issue #17972: [SPARK-20723][ML]Add intermediate storage level to tree ...

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

    https://github.com/apache/spark/pull/17972
  
    Have you checked other algorithms which can also apply this parameter ?


---

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


[GitHub] spark issue #17972: [SPARK-20723][ML]Add intermediate storage level to tree ...

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

    https://github.com/apache/spark/pull/17972
  
    @MLnick should i go ahead and move it to SharedParamsCodeGen?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #17972: [SPARK-20723][ML]Add intermediate storage level t...

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

    https://github.com/apache/spark/pull/17972#discussion_r118479868
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala ---
    @@ -199,7 +199,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext {
         withClue("DecisionTree requires number of features > 0," +
           " but was given an empty features vector") {
           intercept[IllegalArgumentException] {
    -        RandomForest.run(rdd, strategy, 1, "all", 42L, instr = None)
    +        RandomForest.run(rdd, strategy, 1, "all", 42L, instr = None, "MEMORY_AND_DISK")
    --- End diff --
    
    extract out the storage level `MEMORY_AND_DISK` into a `val` that can be reused


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

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


[GitHub] spark issue #17972: [SPARK-20723][ML]Add intermediate storage level to tree ...

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

    https://github.com/apache/spark/pull/17972
  
    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 #17972: [SPARK-20723][ML]Add intermediate storage level t...

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

    https://github.com/apache/spark/pull/17972#discussion_r118479964
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala ---
    @@ -398,6 +398,14 @@ class DecisionTreeClassifierSuite
     
         testDefaultReadWrite(model)
       }
    +
    +  test("intermediate dataset storage level") {
    --- End diff --
    
    Ideally we need to test that the actual storage levels for the intermediate RDDs are correct. I did this for ALS - not sure if the same approach might be used here.


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

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


[GitHub] spark issue #17972: [SPARK-20723][ML]Add intermediate storage level to tree ...

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

    https://github.com/apache/spark/pull/17972
  
    Can one of the admins verify this patch?


---

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


[GitHub] spark issue #17972: [SPARK-20723][ML]Add intermediate storage level to tree ...

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

    https://github.com/apache/spark/pull/17972
  
    **[Test build #77498 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77498/testReport)** for PR 17972 at commit [`4e48f5c`](https://github.com/apache/spark/commit/4e48f5c42d746ac0ab48c30e87d781e47bab4820).
     * This patch **fails MiMa tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


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

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


[GitHub] spark issue #17972: [SPARK-20723][ML]Add intermediate storage level to tree ...

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

    https://github.com/apache/spark/pull/17972
  
    @MLnick can you start a jenkins build?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #17972: [SPARK-20723][ML]Add intermediate storage level t...

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

    https://github.com/apache/spark/pull/17972#discussion_r150470524
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala ---
    @@ -129,7 +129,7 @@ private[recommendation] trait ALSModelParams extends Params with HasPredictionCo
      * Common params for ALS.
      */
     private[recommendation] trait ALSParams extends ALSModelParams with HasMaxIter with HasRegParam
    -  with HasPredictionCol with HasCheckpointInterval with HasSeed {
    +  with HasPredictionCol with HasCheckpointInterval with HasSeed with HasIntermediateStorageLevel{
    --- End diff --
    
    space: `HasIntermediateStorageLevel {`


---

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


[GitHub] spark issue #17972: [SPARK-20723][ML]Add intermediate storage level to tree ...

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

    https://github.com/apache/spark/pull/17972
  
    I have added to all the algorithms I know that uses intermediate state.


---

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


[GitHub] spark issue #17972: [SPARK-20723][ML]Add intermediate storage level to tree ...

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

    https://github.com/apache/spark/pull/17972
  
    Can one of the admins verify this patch?


---

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


[GitHub] spark issue #17972: [SPARK-20723][ML]Add intermediate storage level to tree ...

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

    https://github.com/apache/spark/pull/17972
  
    perhaps related: 
    Big Random Forest Models (example: 100 or more trees with depth of around 20):
    
    
    
    Big models can be trained effectively even on machines with limited RAM (such as C series in AWS). However, they fail during .transform stage. My guess is that it tries to load model to RAM of individual worker nodes which do not have that much capacity. Above proposed fix (relying on DISK as well) can potentially fix this.
    
    An alternative fix is to load model tree by tree evaluate results and store evaluation/transform. This way, we may not need the Disk storage yet and it would better resemble the training process (also tree by tree).
    
    



---

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


[GitHub] spark issue #17972: [SPARK-20723][ML]Add intermediate storage level to tree ...

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

    https://github.com/apache/spark/pull/17972
  
    Can you please initiate the Jenkins build?


---

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


[GitHub] spark issue #17972: [SPARK-20723][ML]Add intermediate storage level to tree ...

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

    https://github.com/apache/spark/pull/17972
  
    @MLnick Any updates on this?


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

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


[GitHub] spark issue #17972: [SPARK-20723][ML]Add intermediate storage level to tree ...

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

    https://github.com/apache/spark/pull/17972
  
    Can one of the admins verify this patch?


---

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


[GitHub] spark issue #17972: [SPARK-20723][ML]Add intermediate storage level to tree ...

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

    https://github.com/apache/spark/pull/17972
  
    @WeichenXu123 Done the changes to mima.


---

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


[GitHub] spark issue #17972: [SPARK-20723][ML]Add intermediate storage level to tree ...

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

    https://github.com/apache/spark/pull/17972
  
    Current unit test case is rudimentary. Any help in improving it is appreciated.


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

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