aboutsummaryrefslogtreecommitdiff
path: root/docs
diff options
context:
space:
mode:
authorMatei Zaharia <matei@eecs.berkeley.edu>2012-09-27 17:50:59 -0700
committerMatei Zaharia <matei@eecs.berkeley.edu>2012-09-27 17:50:59 -0700
commit7bcb08cef5e6438ce8c8efa3da3a8f94f2a1fbf9 (patch)
treeed08d9f1d1d6a1e95f247a7c6541780a21acdac4 /docs
parent0850d641afa1f7181c7dc611a08e2b9530540adc (diff)
downloadspark-7bcb08cef5e6438ce8c8efa3da3a8f94f2a1fbf9.tar.gz
spark-7bcb08cef5e6438ce8c8efa3da3a8f94f2a1fbf9.tar.bz2
spark-7bcb08cef5e6438ce8c8efa3da3a8f94f2a1fbf9.zip
Renamed storage levels to something cleaner; fixes #223.
Diffstat (limited to 'docs')
-rw-r--r--docs/scala-programming-guide.md24
1 files changed, 12 insertions, 12 deletions
diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md
index 28e7bdd4c9..a370bf3ddc 100644
--- a/docs/scala-programming-guide.md
+++ b/docs/scala-programming-guide.md
@@ -231,30 +231,30 @@ One of the most important capabilities in Spark is *persisting* (or *caching*) a
You can mark an RDD to be persisted using the `persist()` or `cache()` methods on it. The first time it is computed in an action, it will be kept in memory on the nodes. The cache is fault-tolerant -- if any partition of an RDD is lost, it will automatically be recomputed using the transformations that originally created it.
-In addition, each RDD can be stored using a different *storage level*, allowing you, for example, to persist the dataset on disk, or persist it in memory but as serialized Java objects (to save space), or even replicate it across nodes. These levels are chosen by passing a [`spark.storage.StorageLevel`]({{HOME_PATH}}api/core/index.html#spark.storage.StorageLevel) object to `persist()`. The `cache()` method is a shorthand for using the default storage level, which is `StorageLevel.MEMORY_ONLY_DESER` (store deserialized objects in memory). The complete set of available storage levels is:
+In addition, each RDD can be stored using a different *storage level*, allowing you, for example, to persist the dataset on disk, or persist it in memory but as serialized Java objects (to save space), or even replicate it across nodes. These levels are chosen by passing a [`spark.storage.StorageLevel`]({{HOME_PATH}}api/core/index.html#spark.storage.StorageLevel) object to `persist()`. The `cache()` method is a shorthand for using the default storage level, which is `StorageLevel.MEMORY_ONLY` (store deserialized objects in memory). The complete set of available storage levels is:
<table class="table">
-<tr><th style="width:30%">Storage Level</th><th>Meaning</th></tr>
+<tr><th style="width:23%">Storage Level</th><th>Meaning</th></tr>
<tr>
- <td> MEMORY_ONLY_DESER </td>
+ <td> MEMORY_ONLY </td>
<td> Store RDD as deserialized Java objects in the JVM. If the RDD does not fit in memory, some partitions will
not be cached and will be recomputed on the fly each time they're needed. This is the default level. </td>
</tr>
<tr>
- <td> DISK_AND_MEMORY_DESER </td>
+ <td> MEMORY_AND_DISK </td>
<td> Store RDD as deserialized Java objects in the JVM. If the RDD does not fit in memory, store the
partitions that don't fit on disk, and read them from there when they're needed. </td>
</tr>
<tr>
- <td> MEMORY_ONLY </td>
- <td> Store RDD as <i>serialized</i> Java objects (that is, one byte array per partition).
+ <td> MEMORY_ONLY_SER </td>
+ <td> Store RDD as <i>serialized</i> Java objects (one byte array per partition).
This is generally more space-efficient than deserialized objects, especially when using a
<a href="{{HOME_PATH}}tuning.html">fast serializer</a>, but more CPU-intensive to read.
</td>
</tr>
<tr>
- <td> DISK_AND_MEMORY </td>
- <td> Similar to MEMORY_ONLY, but spill partitions that don't fit in memory to disk instead of recomputing them
+ <td> MEMORY_AND_DISK_SER </td>
+ <td> Similar to MEMORY_ONLY_SER, but spill partitions that don't fit in memory to disk instead of recomputing them
on the fly each time they're needed. </td>
</tr>
<tr>
@@ -262,8 +262,8 @@ In addition, each RDD can be stored using a different *storage level*, allowing
<td> Store the RDD partitions only on disk. </td>
</tr>
<tr>
- <td> MEMORY_ONLY_DESER_2 / DISK_AND_MEMORY_DESER_2 / MEMORY_ONLY_2 / DISK_ONLY_2 / DISK_AND_MEMORY_2 </td>
- <td> Same as the levels above, but replicate each partition on two nodes. </td>
+ <td> MEMORY_ONLY_2, MEMORY_AND_DISK_2, etc. </td>
+ <td> Same as the levels above, but replicate each partition on two cluster nodes. </td>
</tr>
</table>
@@ -272,9 +272,9 @@ In addition, each RDD can be stored using a different *storage level*, allowing
Spark's storage levels are meant to provide different tradeoffs between memory usage and CPU efficiency.
We recommend going through the following process to select one:
-* If your RDDs fit comfortably with the default storage level (`MEMORY_ONLY_DESER`), leave them that way. This is the most
+* If your RDDs fit comfortably with the default storage level (`MEMORY_ONLY`), leave them that way. This is the most
CPU-efficient option, allowing operations on the RDDs to run as fast as possible.
-* If not, try using `MEMORY_ONLY` and [selecting a fast serialization library]({{HOME_PATH}}tuning.html) to make the objects
+* If not, try using `MEMORY_ONLY_SER` and [selecting a fast serialization library]({{HOME_PATH}}tuning.html) to make the objects
much more space-efficient, but still reasonably fast to access.
* Don't spill to disk unless the functions that computed your datasets are expensive, or they filter a large
amount of the data. Otherwise, recomputing a partition is about as fast as reading it from disk.