spark

PairRDDFunctions

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

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

Linear Supertypes
Serializable, Serializable, HadoopMapReduceUtil, Logging, AnyRef, Any
Ordering
  1. Alphabetic
  2. By inheritance
Inherited
  1. Hide All
  2. Show all
  1. PairRDDFunctions
  2. Serializable
  3. Serializable
  4. HadoopMapReduceUtil
  5. Logging
  6. AnyRef
  7. Any
Visibility
  1. Public
  2. All

Instance Constructors

  1. new PairRDDFunctions(self: RDD[(K, V)])(implicit arg0: ClassManifest[K], arg1: ClassManifest[V])

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. final def asInstanceOf[T0]: T0

    Definition Classes
    Any
  7. def clone(): AnyRef

    Attributes
    protected[lang]
    Definition Classes
    AnyRef
    Annotations
    @throws()
  8. def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], numPartitions: Int): RDD[(K, (Seq[V], Seq[W1], Seq[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.

  9. def cogroup[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Seq[V], Seq[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.

  10. def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]): RDD[(K, (Seq[V], Seq[W1], Seq[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.

  11. def cogroup[W](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[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.

  12. def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], partitioner: Partitioner): RDD[(K, (Seq[V], Seq[W1], Seq[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)], partitioner: Partitioner): RDD[(K, (Seq[V], Seq[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 collectAsMap(): Map[K, V]

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

  15. 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.

  16. 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.

  17. def combineByKey[C](createCombiner: (V) ⇒ C, mergeValue: (C, V) ⇒ C, mergeCombiners: (C, C) ⇒ C, partitioner: Partitioner, mapSideCombine: Boolean): 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).

  18. def countByKey(): Map[K, Long]

    Count the number of elements for each key, and return the result to the master as a Map.

  19. 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.

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

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

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

    Attributes
    protected[lang]
    Definition Classes
    AnyRef
    Annotations
    @throws()
  23. 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.

  24. final def getClass(): java.lang.Class[_]

    Definition Classes
    AnyRef → Any
  25. def groupByKey(): RDD[(K, Seq[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.

  26. def groupByKey(numPartitions: Int): RDD[(K, Seq[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.

  27. def groupByKey(partitioner: Partitioner): RDD[(K, Seq[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.

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

    Alias for cogroup.

  29. def groupWith[W](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))]

    Alias for cogroup.

  30. def hashCode(): Int

    Definition Classes
    AnyRef → Any
  31. def initLogging(): Unit

    Attributes
    protected
    Definition Classes
    Logging
  32. final def isInstanceOf[T0]: Boolean

    Definition Classes
    Any
  33. 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.

  34. 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.

  35. 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.

  36. def keys: RDD[K]

    Return an RDD with the keys of each tuple.

  37. 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.

  38. 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.

  39. 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.

  40. def log: Logger

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

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

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

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

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

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

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

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

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

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

    Attributes
    protected
    Definition Classes
    Logging
  51. 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.

  52. 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.

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

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

    Definition Classes
    HadoopMapReduceUtil
  55. def newTaskAttemptContext(conf: Configuration, attemptId: TaskAttemptID): TaskAttemptContext

    Definition Classes
    HadoopMapReduceUtil
  56. final def notify(): Unit

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

    Definition Classes
    AnyRef
  58. def partitionBy(partitioner: Partitioner, mapSideCombine: Boolean = false): RDD[(K, V)]

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

    Return a copy of the RDD partitioned using the specified partitioner. If mapSideCombine is true, Spark will group values of the same key together on the map side before the repartitioning, to only send each key over the network once. If a large number of duplicated keys are expected, and the size of the keys are large, mapSideCombine should be set to true.

  59. 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.

  60. 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.

  61. 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.

  62. 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.

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

    Alias for reduceByKeyLocally

  64. 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.

  65. 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.

  66. 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.

  67. 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.

  68. def saveAsHadoopFile(path: String, keyClass: Class[_], valueClass: Class[_], outputFormatClass: Class[_ <: org.apache.hadoop.mapred.OutputFormat[_, _]], conf: JobConf = ...): 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.

  69. def saveAsHadoopFile[F <: OutputFormat[K, V]](path: String)(implicit fm: ClassManifest[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.

  70. def saveAsNewAPIHadoopFile(path: String, keyClass: Class[_], valueClass: Class[_], outputFormatClass: Class[_ <: org.apache.hadoop.mapreduce.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.

  71. def saveAsNewAPIHadoopFile[F <: OutputFormat[K, V]](path: String)(implicit fm: ClassManifest[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.

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

    Definition Classes
    AnyRef
  73. def toString(): String

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

    Return an RDD with the values of each tuple.

  75. final def wait(): Unit

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

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

    Definition Classes
    AnyRef
    Annotations
    @throws()

Inherited from Serializable

Inherited from Serializable

Inherited from HadoopMapReduceUtil

Inherited from Logging

Inherited from AnyRef

Inherited from Any