org.apache.spark.rdd

RDD

abstract class RDD[T] extends Serializable with Logging

A Resilient Distributed Dataset (RDD), the basic abstraction in Spark. Represents an immutable, partitioned collection of elements that can be operated on in parallel. This class contains the basic operations available on all RDDs, such as map, filter, and persist. In addition, org.apache.spark.rdd.PairRDDFunctions contains operations available only on RDDs of key-value pairs, such as groupByKey and join; org.apache.spark.rdd.DoubleRDDFunctions contains operations available only on RDDs of Doubles; and org.apache.spark.rdd.SequenceFileRDDFunctions contains operations available on RDDs that can be saved as SequenceFiles. All operations are automatically available on any RDD of the right type (e.g. RDD[(Int, Int)] through implicit.

Internally, each RDD is characterized by five main properties:

All of the scheduling and execution in Spark is done based on these methods, allowing each RDD to implement its own way of computing itself. Indeed, users can implement custom RDDs (e.g. for reading data from a new storage system) by overriding these functions. Please refer to the Spark paper for more details on RDD internals.

Linear Supertypes
Logging, Serializable, Serializable, AnyRef, Any
Known Subclasses
Ordering
  1. Alphabetic
  2. By inheritance
Inherited
  1. RDD
  2. Logging
  3. Serializable
  4. Serializable
  5. AnyRef
  6. Any
  1. Hide All
  2. Show all
Learn more about member selection
Visibility
  1. Public
  2. All

Instance Constructors

  1. new RDD(oneParent: RDD[_])(implicit arg0: ClassTag[T])

    Construct an RDD with just a one-to-one dependency on one parent

  2. new RDD(_sc: SparkContext, deps: Seq[Dependency[_]])(implicit arg0: ClassTag[T])

Abstract Value Members

  1. abstract def compute(split: Partition, context: TaskContext): Iterator[T]

    :: DeveloperApi :: Implemented by subclasses to compute a given partition.

    :: DeveloperApi :: Implemented by subclasses to compute a given partition.

    Annotations
    @DeveloperApi()
  2. abstract def getPartitions: Array[Partition]

    Implemented by subclasses to return the set of partitions in this RDD.

    Implemented by subclasses to return the set of partitions in this RDD. This method will only be called once, so it is safe to implement a time-consuming computation in it.

    Attributes
    protected

Concrete Value Members

  1. final def !=(arg0: AnyRef): Boolean

    Definition Classes
    AnyRef
  2. final def !=(arg0: Any): Boolean

    Definition Classes
    Any
  3. final def ##(): Int

    Definition Classes
    AnyRef → Any
  4. def ++(other: RDD[T]): RDD[T]

    Return the union of this RDD and another one.

    Return the union of this RDD and another one. Any identical elements will appear multiple times (use .distinct() to eliminate them).

  5. final def ==(arg0: AnyRef): Boolean

    Definition Classes
    AnyRef
  6. final def ==(arg0: Any): Boolean

    Definition Classes
    Any
  7. def aggregate[U](zeroValue: U)(seqOp: (U, T) ⇒ U, combOp: (U, U) ⇒ U)(implicit arg0: ClassTag[U]): U

    Aggregate the elements of each partition, and then the results for all the partitions, using given combine functions and a neutral "zero value".

    Aggregate the elements of each partition, and then the results for all the partitions, using given combine functions and a neutral "zero value". This function can return a different result type, U, than the type of this RDD, T. Thus, we need one operation for merging a T into an U and one operation for merging two U's, as in scala.TraversableOnce. Both of these functions are allowed to modify and return their first argument instead of creating a new U to avoid memory allocation.

  8. final def asInstanceOf[T0]: T0

    Definition Classes
    Any
  9. def cache(): RDD.this.type

    Persist this RDD with the default storage level (MEMORY_ONLY).

  10. def cartesian[U](other: RDD[U])(implicit arg0: ClassTag[U]): RDD[(T, U)]

    Return the Cartesian product of this RDD and another one, that is, the RDD of all pairs of elements (a, b) where a is in this and b is in other.

  11. def checkpoint(): Unit

    Mark this RDD for checkpointing.

    Mark this RDD for checkpointing. It will be saved to a file inside the checkpoint directory set with SparkContext#setCheckpointDir and all references to its parent RDDs will be removed. This function must be called before any job has been executed on this RDD. It is strongly recommended that this RDD is persisted in memory, otherwise saving it on a file will require recomputation.

  12. def clearDependencies(): Unit

    Clears the dependencies of this RDD.

    Clears the dependencies of this RDD. This method must ensure that all references to the original parent RDDs is removed to enable the parent RDDs to be garbage collected. Subclasses of RDD may override this method for implementing their own cleaning logic. See org.apache.spark.rdd.UnionRDD for an example.

    Attributes
    protected
  13. def clone(): AnyRef

    Attributes
    protected[java.lang]
    Definition Classes
    AnyRef
    Annotations
    @throws( ... )
  14. def coalesce(numPartitions: Int, shuffle: Boolean = false)(implicit ord: Ordering[T] = null): RDD[T]

    Return a new RDD that is reduced into numPartitions partitions.

    Return a new RDD that is reduced into numPartitions partitions.

    This results in a narrow dependency, e.g. if you go from 1000 partitions to 100 partitions, there will not be a shuffle, instead each of the 100 new partitions will claim 10 of the current partitions.

    However, if you're doing a drastic coalesce, e.g. to numPartitions = 1, this may result in your computation taking place on fewer nodes than you like (e.g. one node in the case of numPartitions = 1). To avoid this, you can pass shuffle = true. This will add a shuffle step, but means the current upstream partitions will be executed in parallel (per whatever the current partitioning is).

    Note: With shuffle = true, you can actually coalesce to a larger number of partitions. This is useful if you have a small number of partitions, say 100, potentially with a few partitions being abnormally large. Calling coalesce(1000, shuffle = true) will result in 1000 partitions with the data distributed using a hash partitioner.

  15. def collect[U](f: PartialFunction[T, U])(implicit arg0: ClassTag[U]): RDD[U]

    Return an RDD that contains all matching values by applying f.

  16. def collect(): Array[T]

    Return an array that contains all of the elements in this RDD.

  17. def context: SparkContext

    The org.apache.spark.SparkContext that this RDD was created on.

  18. def count(): Long

    Return the number of elements in the RDD.

  19. def countApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble]

    :: Experimental :: Approximate version of count() that returns a potentially incomplete result within a timeout, even if not all tasks have finished.

    :: Experimental :: Approximate version of count() that returns a potentially incomplete result within a timeout, even if not all tasks have finished.

    Annotations
    @Experimental()
  20. def countApproxDistinct(relativeSD: Double = 0.05): Long

    Return approximate number of distinct elements in the RDD.

    Return approximate number of distinct elements in the RDD.

    The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available here.

    relativeSD

    Relative accuracy. Smaller values create counters that require more space. It must be greater than 0.000017.

  21. def countApproxDistinct(p: Int, sp: Int): Long

    :: Experimental :: Return approximate number of distinct elements in the RDD.

    :: Experimental :: Return approximate number of distinct elements in the RDD.

    The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available here.

    The relative accuracy is approximately 1.054 / sqrt(2^p). Setting a nonzero sp > p would trigger sparse representation of registers, which may reduce the memory consumption and increase accuracy when the cardinality is small.

    p

    The precision value for the normal set. p must be a value between 4 and sp if sp is not zero (32 max).

    sp

    The precision value for the sparse set, between 0 and 32. If sp equals 0, the sparse representation is skipped.

    Annotations
    @Experimental()
  22. def countByValue()(implicit ord: Ordering[T] = null): Map[T, Long]

    Return the count of each unique value in this RDD as a local map of (value, count) pairs.

    Return the count of each unique value in this RDD as a local map of (value, count) pairs.

    Note that this method should only be used if the resulting map is expected to be small, as the whole thing is loaded into the driver's memory. To handle very large results, consider using rdd.map(x => (x, 1L)).reduceByKey(_ + _), which returns an RDD[T, Long] instead of a map.

  23. def countByValueApprox(timeout: Long, confidence: Double = 0.95)(implicit ord: Ordering[T] = null): PartialResult[Map[T, BoundedDouble]]

    :: Experimental :: Approximate version of countByValue().

    :: Experimental :: Approximate version of countByValue().

    Annotations
    @Experimental()
  24. final def dependencies: Seq[Dependency[_]]

    Get the list of dependencies of this RDD, taking into account whether the RDD is checkpointed or not.

  25. def distinct(): RDD[T]

    Return a new RDD containing the distinct elements in this RDD.

  26. def distinct(numPartitions: Int)(implicit ord: Ordering[T] = null): RDD[T]

    Return a new RDD containing the distinct elements in this RDD.

  27. final def eq(arg0: AnyRef): Boolean

    Definition Classes
    AnyRef
  28. def equals(arg0: Any): Boolean

    Definition Classes
    AnyRef → Any
  29. def filter(f: (T) ⇒ Boolean): RDD[T]

    Return a new RDD containing only the elements that satisfy a predicate.

  30. def finalize(): Unit

    Attributes
    protected[java.lang]
    Definition Classes
    AnyRef
    Annotations
    @throws( classOf[java.lang.Throwable] )
  31. def first(): T

    Return the first element in this RDD.

  32. def firstParent[U](implicit arg0: ClassTag[U]): RDD[U]

    Returns the first parent RDD

    Returns the first parent RDD

    Attributes
    protected[org.apache.spark]
  33. def flatMap[U](f: (T) ⇒ TraversableOnce[U])(implicit arg0: ClassTag[U]): RDD[U]

    Return a new RDD by first applying a function to all elements of this RDD, and then flattening the results.

  34. def fold(zeroValue: T)(op: (T, T) ⇒ T): T

    Aggregate the elements of each partition, and then the results for all the partitions, using a given associative and commutative function and a neutral "zero value".

    Aggregate the elements of each partition, and then the results for all the partitions, using a given associative and commutative function and a neutral "zero value". The function op(t1, t2) is allowed to modify t1 and return it as its result value to avoid object allocation; however, it should not modify t2.

    This behaves somewhat differently from fold operations implemented for non-distributed collections in functional languages like Scala. This fold operation may be applied to partitions individually, and then fold those results into the final result, rather than apply the fold to each element sequentially in some defined ordering. For functions that are not commutative, the result may differ from that of a fold applied to a non-distributed collection.

  35. def foreach(f: (T) ⇒ Unit): Unit

    Applies a function f to all elements of this RDD.

  36. def foreachPartition(f: (Iterator[T]) ⇒ Unit): Unit

    Applies a function f to each partition of this RDD.

  37. def getCheckpointFile: Option[String]

    Gets the name of the directory to which this RDD was checkpointed.

    Gets the name of the directory to which this RDD was checkpointed. This is not defined if the RDD is checkpointed locally.

  38. final def getClass(): Class[_]

    Definition Classes
    AnyRef → Any
  39. def getDependencies: Seq[Dependency[_]]

    Implemented by subclasses to return how this RDD depends on parent RDDs.

    Implemented by subclasses to return how this RDD depends on parent RDDs. This method will only be called once, so it is safe to implement a time-consuming computation in it.

    Attributes
    protected
  40. def getPreferredLocations(split: Partition): Seq[String]

    Optionally overridden by subclasses to specify placement preferences.

    Optionally overridden by subclasses to specify placement preferences.

    Attributes
    protected
  41. def getStorageLevel: StorageLevel

    Get the RDD's current storage level, or StorageLevel.

    Get the RDD's current storage level, or StorageLevel.NONE if none is set.

  42. def glom(): RDD[Array[T]]

    Return an RDD created by coalescing all elements within each partition into an array.

  43. def groupBy[K](f: (T) ⇒ K, p: Partitioner)(implicit kt: ClassTag[K], ord: Ordering[K] = null): RDD[(K, Iterable[T])]

    Return an RDD of grouped items.

    Return an RDD of grouped items. Each group consists of a key and a sequence of elements mapping to that key. The ordering of elements within each group is not guaranteed, and may even differ each time the resulting RDD is evaluated.

    Note: This operation may be very expensive. If you are grouping in order to perform an aggregation (such as a sum or average) over each key, using PairRDDFunctions.aggregateByKey or PairRDDFunctions.reduceByKey will provide much better performance.

  44. def groupBy[K](f: (T) ⇒ K, numPartitions: Int)(implicit kt: ClassTag[K]): RDD[(K, Iterable[T])]

    Return an RDD of grouped elements.

    Return an RDD of grouped elements. Each group consists of a key and a sequence of elements mapping to that key. The ordering of elements within each group is not guaranteed, and may even differ each time the resulting RDD is evaluated.

    Note: This operation may be very expensive. If you are grouping in order to perform an aggregation (such as a sum or average) over each key, using PairRDDFunctions.aggregateByKey or PairRDDFunctions.reduceByKey will provide much better performance.

  45. def groupBy[K](f: (T) ⇒ K)(implicit kt: ClassTag[K]): RDD[(K, Iterable[T])]

    Return an RDD of grouped items.

    Return an RDD of grouped items. Each group consists of a key and a sequence of elements mapping to that key. The ordering of elements within each group is not guaranteed, and may even differ each time the resulting RDD is evaluated.

    Note: This operation may be very expensive. If you are grouping in order to perform an aggregation (such as a sum or average) over each key, using PairRDDFunctions.aggregateByKey or PairRDDFunctions.reduceByKey will provide much better performance.

  46. def hashCode(): Int

    Definition Classes
    AnyRef → Any
  47. val id: Int

    A unique ID for this RDD (within its SparkContext).

  48. def intersection(other: RDD[T], numPartitions: Int): RDD[T]

    Return the intersection of this RDD and another one.

    Return the intersection of this RDD and another one. The output will not contain any duplicate elements, even if the input RDDs did. Performs a hash partition across the cluster

    Note that this method performs a shuffle internally.

    numPartitions

    How many partitions to use in the resulting RDD

  49. def intersection(other: RDD[T], partitioner: Partitioner)(implicit ord: Ordering[T] = null): RDD[T]

    Return the intersection of this RDD and another one.

    Return the intersection of this RDD and another one. The output will not contain any duplicate elements, even if the input RDDs did.

    Note that this method performs a shuffle internally.

    partitioner

    Partitioner to use for the resulting RDD

  50. def intersection(other: RDD[T]): RDD[T]

    Return the intersection of this RDD and another one.

    Return the intersection of this RDD and another one. The output will not contain any duplicate elements, even if the input RDDs did.

    Note that this method performs a shuffle internally.

  51. def isCheckpointed: Boolean

    Return whether this RDD is marked for checkpointing, either reliably or locally.

  52. def isEmpty(): Boolean

    returns

    true if and only if the RDD contains no elements at all. Note that an RDD may be empty even when it has at least 1 partition.

    Note

    due to complications in the internal implementation, this method will raise an exception if called on an RDD of Nothing or Null. This may be come up in practice because, for example, the type of parallelize(Seq()) is RDD[Nothing]. (parallelize(Seq()) should be avoided anyway in favor of parallelize(Seq[T]()).)

  53. final def isInstanceOf[T0]: Boolean

    Definition Classes
    Any
  54. def isTraceEnabled(): Boolean

    Attributes
    protected
    Definition Classes
    Logging
  55. final def iterator(split: Partition, context: TaskContext): Iterator[T]

    Internal method to this RDD; will read from cache if applicable, or otherwise compute it.

    Internal method to this RDD; will read from cache if applicable, or otherwise compute it. This should not be called by users directly, but is available for implementors of custom subclasses of RDD.

  56. def keyBy[K](f: (T) ⇒ K): RDD[(K, T)]

    Creates tuples of the elements in this RDD by applying f.

  57. def localCheckpoint(): RDD.this.type

    Mark this RDD for local checkpointing using Spark's existing caching layer.

    Mark this RDD for local checkpointing using Spark's existing caching layer.

    This method is for users who wish to truncate RDD lineages while skipping the expensive step of replicating the materialized data in a reliable distributed file system. This is useful for RDDs with long lineages that need to be truncated periodically (e.g. GraphX).

    Local checkpointing sacrifices fault-tolerance for performance. In particular, checkpointed data is written to ephemeral local storage in the executors instead of to a reliable, fault-tolerant storage. The effect is that if an executor fails during the computation, the checkpointed data may no longer be accessible, causing an irrecoverable job failure.

    This is NOT safe to use with dynamic allocation, which removes executors along with their cached blocks. If you must use both features, you are advised to set spark.dynamicAllocation.cachedExecutorIdleTimeout to a high value.

    The checkpoint directory set through SparkContext#setCheckpointDir is not used.

  58. def log: Logger

    Attributes
    protected
    Definition Classes
    Logging
  59. def logDebug(msg: ⇒ String, throwable: Throwable): Unit

    Attributes
    protected
    Definition Classes
    Logging
  60. def logDebug(msg: ⇒ String): Unit

    Attributes
    protected
    Definition Classes
    Logging
  61. def logError(msg: ⇒ String, throwable: Throwable): Unit

    Attributes
    protected
    Definition Classes
    Logging
  62. def logError(msg: ⇒ String): Unit

    Attributes
    protected
    Definition Classes
    Logging
  63. def logInfo(msg: ⇒ String, throwable: Throwable): Unit

    Attributes
    protected
    Definition Classes
    Logging
  64. def logInfo(msg: ⇒ String): Unit

    Attributes
    protected
    Definition Classes
    Logging
  65. def logName: String

    Attributes
    protected
    Definition Classes
    Logging
  66. def logTrace(msg: ⇒ String, throwable: Throwable): Unit

    Attributes
    protected
    Definition Classes
    Logging
  67. def logTrace(msg: ⇒ String): Unit

    Attributes
    protected
    Definition Classes
    Logging
  68. def logWarning(msg: ⇒ String, throwable: Throwable): Unit

    Attributes
    protected
    Definition Classes
    Logging
  69. def logWarning(msg: ⇒ String): Unit

    Attributes
    protected
    Definition Classes
    Logging
  70. def map[U](f: (T) ⇒ U)(implicit arg0: ClassTag[U]): RDD[U]

    Return a new RDD by applying a function to all elements of this RDD.

  71. def mapPartitions[U](f: (Iterator[T]) ⇒ Iterator[U], preservesPartitioning: Boolean = false)(implicit arg0: ClassTag[U]): RDD[U]

    Return a new RDD by applying a function to each partition of this RDD.

    Return a new RDD by applying a function to each partition of this RDD.

    preservesPartitioning indicates whether the input function preserves the partitioner, which should be false unless this is a pair RDD and the input function doesn't modify the keys.

  72. def mapPartitionsWithIndex[U](f: (Int, Iterator[T]) ⇒ Iterator[U], preservesPartitioning: Boolean = false)(implicit arg0: ClassTag[U]): RDD[U]

    Return a new RDD by applying a function to each partition of this RDD, while tracking the index of the original partition.

    Return a new RDD by applying a function to each partition of this RDD, while tracking the index of the original partition.

    preservesPartitioning indicates whether the input function preserves the partitioner, which should be false unless this is a pair RDD and the input function doesn't modify the keys.

  73. def max()(implicit ord: Ordering[T]): T

    Returns the max of this RDD as defined by the implicit Ordering[T].

    Returns the max of this RDD as defined by the implicit Ordering[T].

    returns

    the maximum element of the RDD

  74. def min()(implicit ord: Ordering[T]): T

    Returns the min of this RDD as defined by the implicit Ordering[T].

    Returns the min of this RDD as defined by the implicit Ordering[T].

    returns

    the minimum element of the RDD

  75. var name: String

    A friendly name for this RDD

  76. final def ne(arg0: AnyRef): Boolean

    Definition Classes
    AnyRef
  77. final def notify(): Unit

    Definition Classes
    AnyRef
  78. final def notifyAll(): Unit

    Definition Classes
    AnyRef
  79. def parent[U](j: Int)(implicit arg0: ClassTag[U]): RDD[U]

    Returns the jth parent RDD: e.

    Returns the jth parent RDD: e.g. rdd.parent[T](0) is equivalent to rdd.firstParent[T]

    Attributes
    protected[org.apache.spark]
  80. val partitioner: Option[Partitioner]

    Optionally overridden by subclasses to specify how they are partitioned.

  81. final def partitions: Array[Partition]

    Get the array of partitions of this RDD, taking into account whether the RDD is checkpointed or not.

  82. def persist(): RDD.this.type

    Persist this RDD with the default storage level (MEMORY_ONLY).

  83. def persist(newLevel: StorageLevel): RDD.this.type

    Set this RDD's storage level to persist its values across operations after the first time it is computed.

    Set this RDD's storage level to persist its values across operations after the first time it is computed. This can only be used to assign a new storage level if the RDD does not have a storage level set yet. Local checkpointing is an exception.

  84. def pipe(command: Seq[String], env: Map[String, String] = Map(), printPipeContext: ((String) ⇒ Unit) ⇒ Unit = null, printRDDElement: (T, (String) ⇒ Unit) ⇒ Unit = null, separateWorkingDir: Boolean = false): RDD[String]

    Return an RDD created by piping elements to a forked external process.

    Return an RDD created by piping elements to a forked external process. The print behavior can be customized by providing two functions.

    command

    command to run in forked process.

    env

    environment variables to set.

    printPipeContext

    Before piping elements, this function is called as an opportunity to pipe context data. Print line function (like out.println) will be passed as printPipeContext's parameter.

    printRDDElement

    Use this function to customize how to pipe elements. This function will be called with each RDD element as the 1st parameter, and the print line function (like out.println()) as the 2nd parameter. An example of pipe the RDD data of groupBy() in a streaming way, instead of constructing a huge String to concat all the elements: def printRDDElement(record:(String, Seq[String]), f:String=>Unit) = for (e <- record._2){f(e)}

    separateWorkingDir

    Use separate working directories for each task.

    returns

    the result RDD

  85. def pipe(command: String, env: Map[String, String]): RDD[String]

    Return an RDD created by piping elements to a forked external process.

  86. def pipe(command: String): RDD[String]

    Return an RDD created by piping elements to a forked external process.

  87. final def preferredLocations(split: Partition): Seq[String]

    Get the preferred locations of a partition, taking into account whether the RDD is checkpointed.

  88. def randomSplit(weights: Array[Double], seed: Long = Utils.random.nextLong): Array[RDD[T]]

    Randomly splits this RDD with the provided weights.

    Randomly splits this RDD with the provided weights.

    weights

    weights for splits, will be normalized if they don't sum to 1

    seed

    random seed

    returns

    split RDDs in an array

  89. def reduce(f: (T, T) ⇒ T): T

    Reduces the elements of this RDD using the specified commutative and associative binary operator.

  90. def repartition(numPartitions: Int)(implicit ord: Ordering[T] = null): RDD[T]

    Return a new RDD that has exactly numPartitions partitions.

    Return a new RDD that has exactly numPartitions partitions.

    Can increase or decrease the level of parallelism in this RDD. Internally, this uses a shuffle to redistribute data.

    If you are decreasing the number of partitions in this RDD, consider using coalesce, which can avoid performing a shuffle.

  91. def sample(withReplacement: Boolean, fraction: Double, seed: Long = Utils.random.nextLong): RDD[T]

    Return a sampled subset of this RDD.

    Return a sampled subset of this RDD.

    withReplacement

    can elements be sampled multiple times (replaced when sampled out)

    fraction

    expected size of the sample as a fraction of this RDD's size without replacement: probability that each element is chosen; fraction must be [0, 1] with replacement: expected number of times each element is chosen; fraction must be >= 0

    seed

    seed for the random number generator

  92. def saveAsObjectFile(path: String): Unit

    Save this RDD as a SequenceFile of serialized objects.

  93. def saveAsTextFile(path: String, codec: Class[_ <: CompressionCodec]): Unit

    Save this RDD as a compressed text file, using string representations of elements.

  94. def saveAsTextFile(path: String): Unit

    Save this RDD as a text file, using string representations of elements.

  95. def setName(_name: String): RDD.this.type

    Assign a name to this RDD

  96. def sortBy[K](f: (T) ⇒ K, ascending: Boolean = true, numPartitions: Int = this.partitions.length)(implicit ord: Ordering[K], ctag: ClassTag[K]): RDD[T]

    Return this RDD sorted by the given key function.

  97. def sparkContext: SparkContext

    The SparkContext that created this RDD.

  98. def subtract(other: RDD[T], p: Partitioner)(implicit ord: Ordering[T] = null): RDD[T]

    Return an RDD with the elements from this that are not in other.

  99. def subtract(other: RDD[T], numPartitions: Int): RDD[T]

    Return an RDD with the elements from this that are not in other.

  100. def subtract(other: RDD[T]): RDD[T]

    Return an RDD with the elements from this that are not in other.

    Return an RDD with the elements from this that are not in other.

    Uses this partitioner/partition size, because even if other is huge, the resulting RDD will be <= us.

  101. final def synchronized[T0](arg0: ⇒ T0): T0

    Definition Classes
    AnyRef
  102. def take(num: Int): Array[T]

    Take the first num elements of the RDD.

    Take the first num elements of the RDD. It works by first scanning one partition, and use the results from that partition to estimate the number of additional partitions needed to satisfy the limit.

    Note

    due to complications in the internal implementation, this method will raise an exception if called on an RDD of Nothing or Null.

  103. def takeOrdered(num: Int)(implicit ord: Ordering[T]): Array[T]

    Returns the first k (smallest) elements from this RDD as defined by the specified implicit Ordering[T] and maintains the ordering.

    Returns the first k (smallest) elements from this RDD as defined by the specified implicit Ordering[T] and maintains the ordering. This does the opposite of top. For example:

    sc.parallelize(Seq(10, 4, 2, 12, 3)).takeOrdered(1)
    // returns Array(2)
    
    sc.parallelize(Seq(2, 3, 4, 5, 6)).takeOrdered(2)
    // returns Array(2, 3)
    num

    k, the number of elements to return

    ord

    the implicit ordering for T

    returns

    an array of top elements

  104. def takeSample(withReplacement: Boolean, num: Int, seed: Long = Utils.random.nextLong): Array[T]

    Return a fixed-size sampled subset of this RDD in an array

    Return a fixed-size sampled subset of this RDD in an array

    withReplacement

    whether sampling is done with replacement

    num

    size of the returned sample

    seed

    seed for the random number generator

    returns

    sample of specified size in an array

  105. def toDebugString: String

    A description of this RDD and its recursive dependencies for debugging.

  106. def toJavaRDD(): JavaRDD[T]

  107. def toLocalIterator: Iterator[T]

    Return an iterator that contains all of the elements in this RDD.

    Return an iterator that contains all of the elements in this RDD.

    The iterator will consume as much memory as the largest partition in this RDD.

    Note: this results in multiple Spark jobs, and if the input RDD is the result of a wide transformation (e.g. join with different partitioners), to avoid recomputing the input RDD should be cached first.

  108. def toString(): String

    Definition Classes
    RDD → AnyRef → Any
  109. def top(num: Int)(implicit ord: Ordering[T]): Array[T]

    Returns the top k (largest) elements from this RDD as defined by the specified implicit Ordering[T].

    Returns the top k (largest) elements from this RDD as defined by the specified implicit Ordering[T]. This does the opposite of takeOrdered. For example:

    sc.parallelize(Seq(10, 4, 2, 12, 3)).top(1)
    // returns Array(12)
    
    sc.parallelize(Seq(2, 3, 4, 5, 6)).top(2)
    // returns Array(6, 5)
    num

    k, the number of top elements to return

    ord

    the implicit ordering for T

    returns

    an array of top elements

  110. def treeAggregate[U](zeroValue: U)(seqOp: (U, T) ⇒ U, combOp: (U, U) ⇒ U, depth: Int = 2)(implicit arg0: ClassTag[U]): U

    Aggregates the elements of this RDD in a multi-level tree pattern.

    Aggregates the elements of this RDD in a multi-level tree pattern.

    depth

    suggested depth of the tree (default: 2)

    See also

    org.apache.spark.rdd.RDD#aggregate

  111. def treeReduce(f: (T, T) ⇒ T, depth: Int = 2): T

    Reduces the elements of this RDD in a multi-level tree pattern.

    Reduces the elements of this RDD in a multi-level tree pattern.

    depth

    suggested depth of the tree (default: 2)

    See also

    org.apache.spark.rdd.RDD#reduce

  112. def union(other: RDD[T]): RDD[T]

    Return the union of this RDD and another one.

    Return the union of this RDD and another one. Any identical elements will appear multiple times (use .distinct() to eliminate them).

  113. def unpersist(blocking: Boolean = true): RDD.this.type

    Mark the RDD as non-persistent, and remove all blocks for it from memory and disk.

    Mark the RDD as non-persistent, and remove all blocks for it from memory and disk.

    blocking

    Whether to block until all blocks are deleted.

    returns

    This RDD.

  114. final def wait(): Unit

    Definition Classes
    AnyRef
    Annotations
    @throws( ... )
  115. final def wait(arg0: Long, arg1: Int): Unit

    Definition Classes
    AnyRef
    Annotations
    @throws( ... )
  116. final def wait(arg0: Long): Unit

    Definition Classes
    AnyRef
    Annotations
    @throws( ... )
  117. def zip[U](other: RDD[U])(implicit arg0: ClassTag[U]): RDD[(T, U)]

    Zips this RDD with another one, returning key-value pairs with the first element in each RDD, second element in each RDD, etc.

    Zips this RDD with another one, returning key-value pairs with the first element in each RDD, second element in each RDD, etc. Assumes that the two RDDs have the *same number of partitions* and the *same number of elements in each partition* (e.g. one was made through a map on the other).

  118. def zipPartitions[B, C, D, V](rdd2: RDD[B], rdd3: RDD[C], rdd4: RDD[D])(f: (Iterator[T], Iterator[B], Iterator[C], Iterator[D]) ⇒ Iterator[V])(implicit arg0: ClassTag[B], arg1: ClassTag[C], arg2: ClassTag[D], arg3: ClassTag[V]): RDD[V]

  119. def zipPartitions[B, C, D, V](rdd2: RDD[B], rdd3: RDD[C], rdd4: RDD[D], preservesPartitioning: Boolean)(f: (Iterator[T], Iterator[B], Iterator[C], Iterator[D]) ⇒ Iterator[V])(implicit arg0: ClassTag[B], arg1: ClassTag[C], arg2: ClassTag[D], arg3: ClassTag[V]): RDD[V]

  120. def zipPartitions[B, C, V](rdd2: RDD[B], rdd3: RDD[C])(f: (Iterator[T], Iterator[B], Iterator[C]) ⇒ Iterator[V])(implicit arg0: ClassTag[B], arg1: ClassTag[C], arg2: ClassTag[V]): RDD[V]

  121. def zipPartitions[B, C, V](rdd2: RDD[B], rdd3: RDD[C], preservesPartitioning: Boolean)(f: (Iterator[T], Iterator[B], Iterator[C]) ⇒ Iterator[V])(implicit arg0: ClassTag[B], arg1: ClassTag[C], arg2: ClassTag[V]): RDD[V]

  122. def zipPartitions[B, V](rdd2: RDD[B])(f: (Iterator[T], Iterator[B]) ⇒ Iterator[V])(implicit arg0: ClassTag[B], arg1: ClassTag[V]): RDD[V]

  123. def zipPartitions[B, V](rdd2: RDD[B], preservesPartitioning: Boolean)(f: (Iterator[T], Iterator[B]) ⇒ Iterator[V])(implicit arg0: ClassTag[B], arg1: ClassTag[V]): RDD[V]

    Zip this RDD's partitions with one (or more) RDD(s) and return a new RDD by applying a function to the zipped partitions.

    Zip this RDD's partitions with one (or more) RDD(s) and return a new RDD by applying a function to the zipped partitions. Assumes that all the RDDs have the *same number of partitions*, but does *not* require them to have the same number of elements in each partition.

  124. def zipWithIndex(): RDD[(T, Long)]

    Zips this RDD with its element indices.

    Zips this RDD with its element indices. The ordering is first based on the partition index and then the ordering of items within each partition. So the first item in the first partition gets index 0, and the last item in the last partition receives the largest index.

    This is similar to Scala's zipWithIndex but it uses Long instead of Int as the index type. This method needs to trigger a spark job when this RDD contains more than one partitions.

    Note that some RDDs, such as those returned by groupBy(), do not guarantee order of elements in a partition. The index assigned to each element is therefore not guaranteed, and may even change if the RDD is reevaluated. If a fixed ordering is required to guarantee the same index assignments, you should sort the RDD with sortByKey() or save it to a file.

  125. def zipWithUniqueId(): RDD[(T, Long)]

    Zips this RDD with generated unique Long ids.

    Zips this RDD with generated unique Long ids. Items in the kth partition will get ids k, n+k, 2*n+k, ..., where n is the number of partitions. So there may exist gaps, but this method won't trigger a spark job, which is different from org.apache.spark.rdd.RDD#zipWithIndex.

    Note that some RDDs, such as those returned by groupBy(), do not guarantee order of elements in a partition. The unique ID assigned to each element is therefore not guaranteed, and may even change if the RDD is reevaluated. If a fixed ordering is required to guarantee the same index assignments, you should sort the RDD with sortByKey() or save it to a file.

Deprecated Value Members

  1. def filterWith[A](constructA: (Int) ⇒ A)(p: (T, A) ⇒ Boolean): RDD[T]

    Filters this RDD with p, where p takes an additional parameter of type A.

    Filters this RDD with p, where p takes an additional parameter of type A. This additional parameter is produced by constructA, which is called in each partition with the index of that partition.

    Annotations
    @deprecated
    Deprecated

    (Since version 1.0.0) use mapPartitionsWithIndex and filter

  2. def flatMapWith[A, U](constructA: (Int) ⇒ A, preservesPartitioning: Boolean = false)(f: (T, A) ⇒ Seq[U])(implicit arg0: ClassTag[U]): RDD[U]

    FlatMaps f over this RDD, where f takes an additional parameter of type A.

    FlatMaps f over this RDD, where f takes an additional parameter of type A. This additional parameter is produced by constructA, which is called in each partition with the index of that partition.

    Annotations
    @deprecated
    Deprecated

    (Since version 1.0.0) use mapPartitionsWithIndex and flatMap

  3. def foreachWith[A](constructA: (Int) ⇒ A)(f: (T, A) ⇒ Unit): Unit

    Applies f to each element of this RDD, where f takes an additional parameter of type A.

    Applies f to each element of this RDD, where f takes an additional parameter of type A. This additional parameter is produced by constructA, which is called in each partition with the index of that partition.

    Annotations
    @deprecated
    Deprecated

    (Since version 1.0.0) use mapPartitionsWithIndex and foreach

  4. def mapPartitionsWithContext[U](f: (TaskContext, Iterator[T]) ⇒ Iterator[U], preservesPartitioning: Boolean = false)(implicit arg0: ClassTag[U]): RDD[U]

    :: DeveloperApi :: Return a new RDD by applying a function to each partition of this RDD.

    :: DeveloperApi :: Return a new RDD by applying a function to each partition of this RDD. This is a variant of mapPartitions that also passes the TaskContext into the closure.

    preservesPartitioning indicates whether the input function preserves the partitioner, which should be false unless this is a pair RDD and the input function doesn't modify the keys.

    Annotations
    @DeveloperApi() @deprecated
    Deprecated

    (Since version 1.2.0) use TaskContext.get

  5. def mapPartitionsWithSplit[U](f: (Int, Iterator[T]) ⇒ Iterator[U], preservesPartitioning: Boolean = false)(implicit arg0: ClassTag[U]): RDD[U]

    Return a new RDD by applying a function to each partition of this RDD, while tracking the index of the original partition.

    Return a new RDD by applying a function to each partition of this RDD, while tracking the index of the original partition.

    Annotations
    @deprecated
    Deprecated

    (Since version 0.7.0) use mapPartitionsWithIndex

  6. def mapWith[A, U](constructA: (Int) ⇒ A, preservesPartitioning: Boolean = false)(f: (T, A) ⇒ U)(implicit arg0: ClassTag[U]): RDD[U]

    Maps f over this RDD, where f takes an additional parameter of type A.

    Maps f over this RDD, where f takes an additional parameter of type A. This additional parameter is produced by constructA, which is called in each partition with the index of that partition.

    Annotations
    @deprecated
    Deprecated

    (Since version 1.0.0) use mapPartitionsWithIndex

  7. def toArray(): Array[T]

    Return an array that contains all of the elements in this RDD.

    Return an array that contains all of the elements in this RDD.

    Annotations
    @deprecated
    Deprecated

    (Since version 1.0.0) use collect

Inherited from Logging

Inherited from Serializable

Inherited from Serializable

Inherited from AnyRef

Inherited from Any

Ungrouped