RDD Persistence
One of the most important capabilities in Spark is persisting (or caching) a dataset in memory across operations. When you persist an RDD, each node stores any partitions of it that it computes in memory and reuses them in other actions on that dataset (or datasets derived from it). This allows future actions to be much faster (often by more than 10x). Caching is a key tool for iterative algorithms and fast interactive use.
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. Spark’s 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 persisted RDD can be stored using a different storage level, allowing you, for example, to persist the dataset on disk, persist it in memory but as serialized Java objects (to save space), replicate it across nodes. These levels are set by passing a StorageLevel object (Scala,JavaPython) 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 full set of storage levels is:
Storage Level
Meaning
MEMORY_ONLY
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.
MEMORY_AND_DISK
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.
MEMORY_ONLY_SER  (Java and Scala)
Store RDD as serialized Java objects (one byte array per partition). This is generally more space-efficient than deserialized objects, especially when using a fast serializer, but more CPU-intensive to read.
MEMORY_AND_DISK_SER  (Java and Scala)
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.
DISK_ONLY
Store the RDD partitions only on disk.
MEMORY_ONLY_2, MEMORY_AND_DISK_2, etc.
Same as the levels above, but replicate each partition on two cluster nodes.
OFF_HEAP (experimental)
Similar to MEMORY_ONLY_SER, but store the data in off-heap memory. This requires off-heap memory to be enabled.
Note: In Python, stored objects will always be serialized with the Pickle library, so it does not matter whether you choose a serialized level. The available storage levels in Python include MEMORY_ONLY, MEMORY_ONLY_2, MEMORY_AND_DISK, MEMORY_AND_DISK_2, DISK_ONLY, and DISK_ONLY_2.
Spark also automatically persists some intermediate data in shuffle operations (e.g. reduceByKey), even without users calling persist. This is done to avoid recomputing the entire input if a node fails during the shuffle. We still recommend users call persist on the resulting RDD if they plan to reuse it.
Which Storage Level to Choose?
Spark’s storage levels are meant to provide different trade-offs 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), 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_SER and selecting a fast serialization library to make the objects much more space-efficient, but still reasonably fast to access. (Java and Scala)
  • 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 may be as fast as reading it from disk.
  • Use the replicated storage levels if you want fast fault recovery (e.g. if using Spark to serve requests from a web application). All the storage levels provide full fault tolerance by recomputing lost data, but the replicated ones let you continue running tasks on the RDD without waiting to recompute a lost partition.
Removing Data
Spark automatically monitors cache usage on each node and drops out old data partitions in a least-recently-used (LRU) fashion. If you would like to manually remove an RDD instead of waiting for it to fall out of the cache, use the RDD.unpersist() method.
 

9.spark Core 进阶2--Cashe的更多相关文章

  1. 8.spark Core 进阶1

        (e.g. standalone manager, Mesos, YARN)   In "cluster" mode, the framework launches the ...

  2. Spark 3.x Spark Core详解 & 性能优化

    Spark Core 1. 概述 Spark 是一种基于内存的快速.通用.可扩展的大数据分析计算引擎 1.1 Hadoop vs Spark 上面流程对应Hadoop的处理流程,下面对应着Spark的 ...

  3. Spark Streaming揭秘 Day35 Spark core思考

    Spark Streaming揭秘 Day35 Spark core思考 Spark上的子框架,都是后来加上去的.都是在Spark core上完成的,所有框架一切的实现最终还是由Spark core来 ...

  4. 【Spark Core】任务运行机制和Task源代码浅析1

    引言 上一小节<TaskScheduler源代码与任务提交原理浅析2>介绍了Driver側将Stage进行划分.依据Executor闲置情况分发任务,终于通过DriverActor向exe ...

  5. TypeError: Error #1034: 强制转换类型失败:无法将 mx.controls::DataGrid@9a7c0a1 转换为 spark.core.IViewport。

    1.错误描述 TypeError: Error #1034: 强制转换类型失败:无法将 mx.controls::DataGrid@9aa90a1 转换为 spark.core.IViewport. ...

  6. Spark Core

    Spark Core    DAG概念        有向无环图        Spark会根据用户提交的计算逻辑中的RDD的转换(变换方法)和动作(action方法)来生成RDD之间的依赖关系,同时 ...

  7. Spark Streaming 进阶与案例实战

    Spark Streaming 进阶与案例实战 1.带状态的算子: UpdateStateByKey 2.实战:计算到目前位置累积出现的单词个数写入到MySql中 1.create table CRE ...

  8. spark core (二)

    一.Spark-Shell交互式工具 1.Spark-Shell交互式工具 Spark-Shell提供了一种学习API的简单方式, 以及一个能够交互式分析数据的强大工具. 在Scala语言环境下或Py ...

  9. Spark Core 资源调度与任务调度(standalone client 流程描述)

    Spark Core 资源调度与任务调度(standalone client 流程描述) Spark集群启动:      集群启动后,Worker会向Master汇报资源情况(实际上将Worker的资 ...

随机推荐

  1. 【二】Jmeter接口自动化测试系列之函数使用及扩展

    上一篇文章我们了解了Jmeter的参数化的集中方法,虽然方法不是很多,但已经足够使用! 本篇文章,介绍一下Jmeter自带函数的使用和 函数扩展,来满足测试工作中的各种需求! Jmeter自带函数 点 ...

  2. 三(1)、springcloud之Eureka服务注册与发现

    1.认识Eureka ​ Eureka是Netflix的一个子模块,也是核心模块之一.Eureka是一个基于REST的服务,用于定位服务,以实现云端中间层服务发现和故障转移.服务注册与发现对于微服务架 ...

  3. 屏幕操作录制成gif图的技巧

    我呢,在记录一些做过得实例的时候,总需要上一两张效果图,截静态图太浪费时间了,于是就找了一些录制git图的软件 一.Gif动画录制工具 这是我在360软件中心下载的,用了一下,不好用,录制出来的图是黑 ...

  4. mongo之$max

    原集合: { _id: 1, highScore: 800, lowScore: 200 } 应用: #意思是:更新_id 等于1 的记录,条件是highScore 950>原纪录的highSc ...

  5. zless - 用于在显示器上阅读被压缩的文本文件的过滤器

    总览 (SYNOPSIS) zless [ name ... ] 描述 (DESCRIPTION) Zless 是一个文件过滤器, 用于在终端上全屏幕形式查看压缩的或没压缩的文本文件. 它等于把环境变 ...

  6. Ansible介绍

    第一章 ansible服务介绍 1.1 ansible批量管理服务概述 是基于python语言开发的自动化软件工具 是基于SSH远程管理服务实现远程主机批量管理 1.2 ansible批量管理服务意义 ...

  7. Echart中X轴数据过多时横向拉动展示

    chart.setOption( { tooltip: { trigger: 'axis' }, toolbox: { feature: { saveAsImage: {} } }, grid: { ...

  8. nodejs 模板引擎ejs的简单使用(2)

    test.ejs <!DOCTYPE html> <html> <head> <meta charset="utf-8"> < ...

  9. php漂亮的分页类

    <?php    /*    * PHP分页类    * @package Page    * @Created 2013-03-27    * @Modify  2013-03-27    * ...

  10. jq-demo-在列表中添加新节点,点击删除

    <!DOCTYPE html> <html> <head> <meta charset="UTF-8"> <title> ...