Skip to content
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 3 additions & 1 deletion sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
Original file line number Diff line number Diff line change
Expand Up @@ -3833,7 +3833,9 @@ class Dataset[T] private[sql](
* @since 1.6.0
*/
def persist(newLevel: StorageLevel): this.type = {
sparkSession.sharedState.cacheManager.cacheQuery(this, None, newLevel)
if (newLevel != StorageLevel.NONE) {
sparkSession.sharedState.cacheManager.cacheQuery(this, None, newLevel)
}
this
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@ import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
import org.apache.spark.storage.StorageLevel

case class TestDataPoint(x: Int, y: Double, s: String, t: TestDataPoint2)
case class TestDataPoint2(x: Int, s: String)
Expand Down Expand Up @@ -2597,6 +2598,11 @@ class DatasetSuite extends QueryTest
parameters = Map("cls" -> classOf[Array[Int]].getName))
}
}

test("SPARK-45386: persist with StorageLevel.NONE should give correct count") {
val ds = Seq(1, 2).toDS().persist(StorageLevel.NONE)
assert(ds.count() == 2)
}
}

class DatasetLargeResultCollectingSuite extends QueryTest
Expand Down