Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16020#discussion_r89740051
  
    --- Diff: 
mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala ---
    @@ -255,10 +256,19 @@ class BisectingKMeans @Since("2.0.0") (
     
       @Since("2.0.0")
       override def fit(dataset: Dataset[_]): BisectingKMeansModel = {
    +    val handlePersistence = dataset.rdd.getStorageLevel == 
StorageLevel.NONE
    --- End diff --
    
    By the way, I've been meaning to log a ticket for this issue, but have been 
tied up.
    
    This will actually never work. `dataset.rdd` will always have storage level 
`NONE`. To see this:
    
    ```
    scala> import org.apache.spark.storage.StorageLevel
    import org.apache.spark.storage.StorageLevel
    
    scala> val df = spark.range(10).toDF("num")
    df: org.apache.spark.sql.DataFrame = [num: bigint]
    
    scala> df.storageLevel == StorageLevel.NONE
    res0: Boolean = true
    
    scala> df.persist
    res1: df.type = [num: bigint]
    
    scala> df.storageLevel == StorageLevel.MEMORY_AND_DISK
    res2: Boolean = true
    
    scala> df.rdd.getStorageLevel == StorageLevel.MEMORY_AND_DISK
    res3: Boolean = false
    
    scala> df.rdd.getStorageLevel == StorageLevel.NONE
    res4: Boolean = true
    ```
    
    So in fact all the algorithms that are checking for storage level using 
`dataset.rdd` are actually double-caching the data if the input DataFrame is 
actually cached, because the RDD will not appear to be cached.
    
    So we should migrate all the checks to use `dataset.storageLevel` which was 
added in https://github.com/apache/spark/pull/13780


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

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

Reply via email to