org.apache.spark.rdd

PairRDDFunctions

class PairRDDFunctions[K, V] extends Logging with SparkHadoopMapReduceUtil with Serializable

Extra functions available on RDDs of (key, value) pairs through an implicit conversion. Import org.apache.spark.SparkContext._ at the top of your program to use these functions.

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

Instance Constructors

  1. new PairRDDFunctions(self: RDD[(K, V)])(implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null)

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. final def ==(arg0: AnyRef): Boolean

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

    Definition Classes
    Any
  6. def aggregateByKey[U](zeroValue: U)(seqOp: (U, V) ⇒ U, combOp: (U, U) ⇒ U)(implicit arg0: ClassTag[U]): RDD[(K, U)]

    Aggregate the values of each key, using given combine functions and a neutral "zero value".

    Aggregate the values of each key, using given combine functions and a neutral "zero value". This function can return a different result type, U, than the type of the values in this RDD, V. Thus, we need one operation for merging a V into a U and one operation for merging two U's, as in scala.TraversableOnce. The former operation is used for merging values within a partition, and the latter is used for merging values between partitions. To avoid memory allocation, both of these functions are allowed to modify and return their first argument instead of creating a new U.

  7. def aggregateByKey[U](zeroValue: U, numPartitions: Int)(seqOp: (U, V) ⇒ U, combOp: (U, U) ⇒ U)(implicit arg0: ClassTag[U]): RDD[(K, U)]

    Aggregate the values of each key, using given combine functions and a neutral "zero value".

    Aggregate the values of each key, using given combine functions and a neutral "zero value". This function can return a different result type, U, than the type of the values in this RDD, V. Thus, we need one operation for merging a V into a U and one operation for merging two U's, as in scala.TraversableOnce. The former operation is used for merging values within a partition, and the latter is used for merging values between partitions. To avoid memory allocation, both of these functions are allowed to modify and return their first argument instead of creating a new U.

  8. def aggregateByKey[U](zeroValue: U, partitioner: Partitioner)(seqOp: (U, V) ⇒ U, combOp: (U, U) ⇒ U)(implicit arg0: ClassTag[U]): RDD[(K, U)]

    Aggregate the values of each key, using given combine functions and a neutral "zero value".

    Aggregate the values of each key, using given combine functions and a neutral "zero value". This function can return a different result type, U, than the type of the values in this RDD, V. Thus, we need one operation for merging a V into a U and one operation for merging two U's, as in scala.TraversableOnce. The former operation is used for merging values within a partition, and the latter is used for merging values between partitions. To avoid memory allocation, both of these functions are allowed to modify and return their first argument instead of creating a new U.

  9. final def asInstanceOf[T0]: T0

    Definition Classes
    Any
  10. def clone(): AnyRef

    Attributes
    protected[java.lang]
    Definition Classes
    AnyRef
    Annotations
    @throws( ... )
  11. def cogroup[W1, W2, W3](other1: RDD[(K, W1)], other2: RDD[(K, W2)], other3: RDD[(K, W3)], numPartitions: Int): RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2], Iterable[W3]))]

    For each key k in this or other1 or other2 or other3, return a resulting RDD that contains a tuple with the list of values for that key in this, other1, other2 and other3.

  12. def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], numPartitions: Int): RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))]

    For each key k in this or other1 or other2, return a resulting RDD that contains a tuple with the list of values for that key in this, other1 and other2.

  13. def cogroup[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Iterable[V], Iterable[W]))]

    For each key k in this or other, return a resulting RDD that contains a tuple with the list of values for that key in this as well as other.

  14. def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]): RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))]

    For each key k in this or other1 or other2, return a resulting RDD that contains a tuple with the list of values for that key in this, other1 and other2.

  15. def cogroup[W](other: RDD[(K, W)]): RDD[(K, (Iterable[V], Iterable[W]))]

    For each key k in this or other, return a resulting RDD that contains a tuple with the list of values for that key in this as well as other.

  16. def cogroup[W1, W2, W3](other1: RDD[(K, W1)], other2: RDD[(K, W2)], other3: RDD[(K, W3)]): RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2], Iterable[W3]))]

    For each key k in this or other1 or other2 or other3, return a resulting RDD that contains a tuple with the list of values for that key in this, other1, other2 and other3.

  17. def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], partitioner: Partitioner): RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))]

    For each key k in this or other1 or other2, return a resulting RDD that contains a tuple with the list of values for that key in this, other1 and other2.

  18. def cogroup[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (Iterable[V], Iterable[W]))]

    For each key k in this or other, return a resulting RDD that contains a tuple with the list of values for that key in this as well as other.

  19. def cogroup[W1, W2, W3](other1: RDD[(K, W1)], other2: RDD[(K, W2)], other3: RDD[(K, W3)], partitioner: Partitioner): RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2], Iterable[W3]))]

    For each key k in this or other1 or other2 or other3, return a resulting RDD that contains a tuple with the list of values for that key in this, other1, other2 and other3.

  20. def collectAsMap(): Map[K, V]

    Return the key-value pairs in this RDD to the master as a Map.

    Return the key-value pairs in this RDD to the master as a Map.

    Warning: this doesn't return a multimap (so if you have multiple values to the same key, only one value per key is preserved in the map returned)

  21. def combineByKey[C](createCombiner: (V) ⇒ C, mergeValue: (C, V) ⇒ C, mergeCombiners: (C, C) ⇒ C): RDD[(K, C)]

    Simplified version of combineByKey that hash-partitions the resulting RDD using the existing partitioner/parallelism level.

  22. def combineByKey[C](createCombiner: (V) ⇒ C, mergeValue: (C, V) ⇒ C, mergeCombiners: (C, C) ⇒ C, numPartitions: Int): RDD[(K, C)]

    Simplified version of combineByKey that hash-partitions the output RDD.

  23. def combineByKey[C](createCombiner: (V) ⇒ C, mergeValue: (C, V) ⇒ C, mergeCombiners: (C, C) ⇒ C, partitioner: Partitioner, mapSideCombine: Boolean = true, serializer: Serializer = null): RDD[(K, C)]

    Generic function to combine the elements for each key using a custom set of aggregation functions.

    Generic function to combine the elements for each key using a custom set of aggregation functions. Turns an RDD[(K, V)] into a result of type RDD[(K, C)], for a "combined type" C Note that V and C can be different -- for example, one might group an RDD of type (Int, Int) into an RDD of type (Int, Seq[Int]). Users provide three functions:

    - createCombiner, which turns a V into a C (e.g., creates a one-element list) - mergeValue, to merge a V into a C (e.g., adds it to the end of a list) - mergeCombiners, to combine two C's into a single one.

    In addition, users can control the partitioning of the output RDD, and whether to perform map-side aggregation (if a mapper can produce multiple items with the same key).

  24. def countApproxDistinctByKey(relativeSD: Double = 0.05): RDD[(K, Long)]

    Return approximate number of distinct values for each key in this RDD.

    Return approximate number of distinct values for each key in this 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.

  25. def countApproxDistinctByKey(relativeSD: Double, numPartitions: Int): RDD[(K, Long)]

    Return approximate number of distinct values for each key in this RDD.

    Return approximate number of distinct values for each key in this 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.

    numPartitions

    number of partitions of the resulting RDD

  26. def countApproxDistinctByKey(relativeSD: Double, partitioner: Partitioner): RDD[(K, Long)]

    Return approximate number of distinct values for each key in this RDD.

    Return approximate number of distinct values for each key in this 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.

    partitioner

    partitioner of the resulting RDD

  27. def countApproxDistinctByKey(p: Int, sp: Int, partitioner: Partitioner): RDD[(K, Long)]

    :: Experimental ::

    :: Experimental ::

    Return approximate number of distinct values for each key in this 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.

    partitioner

    Partitioner to use for the resulting RDD.

    Annotations
    @Experimental()
  28. def countByKey(): Map[K, Long]

    Count the number of elements for each key, collecting the results to a local Map.

    Count the number of elements for each key, collecting the results to a local Map.

    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.mapValues(_ => 1L).reduceByKey(_ + _), which returns an RDD[T, Long] instead of a map.

  29. def countByKeyApprox(timeout: Long, confidence: Double = 0.95): PartialResult[Map[K, BoundedDouble]]

    :: Experimental :: Approximate version of countByKey that can return a partial result if it does not finish within a timeout.

    :: Experimental :: Approximate version of countByKey that can return a partial result if it does not finish within a timeout.

    Annotations
    @Experimental()
  30. final def eq(arg0: AnyRef): Boolean

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

    Definition Classes
    AnyRef → Any
  32. def finalize(): Unit

    Attributes
    protected[java.lang]
    Definition Classes
    AnyRef
    Annotations
    @throws( classOf[java.lang.Throwable] )
  33. def flatMapValues[U](f: (V) ⇒ TraversableOnce[U]): RDD[(K, U)]

    Pass each value in the key-value pair RDD through a flatMap function without changing the keys; this also retains the original RDD's partitioning.

  34. def foldByKey(zeroValue: V)(func: (V, V) ⇒ V): RDD[(K, V)]

    Merge the values for each key using an associative function and a neutral "zero value" which may be added to the result an arbitrary number of times, and must not change the result (e.

    Merge the values for each key using an associative function and a neutral "zero value" which may be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for list concatenation, 0 for addition, or 1 for multiplication.).

  35. def foldByKey(zeroValue: V, numPartitions: Int)(func: (V, V) ⇒ V): RDD[(K, V)]

    Merge the values for each key using an associative function and a neutral "zero value" which may be added to the result an arbitrary number of times, and must not change the result (e.

    Merge the values for each key using an associative function and a neutral "zero value" which may be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for list concatenation, 0 for addition, or 1 for multiplication.).

  36. def foldByKey(zeroValue: V, partitioner: Partitioner)(func: (V, V) ⇒ V): RDD[(K, V)]

    Merge the values for each key using an associative function and a neutral "zero value" which may be added to the result an arbitrary number of times, and must not change the result (e.

    Merge the values for each key using an associative function and a neutral "zero value" which may be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for list concatenation, 0 for addition, or 1 for multiplication.).

  37. def fullOuterJoin[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Option[V], Option[W]))]

    Perform a full outer join of this and other.

    Perform a full outer join of this and other. For each element (k, v) in this, the resulting RDD will either contain all pairs (k, (Some(v), Some(w))) for w in other, or the pair (k, (Some(v), None)) if no elements in other have key k. Similarly, for each element (k, w) in other, the resulting RDD will either contain all pairs (k, (Some(v), Some(w))) for v in this, or the pair (k, (None, Some(w))) if no elements in this have key k. Hash-partitions the resulting RDD into the given number of partitions.

  38. def fullOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (Option[V], Option[W]))]

    Perform a full outer join of this and other.

    Perform a full outer join of this and other. For each element (k, v) in this, the resulting RDD will either contain all pairs (k, (Some(v), Some(w))) for w in other, or the pair (k, (Some(v), None)) if no elements in other have key k. Similarly, for each element (k, w) in other, the resulting RDD will either contain all pairs (k, (Some(v), Some(w))) for v in this, or the pair (k, (None, Some(w))) if no elements in this have key k. Hash-partitions the resulting RDD using the existing partitioner/ parallelism level.

  39. def fullOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (Option[V], Option[W]))]

    Perform a full outer join of this and other.

    Perform a full outer join of this and other. For each element (k, v) in this, the resulting RDD will either contain all pairs (k, (Some(v), Some(w))) for w in other, or the pair (k, (Some(v), None)) if no elements in other have key k. Similarly, for each element (k, w) in other, the resulting RDD will either contain all pairs (k, (Some(v), Some(w))) for v in this, or the pair (k, (None, Some(w))) if no elements in this have key k. Uses the given Partitioner to partition the output RDD.

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

    Definition Classes
    AnyRef → Any
  41. def groupByKey(): RDD[(K, Iterable[V])]

    Group the values for each key in the RDD into a single sequence.

    Group the values for each key in the RDD into a single sequence. Hash-partitions the resulting RDD with the existing partitioner/parallelism level. 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.

  42. def groupByKey(numPartitions: Int): RDD[(K, Iterable[V])]

    Group the values for each key in the RDD into a single sequence.

    Group the values for each key in the RDD into a single sequence. Hash-partitions the resulting RDD with into numPartitions partitions. 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.

  43. def groupByKey(partitioner: Partitioner): RDD[(K, Iterable[V])]

    Group the values for each key in the RDD into a single sequence.

    Group the values for each key in the RDD into a single sequence. Allows controlling the partitioning of the resulting key-value pair RDD by passing a Partitioner. 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 groupWith[W1, W2, W3](other1: RDD[(K, W1)], other2: RDD[(K, W2)], other3: RDD[(K, W3)]): RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2], Iterable[W3]))]

    Alias for cogroup.

  45. def groupWith[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]): RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))]

    Alias for cogroup.

  46. def groupWith[W](other: RDD[(K, W)]): RDD[(K, (Iterable[V], Iterable[W]))]

    Alias for cogroup.

  47. def hashCode(): Int

    Definition Classes
    AnyRef → Any
  48. final def isInstanceOf[T0]: Boolean

    Definition Classes
    Any
  49. def isTraceEnabled(): Boolean

    Attributes
    protected
    Definition Classes
    Logging
  50. def join[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, W))]

    Return an RDD containing all pairs of elements with matching keys in this and other.

    Return an RDD containing all pairs of elements with matching keys in this and other. Each pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in this and (k, v2) is in other. Performs a hash join across the cluster.

  51. def join[W](other: RDD[(K, W)]): RDD[(K, (V, W))]

    Return an RDD containing all pairs of elements with matching keys in this and other.

    Return an RDD containing all pairs of elements with matching keys in this and other. Each pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in this and (k, v2) is in other. Performs a hash join across the cluster.

  52. def join[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))]

    Return an RDD containing all pairs of elements with matching keys in this and other.

    Return an RDD containing all pairs of elements with matching keys in this and other. Each pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in this and (k, v2) is in other. Uses the given Partitioner to partition the output RDD.

  53. def keys: RDD[K]

    Return an RDD with the keys of each tuple.

  54. def leftOuterJoin[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, Option[W]))]

    Perform a left outer join of this and other.

    Perform a left outer join of this and other. For each element (k, v) in this, the resulting RDD will either contain all pairs (k, (v, Some(w))) for w in other, or the pair (k, (v, None)) if no elements in other have key k. Hash-partitions the output into numPartitions partitions.

  55. def leftOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (V, Option[W]))]

    Perform a left outer join of this and other.

    Perform a left outer join of this and other. For each element (k, v) in this, the resulting RDD will either contain all pairs (k, (v, Some(w))) for w in other, or the pair (k, (v, None)) if no elements in other have key k. Hash-partitions the output using the existing partitioner/parallelism level.

  56. def leftOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, Option[W]))]

    Perform a left outer join of this and other.

    Perform a left outer join of this and other. For each element (k, v) in this, the resulting RDD will either contain all pairs (k, (v, Some(w))) for w in other, or the pair (k, (v, None)) if no elements in other have key k. Uses the given Partitioner to partition the output RDD.

  57. def log: Logger

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

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

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

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

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

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

    Attributes
    protected
    Definition Classes
    Logging
  64. def logName: String

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

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

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

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

    Attributes
    protected
    Definition Classes
    Logging
  69. def lookup(key: K): Seq[V]

    Return the list of values in the RDD for key key.

    Return the list of values in the RDD for key key. This operation is done efficiently if the RDD has a known partitioner by only searching the partition that the key maps to.

  70. def mapValues[U](f: (V) ⇒ U): RDD[(K, U)]

    Pass each value in the key-value pair RDD through a map function without changing the keys; this also retains the original RDD's partitioning.

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

    Definition Classes
    AnyRef
  72. def newJobContext(conf: Configuration, jobId: JobID): JobContext

    Definition Classes
    SparkHadoopMapReduceUtil
  73. def newTaskAttemptContext(conf: Configuration, attemptId: TaskAttemptID): TaskAttemptContext

    Definition Classes
    SparkHadoopMapReduceUtil
  74. def newTaskAttemptID(jtIdentifier: String, jobId: Int, isMap: Boolean, taskId: Int, attemptId: Int): TaskAttemptID

    Definition Classes
    SparkHadoopMapReduceUtil
  75. final def notify(): Unit

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

    Definition Classes
    AnyRef
  77. def partitionBy(partitioner: Partitioner): RDD[(K, V)]

    Return a copy of the RDD partitioned using the specified partitioner.

  78. def reduceByKey(func: (V, V) ⇒ V): RDD[(K, V)]

    Merge the values for each key using an associative reduce function.

    Merge the values for each key using an associative reduce function. This will also perform the merging locally on each mapper before sending results to a reducer, similarly to a "combiner" in MapReduce. Output will be hash-partitioned with the existing partitioner/ parallelism level.

  79. def reduceByKey(func: (V, V) ⇒ V, numPartitions: Int): RDD[(K, V)]

    Merge the values for each key using an associative reduce function.

    Merge the values for each key using an associative reduce function. This will also perform the merging locally on each mapper before sending results to a reducer, similarly to a "combiner" in MapReduce. Output will be hash-partitioned with numPartitions partitions.

  80. def reduceByKey(partitioner: Partitioner, func: (V, V) ⇒ V): RDD[(K, V)]

    Merge the values for each key using an associative reduce function.

    Merge the values for each key using an associative reduce function. This will also perform the merging locally on each mapper before sending results to a reducer, similarly to a "combiner" in MapReduce.

  81. def reduceByKeyLocally(func: (V, V) ⇒ V): Map[K, V]

    Merge the values for each key using an associative reduce function, but return the results immediately to the master as a Map.

    Merge the values for each key using an associative reduce function, but return the results immediately to the master as a Map. This will also perform the merging locally on each mapper before sending results to a reducer, similarly to a "combiner" in MapReduce.

  82. def rightOuterJoin[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Option[V], W))]

    Perform a right outer join of this and other.

    Perform a right outer join of this and other. For each element (k, w) in other, the resulting RDD will either contain all pairs (k, (Some(v), w)) for v in this, or the pair (k, (None, w)) if no elements in this have key k. Hash-partitions the resulting RDD into the given number of partitions.

  83. def rightOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (Option[V], W))]

    Perform a right outer join of this and other.

    Perform a right outer join of this and other. For each element (k, w) in other, the resulting RDD will either contain all pairs (k, (Some(v), w)) for v in this, or the pair (k, (None, w)) if no elements in this have key k. Hash-partitions the resulting RDD using the existing partitioner/parallelism level.

  84. def rightOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (Option[V], W))]

    Perform a right outer join of this and other.

    Perform a right outer join of this and other. For each element (k, w) in other, the resulting RDD will either contain all pairs (k, (Some(v), w)) for v in this, or the pair (k, (None, w)) if no elements in this have key k. Uses the given Partitioner to partition the output RDD.

  85. def sampleByKey(withReplacement: Boolean, fractions: Map[K, Double], seed: Long = Utils.random.nextLong): RDD[(K, V)]

    Return a subset of this RDD sampled by key (via stratified sampling).

    Return a subset of this RDD sampled by key (via stratified sampling).

    Create a sample of this RDD using variable sampling rates for different keys as specified by fractions, a key to sampling rate map, via simple random sampling with one pass over the RDD, to produce a sample of size that's approximately equal to the sum of math.ceil(numItems * samplingRate) over all key values.

    withReplacement

    whether to sample with or without replacement

    fractions

    map of specific keys to sampling rates

    seed

    seed for the random number generator

    returns

    RDD containing the sampled subset

  86. def sampleByKeyExact(withReplacement: Boolean, fractions: Map[K, Double], seed: Long = Utils.random.nextLong): RDD[(K, V)]

    ::Experimental:: Return a subset of this RDD sampled by key (via stratified sampling) containing exactly math.

    ::Experimental:: Return a subset of this RDD sampled by key (via stratified sampling) containing exactly math.ceil(numItems * samplingRate) for each stratum (group of pairs with the same key).

    This method differs from sampleByKey in that we make additional passes over the RDD to create a sample size that's exactly equal to the sum of math.ceil(numItems * samplingRate) over all key values with a 99.99% confidence. When sampling without replacement, we need one additional pass over the RDD to guarantee sample size; when sampling with replacement, we need two additional passes.

    withReplacement

    whether to sample with or without replacement

    fractions

    map of specific keys to sampling rates

    seed

    seed for the random number generator

    returns

    RDD containing the sampled subset

    Annotations
    @Experimental()
  87. def saveAsHadoopDataset(conf: JobConf): Unit

    Output the RDD to any Hadoop-supported storage system, using a Hadoop JobConf object for that storage system.

    Output the RDD to any Hadoop-supported storage system, using a Hadoop JobConf object for that storage system. The JobConf should set an OutputFormat and any output paths required (e.g. a table name to write to) in the same way as it would be configured for a Hadoop MapReduce job.

  88. def saveAsHadoopFile(path: String, keyClass: Class[_], valueClass: Class[_], outputFormatClass: Class[_ <: OutputFormat[_, _]], conf: JobConf = ..., codec: Option[Class[_ <: CompressionCodec]] = None): Unit

    Output the RDD to any Hadoop-supported file system, using a Hadoop OutputFormat class supporting the key and value types K and V in this RDD.

  89. def saveAsHadoopFile(path: String, keyClass: Class[_], valueClass: Class[_], outputFormatClass: Class[_ <: OutputFormat[_, _]], codec: Class[_ <: CompressionCodec]): Unit

    Output the RDD to any Hadoop-supported file system, using a Hadoop OutputFormat class supporting the key and value types K and V in this RDD.

    Output the RDD to any Hadoop-supported file system, using a Hadoop OutputFormat class supporting the key and value types K and V in this RDD. Compress with the supplied codec.

  90. def saveAsHadoopFile[F <: OutputFormat[K, V]](path: String, codec: Class[_ <: CompressionCodec])(implicit fm: ClassTag[F]): Unit

    Output the RDD to any Hadoop-supported file system, using a Hadoop OutputFormat class supporting the key and value types K and V in this RDD.

    Output the RDD to any Hadoop-supported file system, using a Hadoop OutputFormat class supporting the key and value types K and V in this RDD. Compress the result with the supplied codec.

  91. def saveAsHadoopFile[F <: OutputFormat[K, V]](path: String)(implicit fm: ClassTag[F]): Unit

    Output the RDD to any Hadoop-supported file system, using a Hadoop OutputFormat class supporting the key and value types K and V in this RDD.

  92. def saveAsNewAPIHadoopDataset(conf: Configuration): Unit

    Output the RDD to any Hadoop-supported storage system with new Hadoop API, using a Hadoop Configuration object for that storage system.

    Output the RDD to any Hadoop-supported storage system with new Hadoop API, using a Hadoop Configuration object for that storage system. The Conf should set an OutputFormat and any output paths required (e.g. a table name to write to) in the same way as it would be configured for a Hadoop MapReduce job.

  93. def saveAsNewAPIHadoopFile(path: String, keyClass: Class[_], valueClass: Class[_], outputFormatClass: Class[_ <: OutputFormat[_, _]], conf: Configuration = self.context.hadoopConfiguration): Unit

    Output the RDD to any Hadoop-supported file system, using a new Hadoop API OutputFormat (mapreduce.

    Output the RDD to any Hadoop-supported file system, using a new Hadoop API OutputFormat (mapreduce.OutputFormat) object supporting the key and value types K and V in this RDD.

  94. def saveAsNewAPIHadoopFile[F <: OutputFormat[K, V]](path: String)(implicit fm: ClassTag[F]): Unit

    Output the RDD to any Hadoop-supported file system, using a new Hadoop API OutputFormat (mapreduce.

    Output the RDD to any Hadoop-supported file system, using a new Hadoop API OutputFormat (mapreduce.OutputFormat) object supporting the key and value types K and V in this RDD.

  95. def subtractByKey[W](other: RDD[(K, W)], p: Partitioner)(implicit arg0: ClassTag[W]): RDD[(K, V)]

    Return an RDD with the pairs from this whose keys are not in other.

  96. def subtractByKey[W](other: RDD[(K, W)], numPartitions: Int)(implicit arg0: ClassTag[W]): RDD[(K, V)]

    Return an RDD with the pairs from this whose keys are not in other.

  97. def subtractByKey[W](other: RDD[(K, W)])(implicit arg0: ClassTag[W]): RDD[(K, V)]

    Return an RDD with the pairs from this whose keys are not in other.

    Return an RDD with the pairs from this whose keys are not in other.

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

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

    Definition Classes
    AnyRef
  99. def toString(): String

    Definition Classes
    AnyRef → Any
  100. def values: RDD[V]

    Return an RDD with the values of each tuple.

  101. final def wait(): Unit

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

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

    Definition Classes
    AnyRef
    Annotations
    @throws( ... )

Deprecated Value Members

  1. def reduceByKeyToDriver(func: (V, V) ⇒ V): Map[K, V]

    Alias for reduceByKeyLocally

    Alias for reduceByKeyLocally

    Annotations
    @deprecated
    Deprecated

    (Since version 1.0.0) Use reduceByKeyLocally

Inherited from Serializable

Inherited from Serializable

Inherited from SparkHadoopMapReduceUtil

Inherited from Logging

Inherited from AnyRef

Inherited from Any

Ungrouped