Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
25 commits
Select commit Hold shift + click to select a range
01e4cdf
Merge remote-tracking branch 'upstream/master'
gatorsmile Nov 13, 2015
6835704
Merge remote-tracking branch 'upstream/master'
gatorsmile Nov 14, 2015
9180687
Merge remote-tracking branch 'upstream/master'
gatorsmile Nov 14, 2015
b38a21e
SPARK-11633
gatorsmile Nov 17, 2015
d2b84af
Merge remote-tracking branch 'upstream/master' into joinMakeCopy
gatorsmile Nov 17, 2015
fda8025
Merge remote-tracking branch 'upstream/master'
gatorspark Nov 17, 2015
ac0dccd
Merge branch 'master' of https://github.com/gatorsmile/spark
gatorspark Nov 17, 2015
6e0018b
Merge remote-tracking branch 'upstream/master'
Nov 20, 2015
0546772
converge
gatorsmile Nov 20, 2015
b37a64f
converge
gatorsmile Nov 20, 2015
f061671
Support Persist/Cache and Unpersist in DataSet APIs
gatorsmile Nov 22, 2015
88d5e9d
Merge remote-tracking branch 'upstream/master' into top
gatorsmile Nov 22, 2015
c135e1f
update the @since
gatorsmile Nov 22, 2015
661260b
Merge remote-tracking branch 'upstream/master'
gatorsmile Nov 23, 2015
2517777
adding more test cases
gatorsmile Nov 23, 2015
aa5dc52
Merge remote-tracking branch 'upstream/master' into top
gatorsmile Nov 25, 2015
2dfa0fd
Merge remote-tracking branch 'upstream/master'
gatorsmile Nov 25, 2015
c4489ed
Merge remote-tracking branch 'upstream/master' into top
gatorsmile Nov 25, 2015
683fa6f
resolved all the comments
gatorsmile Nov 25, 2015
1c82396
Merge remote-tracking branch 'upstream/master' into top
gatorsmile Nov 25, 2015
d929d9b
Merge remote-tracking branch 'upstream/master'
gatorsmile Nov 25, 2015
92ede39
Merge branch 'top' into persistDSmerge
gatorsmile Nov 25, 2015
8071d30
Merge remote-tracking branch 'upstream/master' into persistDSmerge
gatorsmile Dec 1, 2015
b9518ee
updated the codes based on the review comments from Michale Armbrust.
gatorsmile Dec 1, 2015
b8d287a
Changed the name from CacheSuite.scala to DatasetCacheSuite.scala
gatorsmile Dec 1, 2015
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
9 changes: 9 additions & 0 deletions sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
Original file line number Diff line number Diff line change
Expand Up @@ -1584,6 +1584,7 @@ class DataFrame private[sql](
def distinct(): DataFrame = dropDuplicates()

/**
* Persist this [[DataFrame]] with the default storage level (`MEMORY_AND_DISK`).
* @group basic
* @since 1.3.0
*/
Expand All @@ -1593,12 +1594,17 @@ class DataFrame private[sql](
}

/**
* Persist this [[DataFrame]] with the default storage level (`MEMORY_AND_DISK`).
* @group basic
* @since 1.3.0
*/
def cache(): this.type = persist()

/**
* Persist this [[DataFrame]] with the given storage level.
* @param newLevel One of: `MEMORY_ONLY`, `MEMORY_AND_DISK`, `MEMORY_ONLY_SER`,
* `MEMORY_AND_DISK_SER`, `DISK_ONLY`, `MEMORY_ONLY_2`,
* `MEMORY_AND_DISK_2`, etc.
* @group basic
* @since 1.3.0
*/
Expand All @@ -1608,6 +1614,8 @@ class DataFrame private[sql](
}

/**
* Mark the [[DataFrame]] as non-persistent, and remove all blocks for it from memory and disk.
* @param blocking Whether to block until all blocks are deleted.
* @group basic
* @since 1.3.0
*/
Expand All @@ -1617,6 +1625,7 @@ class DataFrame private[sql](
}

/**
* Mark the [[DataFrame]] as non-persistent, and remove all blocks for it from memory and disk.
* @group basic
* @since 1.3.0
*/
Expand Down
50 changes: 47 additions & 3 deletions sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.plans.JoinType
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.execution.{Queryable, QueryExecution}
import org.apache.spark.sql.types.StructType
import org.apache.spark.storage.StorageLevel
import org.apache.spark.util.Utils

/**
Expand Down Expand Up @@ -565,7 +566,7 @@ class Dataset[T] private[sql](
* combined.
*
* Note that, this function is not a typical set union operation, in that it does not eliminate
* duplicate items. As such, it is analagous to `UNION ALL` in SQL.
* duplicate items. As such, it is analogous to `UNION ALL` in SQL.
* @since 1.6.0
*/
def union(other: Dataset[T]): Dataset[T] = withPlan[T](other)(Union)
Expand Down Expand Up @@ -618,7 +619,6 @@ class Dataset[T] private[sql](
case _ => Alias(CreateStruct(rightOutput), "_2")()
}


implicit val tuple2Encoder: Encoder[(T, U)] =
ExpressionEncoder.tuple(this.unresolvedTEncoder, other.unresolvedTEncoder)
withPlan[(T, U)](other) { (left, right) =>
Expand Down Expand Up @@ -697,11 +697,55 @@ class Dataset[T] private[sql](
*/
def takeAsList(num: Int): java.util.List[T] = java.util.Arrays.asList(take(num) : _*)

/**
* Persist this [[Dataset]] with the default storage level (`MEMORY_AND_DISK`).
* @since 1.6.0
*/
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The comment style here is off and we should actually have a description. Could we just move the functions/docs from DataFrame to Queryable?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

So far, we are unable to move the functions to Queryable because the types of the returned values are different. I just added the descriptions in both DataFrame and Dataset. Hopefully, it resolves your concern. Thanks!

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@marmbrus moving functions into Queryable actually breaks both scaladoc and javadoc.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@rxin I think thats only because we explicitly exclude execution from scaladoc. Maybe we should move queryable? or don't exclude that class. I don't want to duplicate a ton of docs.

def persist(): this.type = {
sqlContext.cacheManager.cacheQuery(this)
this
}

/**
* Persist this [[Dataset]] with the default storage level (`MEMORY_AND_DISK`).
* @since 1.6.0
*/
def cache(): this.type = persist()

/**
* Persist this [[Dataset]] with the given storage level.
* @param newLevel One of: `MEMORY_ONLY`, `MEMORY_AND_DISK`, `MEMORY_ONLY_SER`,
* `MEMORY_AND_DISK_SER`, `DISK_ONLY`, `MEMORY_ONLY_2`,
* `MEMORY_AND_DISK_2`, etc.
* @group basic
* @since 1.6.0
*/
def persist(newLevel: StorageLevel): this.type = {
sqlContext.cacheManager.cacheQuery(this, None, newLevel)
this
}

/**
* Mark the [[Dataset]] as non-persistent, and remove all blocks for it from memory and disk.
* @param blocking Whether to block until all blocks are deleted.
* @since 1.6.0
*/
def unpersist(blocking: Boolean): this.type = {
sqlContext.cacheManager.tryUncacheQuery(this, blocking)
this
}

/**
* Mark the [[Dataset]] as non-persistent, and remove all blocks for it from memory and disk.
* @since 1.6.0
*/
def unpersist(): this.type = unpersist(blocking = false)

/* ******************** *
* Internal Functions *
* ******************** */

private[sql] def logicalPlan = queryExecution.analyzed
private[sql] def logicalPlan: LogicalPlan = queryExecution.analyzed

private[sql] def withPlan(f: LogicalPlan => LogicalPlan): Dataset[T] =
new Dataset[T](sqlContext, sqlContext.executePlan(f(logicalPlan)), tEncoder)
Expand Down
9 changes: 9 additions & 0 deletions sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -338,6 +338,15 @@ class SQLContext private[sql](
cacheManager.lookupCachedData(table(tableName)).nonEmpty
}

/**
* Returns true if the [[Queryable]] is currently cached in-memory.
* @group cachemgmt
* @since 1.3.0
*/
private[sql] def isCached(qName: Queryable): Boolean = {
cacheManager.lookupCachedData(qName).nonEmpty
}

/**
* Caches the specified table in-memory.
* @group cachemgmt
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@ package org.apache.spark.sql.execution
import java.util.concurrent.locks.ReentrantReadWriteLock

import org.apache.spark.Logging
import org.apache.spark.sql.DataFrame
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.execution.columnar.InMemoryRelation
import org.apache.spark.storage.StorageLevel
Expand Down Expand Up @@ -75,12 +74,12 @@ private[sql] class CacheManager extends Logging {
}

/**
* Caches the data produced by the logical representation of the given [[DataFrame]]. Unlike
* `RDD.cache()`, the default storage level is set to be `MEMORY_AND_DISK` because recomputing
* the in-memory columnar representation of the underlying table is expensive.
* Caches the data produced by the logical representation of the given [[Queryable]].
* Unlike `RDD.cache()`, the default storage level is set to be `MEMORY_AND_DISK` because
* recomputing the in-memory columnar representation of the underlying table is expensive.
*/
private[sql] def cacheQuery(
query: DataFrame,
query: Queryable,
tableName: Option[String] = None,
storageLevel: StorageLevel = MEMORY_AND_DISK): Unit = writeLock {
val planToCache = query.queryExecution.analyzed
Expand All @@ -95,23 +94,25 @@ private[sql] class CacheManager extends Logging {
sqlContext.conf.useCompression,
sqlContext.conf.columnBatchSize,
storageLevel,
sqlContext.executePlan(query.logicalPlan).executedPlan,
sqlContext.executePlan(planToCache).executedPlan,
tableName))
}
}

/** Removes the data for the given [[DataFrame]] from the cache */
private[sql] def uncacheQuery(query: DataFrame, blocking: Boolean = true): Unit = writeLock {
/** Removes the data for the given [[Queryable]] from the cache */
private[sql] def uncacheQuery(query: Queryable, blocking: Boolean = true): Unit = writeLock {
val planToCache = query.queryExecution.analyzed
val dataIndex = cachedData.indexWhere(cd => planToCache.sameResult(cd.plan))
require(dataIndex >= 0, s"Table $query is not cached.")
cachedData(dataIndex).cachedRepresentation.uncache(blocking)
cachedData.remove(dataIndex)
}

/** Tries to remove the data for the given [[DataFrame]] from the cache if it's cached */
/** Tries to remove the data for the given [[Queryable]] from the cache
* if it's cached
*/
private[sql] def tryUncacheQuery(
query: DataFrame,
query: Queryable,
blocking: Boolean = true): Boolean = writeLock {
val planToCache = query.queryExecution.analyzed
val dataIndex = cachedData.indexWhere(cd => planToCache.sameResult(cd.plan))
Expand All @@ -123,12 +124,12 @@ private[sql] class CacheManager extends Logging {
found
}

/** Optionally returns cached data for the given [[DataFrame]] */
private[sql] def lookupCachedData(query: DataFrame): Option[CachedData] = readLock {
/** Optionally returns cached data for the given [[Queryable]] */
private[sql] def lookupCachedData(query: Queryable): Option[CachedData] = readLock {
lookupCachedData(query.queryExecution.analyzed)
}

/** Optionally returns cached data for the given LogicalPlan. */
/** Optionally returns cached data for the given [[LogicalPlan]]. */
private[sql] def lookupCachedData(plan: LogicalPlan): Option[CachedData] = readLock {
cachedData.find(cd => plan.sameResult(cd.plan))
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,80 @@
/*
* 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.sql

import scala.language.postfixOps

import org.apache.spark.sql.functions._
import org.apache.spark.sql.test.SharedSQLContext


class DatasetCacheSuite extends QueryTest with SharedSQLContext {
import testImplicits._

test("persist and unpersist") {
val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS().select(expr("_2 + 1").as[Int])
val cached = ds.cache()
// count triggers the caching action. It should not throw.
cached.count()
// Make sure, the Dataset is indeed cached.
assertCached(cached)
// Check result.
checkAnswer(
cached,
2, 3, 4)
// Drop the cache.
cached.unpersist()
assert(!sqlContext.isCached(cached), "The Dataset should not be cached.")
}

test("persist and then rebind right encoder when join 2 datasets") {
val ds1 = Seq("1", "2").toDS().as("a")
val ds2 = Seq(2, 3).toDS().as("b")

ds1.persist()
assertCached(ds1)
ds2.persist()
assertCached(ds2)

val joined = ds1.joinWith(ds2, $"a.value" === $"b.value")
checkAnswer(joined, ("2", 2))
assertCached(joined, 2)

ds1.unpersist()
assert(!sqlContext.isCached(ds1), "The Dataset ds1 should not be cached.")
ds2.unpersist()
assert(!sqlContext.isCached(ds2), "The Dataset ds2 should not be cached.")
}

test("persist and then groupBy columns asKey, map") {
val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS()
val grouped = ds.groupBy($"_1").keyAs[String]
val agged = grouped.mapGroups { case (g, iter) => (g, iter.map(_._2).sum) }
agged.persist()

checkAnswer(
agged.filter(_._1 == "b"),
("b", 3))
assertCached(agged.filter(_._1 == "b"))

ds.unpersist()
assert(!sqlContext.isCached(ds), "The Dataset ds should not be cached.")
agged.unpersist()
assert(!sqlContext.isCached(agged), "The Dataset agged should not be cached.")
}
}
5 changes: 3 additions & 2 deletions sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import scala.collection.JavaConverters._
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.util._
import org.apache.spark.sql.execution.columnar.InMemoryRelation
import org.apache.spark.sql.execution.Queryable

abstract class QueryTest extends PlanTest {

Expand Down Expand Up @@ -163,9 +164,9 @@ abstract class QueryTest extends PlanTest {
}

/**
* Asserts that a given [[DataFrame]] will be executed using the given number of cached results.
* Asserts that a given [[Queryable]] will be executed using the given number of cached results.
*/
def assertCached(query: DataFrame, numCachedTables: Int = 1): Unit = {
def assertCached(query: Queryable, numCachedTables: Int = 1): Unit = {
val planWithCaching = query.queryExecution.withCachedData
val cachedData = planWithCaching collect {
case cached: InMemoryRelation => cached
Expand Down