[ https://issues.apache.org/jira/browse/SPARK-18608?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Joseph K. Bradley updated SPARK-18608: -------------------------------------- Target Version/s: 2.2.1, 2.3.0 > Spark ML algorithms that check RDD cache level for internal caching > double-cache data > ------------------------------------------------------------------------------------- > > Key: SPARK-18608 > URL: https://issues.apache.org/jira/browse/SPARK-18608 > Project: Spark > Issue Type: Bug > Components: ML > Reporter: Nick Pentreath > Assignee: zhengruifeng > > Some algorithms in Spark ML (e.g. {{LogisticRegression}}, > {{LinearRegression}}, and I believe now {{KMeans}}) handle persistence > internally. They check whether the input dataset is cached, and if not they > cache it for performance. > However, the check is done using {{dataset.rdd.getStorageLevel == NONE}}. > This will actually always be true, since even if the dataset itself is > cached, the RDD returned by {{dataset.rdd}} will not be cached. > Hence if the input dataset is cached, the data will end up being cached > twice, which is wasteful. > To see this: > {code} > 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 > {code} > Before SPARK-16063, there was no way to check the storage level of the input > {{DataSet}}, but now we can, so the checks should be migrated to use > {{dataset.storageLevel}}. -- This message was sent by Atlassian JIRA (v6.4.14#64029) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org