Class PairRDDFunctions<K,V>

Object
org.apache.spark.rdd.PairRDDFunctions<K,V>
All Implemented Interfaces:
Serializable, org.apache.spark.internal.Logging, scala.Serializable

public class PairRDDFunctions<K,V> extends Object implements org.apache.spark.internal.Logging, scala.Serializable
Extra functions available on RDDs of (key, value) pairs through an implicit conversion.
See Also:
  • Nested Class Summary

    Nested classes/interfaces inherited from interface org.apache.spark.internal.Logging

    org.apache.spark.internal.Logging.SparkShellLoggingFilter
  • Constructor Summary

    Constructors
    Constructor
    Description
    PairRDDFunctions(RDD<scala.Tuple2<K,V>> self, scala.reflect.ClassTag<K> kt, scala.reflect.ClassTag<V> vt, scala.math.Ordering<K> ord)
     
  • Method Summary

    Modifier and Type
    Method
    Description
    <U> RDD<scala.Tuple2<K,U>>
    aggregateByKey(U zeroValue, int numPartitions, scala.Function2<U,V,U> seqOp, scala.Function2<U,U,U> combOp, scala.reflect.ClassTag<U> evidence$2)
    Aggregate the values of each key, using given combine functions and a neutral "zero value".
    <U> RDD<scala.Tuple2<K,U>>
    aggregateByKey(U zeroValue, Partitioner partitioner, scala.Function2<U,V,U> seqOp, scala.Function2<U,U,U> combOp, scala.reflect.ClassTag<U> evidence$1)
    Aggregate the values of each key, using given combine functions and a neutral "zero value".
    <U> RDD<scala.Tuple2<K,U>>
    aggregateByKey(U zeroValue, scala.Function2<U,V,U> seqOp, scala.Function2<U,U,U> combOp, scala.reflect.ClassTag<U> evidence$3)
    Aggregate the values of each key, using given combine functions and a neutral "zero value".
    <W> RDD<scala.Tuple2<K,scala.Tuple2<scala.collection.Iterable<V>,scala.collection.Iterable<W>>>>
    cogroup(RDD<scala.Tuple2<K,W>> other)
    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.
    <W> RDD<scala.Tuple2<K,scala.Tuple2<scala.collection.Iterable<V>,scala.collection.Iterable<W>>>>
    cogroup(RDD<scala.Tuple2<K,W>> other, int numPartitions)
    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.
    <W> RDD<scala.Tuple2<K,scala.Tuple2<scala.collection.Iterable<V>,scala.collection.Iterable<W>>>>
    cogroup(RDD<scala.Tuple2<K,W>> other, Partitioner partitioner)
    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.
    <W1, W2> RDD<scala.Tuple2<K,scala.Tuple3<scala.collection.Iterable<V>,scala.collection.Iterable<W1>,scala.collection.Iterable<W2>>>>
    cogroup(RDD<scala.Tuple2<K,W1>> other1, RDD<scala.Tuple2<K,W2>> other2)
    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.
    <W1, W2> RDD<scala.Tuple2<K,scala.Tuple3<scala.collection.Iterable<V>,scala.collection.Iterable<W1>,scala.collection.Iterable<W2>>>>
    cogroup(RDD<scala.Tuple2<K,W1>> other1, RDD<scala.Tuple2<K,W2>> other2, int numPartitions)
    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.
    <W1, W2> RDD<scala.Tuple2<K,scala.Tuple3<scala.collection.Iterable<V>,scala.collection.Iterable<W1>,scala.collection.Iterable<W2>>>>
    cogroup(RDD<scala.Tuple2<K,W1>> other1, RDD<scala.Tuple2<K,W2>> other2, Partitioner partitioner)
    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.
    <W1, W2, W3> RDD<scala.Tuple2<K,scala.Tuple4<scala.collection.Iterable<V>,scala.collection.Iterable<W1>,scala.collection.Iterable<W2>,scala.collection.Iterable<W3>>>>
    cogroup(RDD<scala.Tuple2<K,W1>> other1, RDD<scala.Tuple2<K,W2>> other2, RDD<scala.Tuple2<K,W3>> other3)
    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.
    <W1, W2, W3> RDD<scala.Tuple2<K,scala.Tuple4<scala.collection.Iterable<V>,scala.collection.Iterable<W1>,scala.collection.Iterable<W2>,scala.collection.Iterable<W3>>>>
    cogroup(RDD<scala.Tuple2<K,W1>> other1, RDD<scala.Tuple2<K,W2>> other2, RDD<scala.Tuple2<K,W3>> other3, int numPartitions)
    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.
    <W1, W2, W3> RDD<scala.Tuple2<K,scala.Tuple4<scala.collection.Iterable<V>,scala.collection.Iterable<W1>,scala.collection.Iterable<W2>,scala.collection.Iterable<W3>>>>
    cogroup(RDD<scala.Tuple2<K,W1>> other1, RDD<scala.Tuple2<K,W2>> other2, RDD<scala.Tuple2<K,W3>> other3, Partitioner partitioner)
    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.
    scala.collection.Map<K,V>
    Return the key-value pairs in this RDD to the master as a Map.
    <C> RDD<scala.Tuple2<K,C>>
    combineByKey(scala.Function1<V,C> createCombiner, scala.Function2<C,V,C> mergeValue, scala.Function2<C,C,C> mergeCombiners)
    Simplified version of combineByKeyWithClassTag that hash-partitions the resulting RDD using the existing partitioner/parallelism level.
    <C> RDD<scala.Tuple2<K,C>>
    combineByKey(scala.Function1<V,C> createCombiner, scala.Function2<C,V,C> mergeValue, scala.Function2<C,C,C> mergeCombiners, int numPartitions)
    Simplified version of combineByKeyWithClassTag that hash-partitions the output RDD.
    <C> RDD<scala.Tuple2<K,C>>
    combineByKey(scala.Function1<V,C> createCombiner, scala.Function2<C,V,C> mergeValue, scala.Function2<C,C,C> mergeCombiners, Partitioner partitioner, boolean mapSideCombine, Serializer serializer)
    Generic function to combine the elements for each key using a custom set of aggregation functions.
    <C> RDD<scala.Tuple2<K,C>>
    combineByKeyWithClassTag(scala.Function1<V,C> createCombiner, scala.Function2<C,V,C> mergeValue, scala.Function2<C,C,C> mergeCombiners, int numPartitions, scala.reflect.ClassTag<C> ct)
    Simplified version of combineByKeyWithClassTag that hash-partitions the output RDD.
    <C> RDD<scala.Tuple2<K,C>>
    combineByKeyWithClassTag(scala.Function1<V,C> createCombiner, scala.Function2<C,V,C> mergeValue, scala.Function2<C,C,C> mergeCombiners, Partitioner partitioner, boolean mapSideCombine, Serializer serializer, scala.reflect.ClassTag<C> ct)
    Generic function to combine the elements for each key using a custom set of aggregation functions.
    <C> RDD<scala.Tuple2<K,C>>
    combineByKeyWithClassTag(scala.Function1<V,C> createCombiner, scala.Function2<C,V,C> mergeValue, scala.Function2<C,C,C> mergeCombiners, scala.reflect.ClassTag<C> ct)
    Simplified version of combineByKeyWithClassTag that hash-partitions the resulting RDD using the existing partitioner/parallelism level.
    RDD<scala.Tuple2<K,Object>>
    countApproxDistinctByKey(double relativeSD)
    Return approximate number of distinct values for each key in this RDD.
    RDD<scala.Tuple2<K,Object>>
    countApproxDistinctByKey(double relativeSD, int numPartitions)
    Return approximate number of distinct values for each key in this RDD.
    RDD<scala.Tuple2<K,Object>>
    countApproxDistinctByKey(double relativeSD, Partitioner partitioner)
    Return approximate number of distinct values for each key in this RDD.
    RDD<scala.Tuple2<K,Object>>
    countApproxDistinctByKey(int p, int sp, Partitioner partitioner)
    Return approximate number of distinct values for each key in this RDD.
    scala.collection.Map<K,Object>
    Count the number of elements for each key, collecting the results to a local Map.
    PartialResult<scala.collection.Map<K,BoundedDouble>>
    countByKeyApprox(long timeout, double confidence)
    Approximate version of countByKey that can return a partial result if it does not finish within a timeout.
    <U> RDD<scala.Tuple2<K,U>>
    flatMapValues(scala.Function1<V,scala.collection.TraversableOnce<U>> f)
    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.
    RDD<scala.Tuple2<K,V>>
    foldByKey(V zeroValue, int numPartitions, scala.Function2<V,V,V> func)
    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.).
    RDD<scala.Tuple2<K,V>>
    foldByKey(V zeroValue, Partitioner partitioner, scala.Function2<V,V,V> func)
    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.).
    RDD<scala.Tuple2<K,V>>
    foldByKey(V zeroValue, scala.Function2<V,V,V> func)
    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.).
    <W> RDD<scala.Tuple2<K,scala.Tuple2<scala.Option<V>,scala.Option<W>>>>
    fullOuterJoin(RDD<scala.Tuple2<K,W>> other)
    Perform a full outer join of this and other.
    <W> RDD<scala.Tuple2<K,scala.Tuple2<scala.Option<V>,scala.Option<W>>>>
    fullOuterJoin(RDD<scala.Tuple2<K,W>> other, int numPartitions)
    Perform a full outer join of this and other.
    <W> RDD<scala.Tuple2<K,scala.Tuple2<scala.Option<V>,scala.Option<W>>>>
    fullOuterJoin(RDD<scala.Tuple2<K,W>> other, Partitioner partitioner)
    Perform a full outer join of this and other.
    RDD<scala.Tuple2<K,scala.collection.Iterable<V>>>
    Group the values for each key in the RDD into a single sequence.
    RDD<scala.Tuple2<K,scala.collection.Iterable<V>>>
    groupByKey(int numPartitions)
    Group the values for each key in the RDD into a single sequence.
    RDD<scala.Tuple2<K,scala.collection.Iterable<V>>>
    groupByKey(Partitioner partitioner)
    Group the values for each key in the RDD into a single sequence.
    <W> RDD<scala.Tuple2<K,scala.Tuple2<scala.collection.Iterable<V>,scala.collection.Iterable<W>>>>
    groupWith(RDD<scala.Tuple2<K,W>> other)
    Alias for cogroup.
    <W1, W2> RDD<scala.Tuple2<K,scala.Tuple3<scala.collection.Iterable<V>,scala.collection.Iterable<W1>,scala.collection.Iterable<W2>>>>
    groupWith(RDD<scala.Tuple2<K,W1>> other1, RDD<scala.Tuple2<K,W2>> other2)
    Alias for cogroup.
    <W1, W2, W3> RDD<scala.Tuple2<K,scala.Tuple4<scala.collection.Iterable<V>,scala.collection.Iterable<W1>,scala.collection.Iterable<W2>,scala.collection.Iterable<W3>>>>
    groupWith(RDD<scala.Tuple2<K,W1>> other1, RDD<scala.Tuple2<K,W2>> other2, RDD<scala.Tuple2<K,W3>> other3)
    Alias for cogroup.
    <W> RDD<scala.Tuple2<K,scala.Tuple2<V,W>>>
    join(RDD<scala.Tuple2<K,W>> other)
    Return an RDD containing all pairs of elements with matching keys in this and other.
    <W> RDD<scala.Tuple2<K,scala.Tuple2<V,W>>>
    join(RDD<scala.Tuple2<K,W>> other, int numPartitions)
    Return an RDD containing all pairs of elements with matching keys in this and other.
    <W> RDD<scala.Tuple2<K,scala.Tuple2<V,W>>>
    join(RDD<scala.Tuple2<K,W>> other, Partitioner partitioner)
    Return an RDD containing all pairs of elements with matching keys in this and other.
    Return an RDD with the keys of each tuple.
    <W> RDD<scala.Tuple2<K,scala.Tuple2<V,scala.Option<W>>>>
    leftOuterJoin(RDD<scala.Tuple2<K,W>> other)
    Perform a left outer join of this and other.
    <W> RDD<scala.Tuple2<K,scala.Tuple2<V,scala.Option<W>>>>
    leftOuterJoin(RDD<scala.Tuple2<K,W>> other, int numPartitions)
    Perform a left outer join of this and other.
    <W> RDD<scala.Tuple2<K,scala.Tuple2<V,scala.Option<W>>>>
    leftOuterJoin(RDD<scala.Tuple2<K,W>> other, Partitioner partitioner)
    Perform a left outer join of this and other.
    scala.collection.Seq<V>
    lookup(K key)
    Return the list of values in the RDD for key key.
    <U> RDD<scala.Tuple2<K,U>>
    mapValues(scala.Function1<V,U> f)
    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.
    RDD<scala.Tuple2<K,V>>
    partitionBy(Partitioner partitioner)
    Return a copy of the RDD partitioned using the specified partitioner.
    RDD<scala.Tuple2<K,V>>
    reduceByKey(Partitioner partitioner, scala.Function2<V,V,V> func)
    Merge the values for each key using an associative and commutative reduce function.
    RDD<scala.Tuple2<K,V>>
    reduceByKey(scala.Function2<V,V,V> func)
    Merge the values for each key using an associative and commutative reduce function.
    RDD<scala.Tuple2<K,V>>
    reduceByKey(scala.Function2<V,V,V> func, int numPartitions)
    Merge the values for each key using an associative and commutative reduce function.
    scala.collection.Map<K,V>
    reduceByKeyLocally(scala.Function2<V,V,V> func)
    Merge the values for each key using an associative and commutative reduce function, but return the results immediately to the master as a Map.
    <W> RDD<scala.Tuple2<K,scala.Tuple2<scala.Option<V>,W>>>
    rightOuterJoin(RDD<scala.Tuple2<K,W>> other)
    Perform a right outer join of this and other.
    <W> RDD<scala.Tuple2<K,scala.Tuple2<scala.Option<V>,W>>>
    rightOuterJoin(RDD<scala.Tuple2<K,W>> other, int numPartitions)
    Perform a right outer join of this and other.
    <W> RDD<scala.Tuple2<K,scala.Tuple2<scala.Option<V>,W>>>
    rightOuterJoin(RDD<scala.Tuple2<K,W>> other, Partitioner partitioner)
    Perform a right outer join of this and other.
    RDD<scala.Tuple2<K,V>>
    sampleByKey(boolean withReplacement, scala.collection.Map<K,Object> fractions, long seed)
    Return a subset of this RDD sampled by key (via stratified sampling).
    RDD<scala.Tuple2<K,V>>
    sampleByKeyExact(boolean withReplacement, scala.collection.Map<K,Object> fractions, long seed)
    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).
    void
    saveAsHadoopDataset(org.apache.hadoop.mapred.JobConf conf)
    Output the RDD to any Hadoop-supported storage system, using a Hadoop JobConf object for that storage system.
    void
    saveAsHadoopFile(String path, Class<?> keyClass, Class<?> valueClass, Class<? extends org.apache.hadoop.mapred.OutputFormat<?,?>> outputFormatClass, Class<? extends org.apache.hadoop.io.compress.CompressionCodec> codec)
    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.
    void
    saveAsHadoopFile(String path, Class<?> keyClass, Class<?> valueClass, Class<? extends org.apache.hadoop.mapred.OutputFormat<?,?>> outputFormatClass, org.apache.hadoop.mapred.JobConf conf, scala.Option<Class<? extends org.apache.hadoop.io.compress.CompressionCodec>> codec)
    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.
    <F extends org.apache.hadoop.mapred.OutputFormat<K, V>>
    void
    saveAsHadoopFile(String path, Class<? extends org.apache.hadoop.io.compress.CompressionCodec> codec, scala.reflect.ClassTag<F> fm)
    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.
    <F extends org.apache.hadoop.mapred.OutputFormat<K, V>>
    void
    saveAsHadoopFile(String path, scala.reflect.ClassTag<F> fm)
    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.
    void
    saveAsNewAPIHadoopDataset(org.apache.hadoop.conf.Configuration conf)
    Output the RDD to any Hadoop-supported storage system with new Hadoop API, using a Hadoop Configuration object for that storage system.
    void
    saveAsNewAPIHadoopFile(String path, Class<?> keyClass, Class<?> valueClass, Class<? extends org.apache.hadoop.mapreduce.OutputFormat<?,?>> outputFormatClass, org.apache.hadoop.conf.Configuration conf)
    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.
    <F extends org.apache.hadoop.mapreduce.OutputFormat<K, V>>
    void
    saveAsNewAPIHadoopFile(String path, scala.reflect.ClassTag<F> fm)
    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.
    <W> RDD<scala.Tuple2<K,V>>
    subtractByKey(RDD<scala.Tuple2<K,W>> other, int numPartitions, scala.reflect.ClassTag<W> evidence$5)
    Return an RDD with the pairs from this whose keys are not in other.
    <W> RDD<scala.Tuple2<K,V>>
    subtractByKey(RDD<scala.Tuple2<K,W>> other, Partitioner p, scala.reflect.ClassTag<W> evidence$6)
    Return an RDD with the pairs from this whose keys are not in other.
    <W> RDD<scala.Tuple2<K,V>>
    subtractByKey(RDD<scala.Tuple2<K,W>> other, scala.reflect.ClassTag<W> evidence$4)
    Return an RDD with the pairs from this whose keys are not in other.
    Return an RDD with the values of each tuple.

    Methods inherited from class java.lang.Object

    equals, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait

    Methods inherited from interface org.apache.spark.internal.Logging

    initializeForcefully, initializeLogIfNecessary, initializeLogIfNecessary, initializeLogIfNecessary$default$2, isTraceEnabled, log, logDebug, logDebug, logError, logError, logInfo, logInfo, logName, logTrace, logTrace, logWarning, logWarning, org$apache$spark$internal$Logging$$log_, org$apache$spark$internal$Logging$$log__$eq
  • Constructor Details

    • PairRDDFunctions

      public PairRDDFunctions(RDD<scala.Tuple2<K,V>> self, scala.reflect.ClassTag<K> kt, scala.reflect.ClassTag<V> vt, scala.math.Ordering<K> ord)
  • Method Details

    • combineByKeyWithClassTag

      public <C> RDD<scala.Tuple2<K,C>> combineByKeyWithClassTag(scala.Function1<V,C> createCombiner, scala.Function2<C,V,C> mergeValue, scala.Function2<C,C,C> mergeCombiners, Partitioner partitioner, boolean mapSideCombine, Serializer serializer, scala.reflect.ClassTag<C> ct)
      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

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

      Parameters:
      createCombiner - (undocumented)
      mergeValue - (undocumented)
      mergeCombiners - (undocumented)
      partitioner - (undocumented)
      mapSideCombine - (undocumented)
      serializer - (undocumented)
      ct - (undocumented)
      Returns:
      (undocumented)
      Note:
      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]).
    • combineByKey

      public <C> RDD<scala.Tuple2<K,C>> combineByKey(scala.Function1<V,C> createCombiner, scala.Function2<C,V,C> mergeValue, scala.Function2<C,C,C> mergeCombiners, Partitioner partitioner, boolean mapSideCombine, Serializer serializer)
      Generic function to combine the elements for each key using a custom set of aggregation functions. This method is here for backward compatibility. It does not provide combiner classtag information to the shuffle.

      Parameters:
      createCombiner - (undocumented)
      mergeValue - (undocumented)
      mergeCombiners - (undocumented)
      partitioner - (undocumented)
      mapSideCombine - (undocumented)
      serializer - (undocumented)
      Returns:
      (undocumented)
      See Also:
      • combineByKeyWithClassTag
    • combineByKey

      public <C> RDD<scala.Tuple2<K,C>> combineByKey(scala.Function1<V,C> createCombiner, scala.Function2<C,V,C> mergeValue, scala.Function2<C,C,C> mergeCombiners, int numPartitions)
      Simplified version of combineByKeyWithClassTag that hash-partitions the output RDD. This method is here for backward compatibility. It does not provide combiner classtag information to the shuffle.

      Parameters:
      createCombiner - (undocumented)
      mergeValue - (undocumented)
      mergeCombiners - (undocumented)
      numPartitions - (undocumented)
      Returns:
      (undocumented)
      See Also:
      • combineByKeyWithClassTag
    • combineByKeyWithClassTag

      public <C> RDD<scala.Tuple2<K,C>> combineByKeyWithClassTag(scala.Function1<V,C> createCombiner, scala.Function2<C,V,C> mergeValue, scala.Function2<C,C,C> mergeCombiners, int numPartitions, scala.reflect.ClassTag<C> ct)
      Simplified version of combineByKeyWithClassTag that hash-partitions the output RDD.
      Parameters:
      createCombiner - (undocumented)
      mergeValue - (undocumented)
      mergeCombiners - (undocumented)
      numPartitions - (undocumented)
      ct - (undocumented)
      Returns:
      (undocumented)
    • aggregateByKey

      public <U> RDD<scala.Tuple2<K,U>> aggregateByKey(U zeroValue, Partitioner partitioner, scala.Function2<U,V,U> seqOp, scala.Function2<U,U,U> combOp, scala.reflect.ClassTag<U> evidence$1)
      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.
      Parameters:
      zeroValue - (undocumented)
      partitioner - (undocumented)
      seqOp - (undocumented)
      combOp - (undocumented)
      evidence$1 - (undocumented)
      Returns:
      (undocumented)
    • aggregateByKey

      public <U> RDD<scala.Tuple2<K,U>> aggregateByKey(U zeroValue, int numPartitions, scala.Function2<U,V,U> seqOp, scala.Function2<U,U,U> combOp, scala.reflect.ClassTag<U> evidence$2)
      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.
      Parameters:
      zeroValue - (undocumented)
      numPartitions - (undocumented)
      seqOp - (undocumented)
      combOp - (undocumented)
      evidence$2 - (undocumented)
      Returns:
      (undocumented)
    • aggregateByKey

      public <U> RDD<scala.Tuple2<K,U>> aggregateByKey(U zeroValue, scala.Function2<U,V,U> seqOp, scala.Function2<U,U,U> combOp, scala.reflect.ClassTag<U> evidence$3)
      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.
      Parameters:
      zeroValue - (undocumented)
      seqOp - (undocumented)
      combOp - (undocumented)
      evidence$3 - (undocumented)
      Returns:
      (undocumented)
    • foldByKey

      public RDD<scala.Tuple2<K,V>> foldByKey(V zeroValue, Partitioner partitioner, scala.Function2<V,V,V> func)
      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.).
      Parameters:
      zeroValue - (undocumented)
      partitioner - (undocumented)
      func - (undocumented)
      Returns:
      (undocumented)
    • foldByKey

      public RDD<scala.Tuple2<K,V>> foldByKey(V zeroValue, int numPartitions, scala.Function2<V,V,V> func)
      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.).
      Parameters:
      zeroValue - (undocumented)
      numPartitions - (undocumented)
      func - (undocumented)
      Returns:
      (undocumented)
    • foldByKey

      public RDD<scala.Tuple2<K,V>> foldByKey(V zeroValue, scala.Function2<V,V,V> func)
      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.).
      Parameters:
      zeroValue - (undocumented)
      func - (undocumented)
      Returns:
      (undocumented)
    • sampleByKey

      public RDD<scala.Tuple2<K,V>> sampleByKey(boolean withReplacement, scala.collection.Map<K,Object> fractions, long seed)
      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.

      Parameters:
      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
    • sampleByKeyExact

      public RDD<scala.Tuple2<K,V>> sampleByKeyExact(boolean withReplacement, scala.collection.Map<K,Object> fractions, long seed)
      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(boolean,scala.collection.Map<K,java.lang.Object>,long) 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.

      Parameters:
      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
    • reduceByKey

      public RDD<scala.Tuple2<K,V>> reduceByKey(Partitioner partitioner, scala.Function2<V,V,V> func)
      Merge the values for each key using an associative and commutative reduce function. This will also perform the merging locally on each mapper before sending results to a reducer, similarly to a "combiner" in MapReduce.
      Parameters:
      partitioner - (undocumented)
      func - (undocumented)
      Returns:
      (undocumented)
    • reduceByKey

      public RDD<scala.Tuple2<K,V>> reduceByKey(scala.Function2<V,V,V> func, int numPartitions)
      Merge the values for each key using an associative and commutative 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.
      Parameters:
      func - (undocumented)
      numPartitions - (undocumented)
      Returns:
      (undocumented)
    • reduceByKey

      public RDD<scala.Tuple2<K,V>> reduceByKey(scala.Function2<V,V,V> func)
      Merge the values for each key using an associative and commutative 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.
      Parameters:
      func - (undocumented)
      Returns:
      (undocumented)
    • reduceByKeyLocally

      public scala.collection.Map<K,V> reduceByKeyLocally(scala.Function2<V,V,V> func)
      Merge the values for each key using an associative and commutative 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.
      Parameters:
      func - (undocumented)
      Returns:
      (undocumented)
    • countByKey

      public scala.collection.Map<K,Object> countByKey()
      Count the number of elements for each key, collecting the results to a local Map.

      Returns:
      (undocumented)
      Note:
      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.
    • countByKeyApprox

      public PartialResult<scala.collection.Map<K,BoundedDouble>> countByKeyApprox(long timeout, double confidence)
      Approximate version of countByKey that can return a partial result if it does not finish within a timeout.

      The confidence is the probability that the error bounds of the result will contain the true value. That is, if countApprox were called repeatedly with confidence 0.9, we would expect 90% of the results to contain the true count. The confidence must be in the range [0,1] or an exception will be thrown.

      Parameters:
      timeout - maximum time to wait for the job, in milliseconds
      confidence - the desired statistical confidence in the result
      Returns:
      a potentially incomplete result, with error bounds
    • countApproxDistinctByKey

      public RDD<scala.Tuple2<K,Object>> countApproxDistinctByKey(int p, int sp, Partitioner partitioner)
      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 is greater than p) would trigger sparse representation of registers, which may reduce the memory consumption and increase accuracy when the cardinality is small.

      Parameters:
      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.
      Returns:
      (undocumented)
    • countApproxDistinctByKey

      public RDD<scala.Tuple2<K,Object>> countApproxDistinctByKey(double relativeSD, Partitioner partitioner)
      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.

      Parameters:
      relativeSD - Relative accuracy. Smaller values create counters that require more space. It must be greater than 0.000017.
      partitioner - partitioner of the resulting RDD
      Returns:
      (undocumented)
    • countApproxDistinctByKey

      public RDD<scala.Tuple2<K,Object>> countApproxDistinctByKey(double relativeSD, int numPartitions)
      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.

      Parameters:
      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
      Returns:
      (undocumented)
    • countApproxDistinctByKey

      public RDD<scala.Tuple2<K,Object>> countApproxDistinctByKey(double relativeSD)
      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.

      Parameters:
      relativeSD - Relative accuracy. Smaller values create counters that require more space. It must be greater than 0.000017.
      Returns:
      (undocumented)
    • groupByKey

      public RDD<scala.Tuple2<K,scala.collection.Iterable<V>>> groupByKey(Partitioner partitioner)
      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.

      Parameters:
      partitioner - (undocumented)
      Returns:
      (undocumented)
      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.

      , As currently implemented, groupByKey must be able to hold all the key-value pairs for any key in memory. If a key has too many values, it can result in an OutOfMemoryError.

    • groupByKey

      public RDD<scala.Tuple2<K,scala.collection.Iterable<V>>> groupByKey(int numPartitions)
      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.

      Parameters:
      numPartitions - (undocumented)
      Returns:
      (undocumented)
      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.

      , As currently implemented, groupByKey must be able to hold all the key-value pairs for any key in memory. If a key has too many values, it can result in an OutOfMemoryError.

    • partitionBy

      public RDD<scala.Tuple2<K,V>> partitionBy(Partitioner partitioner)
      Return a copy of the RDD partitioned using the specified partitioner.
      Parameters:
      partitioner - (undocumented)
      Returns:
      (undocumented)
    • join

      public <W> RDD<scala.Tuple2<K,scala.Tuple2<V,W>>> join(RDD<scala.Tuple2<K,W>> other, Partitioner partitioner)
      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.
      Parameters:
      other - (undocumented)
      partitioner - (undocumented)
      Returns:
      (undocumented)
    • leftOuterJoin

      public <W> RDD<scala.Tuple2<K,scala.Tuple2<V,scala.Option<W>>>> leftOuterJoin(RDD<scala.Tuple2<K,W>> other, Partitioner partitioner)
      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.
      Parameters:
      other - (undocumented)
      partitioner - (undocumented)
      Returns:
      (undocumented)
    • rightOuterJoin

      public <W> RDD<scala.Tuple2<K,scala.Tuple2<scala.Option<V>,W>>> rightOuterJoin(RDD<scala.Tuple2<K,W>> other, Partitioner partitioner)
      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.
      Parameters:
      other - (undocumented)
      partitioner - (undocumented)
      Returns:
      (undocumented)
    • fullOuterJoin

      public <W> RDD<scala.Tuple2<K,scala.Tuple2<scala.Option<V>,scala.Option<W>>>> fullOuterJoin(RDD<scala.Tuple2<K,W>> other, Partitioner partitioner)
      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.
      Parameters:
      other - (undocumented)
      partitioner - (undocumented)
      Returns:
      (undocumented)
    • combineByKey

      public <C> RDD<scala.Tuple2<K,C>> combineByKey(scala.Function1<V,C> createCombiner, scala.Function2<C,V,C> mergeValue, scala.Function2<C,C,C> mergeCombiners)
      Simplified version of combineByKeyWithClassTag that hash-partitions the resulting RDD using the existing partitioner/parallelism level. This method is here for backward compatibility. It does not provide combiner classtag information to the shuffle.

      Parameters:
      createCombiner - (undocumented)
      mergeValue - (undocumented)
      mergeCombiners - (undocumented)
      Returns:
      (undocumented)
      See Also:
      • combineByKeyWithClassTag
    • combineByKeyWithClassTag

      public <C> RDD<scala.Tuple2<K,C>> combineByKeyWithClassTag(scala.Function1<V,C> createCombiner, scala.Function2<C,V,C> mergeValue, scala.Function2<C,C,C> mergeCombiners, scala.reflect.ClassTag<C> ct)
      Simplified version of combineByKeyWithClassTag that hash-partitions the resulting RDD using the existing partitioner/parallelism level.
      Parameters:
      createCombiner - (undocumented)
      mergeValue - (undocumented)
      mergeCombiners - (undocumented)
      ct - (undocumented)
      Returns:
      (undocumented)
    • groupByKey

      public RDD<scala.Tuple2<K,scala.collection.Iterable<V>>> groupByKey()
      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.

      Returns:
      (undocumented)
      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.
    • join

      public <W> RDD<scala.Tuple2<K,scala.Tuple2<V,W>>> join(RDD<scala.Tuple2<K,W>> 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.
      Parameters:
      other - (undocumented)
      Returns:
      (undocumented)
    • join

      public <W> RDD<scala.Tuple2<K,scala.Tuple2<V,W>>> join(RDD<scala.Tuple2<K,W>> other, int numPartitions)
      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.
      Parameters:
      other - (undocumented)
      numPartitions - (undocumented)
      Returns:
      (undocumented)
    • leftOuterJoin

      public <W> RDD<scala.Tuple2<K,scala.Tuple2<V,scala.Option<W>>>> leftOuterJoin(RDD<scala.Tuple2<K,W>> 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.
      Parameters:
      other - (undocumented)
      Returns:
      (undocumented)
    • leftOuterJoin

      public <W> RDD<scala.Tuple2<K,scala.Tuple2<V,scala.Option<W>>>> leftOuterJoin(RDD<scala.Tuple2<K,W>> other, int numPartitions)
      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.
      Parameters:
      other - (undocumented)
      numPartitions - (undocumented)
      Returns:
      (undocumented)
    • rightOuterJoin

      public <W> RDD<scala.Tuple2<K,scala.Tuple2<scala.Option<V>,W>>> rightOuterJoin(RDD<scala.Tuple2<K,W>> 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.
      Parameters:
      other - (undocumented)
      Returns:
      (undocumented)
    • rightOuterJoin

      public <W> RDD<scala.Tuple2<K,scala.Tuple2<scala.Option<V>,W>>> rightOuterJoin(RDD<scala.Tuple2<K,W>> other, int numPartitions)
      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.
      Parameters:
      other - (undocumented)
      numPartitions - (undocumented)
      Returns:
      (undocumented)
    • fullOuterJoin

      public <W> RDD<scala.Tuple2<K,scala.Tuple2<scala.Option<V>,scala.Option<W>>>> fullOuterJoin(RDD<scala.Tuple2<K,W>> 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.
      Parameters:
      other - (undocumented)
      Returns:
      (undocumented)
    • fullOuterJoin

      public <W> RDD<scala.Tuple2<K,scala.Tuple2<scala.Option<V>,scala.Option<W>>>> fullOuterJoin(RDD<scala.Tuple2<K,W>> other, int numPartitions)
      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.
      Parameters:
      other - (undocumented)
      numPartitions - (undocumented)
      Returns:
      (undocumented)
    • collectAsMap

      public scala.collection.Map<K,V> collectAsMap()
      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)

      Returns:
      (undocumented)
      Note:
      this method should only be used if the resulting data is expected to be small, as all the data is loaded into the driver's memory.
    • mapValues

      public <U> RDD<scala.Tuple2<K,U>> mapValues(scala.Function1<V,U> f)
      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.
      Parameters:
      f - (undocumented)
      Returns:
      (undocumented)
    • flatMapValues

      public <U> RDD<scala.Tuple2<K,U>> flatMapValues(scala.Function1<V,scala.collection.TraversableOnce<U>> f)
      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.
      Parameters:
      f - (undocumented)
      Returns:
      (undocumented)
    • cogroup

      public <W1, W2, W3> RDD<scala.Tuple2<K,scala.Tuple4<scala.collection.Iterable<V>,scala.collection.Iterable<W1>,scala.collection.Iterable<W2>,scala.collection.Iterable<W3>>>> cogroup(RDD<scala.Tuple2<K,W1>> other1, RDD<scala.Tuple2<K,W2>> other2, RDD<scala.Tuple2<K,W3>> other3, Partitioner partitioner)
      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.
      Parameters:
      other1 - (undocumented)
      other2 - (undocumented)
      other3 - (undocumented)
      partitioner - (undocumented)
      Returns:
      (undocumented)
    • cogroup

      public <W> RDD<scala.Tuple2<K,scala.Tuple2<scala.collection.Iterable<V>,scala.collection.Iterable<W>>>> cogroup(RDD<scala.Tuple2<K,W>> other, Partitioner partitioner)
      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.
      Parameters:
      other - (undocumented)
      partitioner - (undocumented)
      Returns:
      (undocumented)
    • cogroup

      public <W1, W2> RDD<scala.Tuple2<K,scala.Tuple3<scala.collection.Iterable<V>,scala.collection.Iterable<W1>,scala.collection.Iterable<W2>>>> cogroup(RDD<scala.Tuple2<K,W1>> other1, RDD<scala.Tuple2<K,W2>> other2, Partitioner partitioner)
      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.
      Parameters:
      other1 - (undocumented)
      other2 - (undocumented)
      partitioner - (undocumented)
      Returns:
      (undocumented)
    • cogroup

      public <W1, W2, W3> RDD<scala.Tuple2<K,scala.Tuple4<scala.collection.Iterable<V>,scala.collection.Iterable<W1>,scala.collection.Iterable<W2>,scala.collection.Iterable<W3>>>> cogroup(RDD<scala.Tuple2<K,W1>> other1, RDD<scala.Tuple2<K,W2>> other2, RDD<scala.Tuple2<K,W3>> other3)
      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.
      Parameters:
      other1 - (undocumented)
      other2 - (undocumented)
      other3 - (undocumented)
      Returns:
      (undocumented)
    • cogroup

      public <W> RDD<scala.Tuple2<K,scala.Tuple2<scala.collection.Iterable<V>,scala.collection.Iterable<W>>>> cogroup(RDD<scala.Tuple2<K,W>> other)
      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.
      Parameters:
      other - (undocumented)
      Returns:
      (undocumented)
    • cogroup

      public <W1, W2> RDD<scala.Tuple2<K,scala.Tuple3<scala.collection.Iterable<V>,scala.collection.Iterable<W1>,scala.collection.Iterable<W2>>>> cogroup(RDD<scala.Tuple2<K,W1>> other1, RDD<scala.Tuple2<K,W2>> other2)
      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.
      Parameters:
      other1 - (undocumented)
      other2 - (undocumented)
      Returns:
      (undocumented)
    • cogroup

      public <W> RDD<scala.Tuple2<K,scala.Tuple2<scala.collection.Iterable<V>,scala.collection.Iterable<W>>>> cogroup(RDD<scala.Tuple2<K,W>> other, int numPartitions)
      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.
      Parameters:
      other - (undocumented)
      numPartitions - (undocumented)
      Returns:
      (undocumented)
    • cogroup

      public <W1, W2> RDD<scala.Tuple2<K,scala.Tuple3<scala.collection.Iterable<V>,scala.collection.Iterable<W1>,scala.collection.Iterable<W2>>>> cogroup(RDD<scala.Tuple2<K,W1>> other1, RDD<scala.Tuple2<K,W2>> other2, int numPartitions)
      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.
      Parameters:
      other1 - (undocumented)
      other2 - (undocumented)
      numPartitions - (undocumented)
      Returns:
      (undocumented)
    • cogroup

      public <W1, W2, W3> RDD<scala.Tuple2<K,scala.Tuple4<scala.collection.Iterable<V>,scala.collection.Iterable<W1>,scala.collection.Iterable<W2>,scala.collection.Iterable<W3>>>> cogroup(RDD<scala.Tuple2<K,W1>> other1, RDD<scala.Tuple2<K,W2>> other2, RDD<scala.Tuple2<K,W3>> other3, int numPartitions)
      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.
      Parameters:
      other1 - (undocumented)
      other2 - (undocumented)
      other3 - (undocumented)
      numPartitions - (undocumented)
      Returns:
      (undocumented)
    • groupWith

      public <W> RDD<scala.Tuple2<K,scala.Tuple2<scala.collection.Iterable<V>,scala.collection.Iterable<W>>>> groupWith(RDD<scala.Tuple2<K,W>> other)
      Alias for cogroup.
    • groupWith

      public <W1, W2> RDD<scala.Tuple2<K,scala.Tuple3<scala.collection.Iterable<V>,scala.collection.Iterable<W1>,scala.collection.Iterable<W2>>>> groupWith(RDD<scala.Tuple2<K,W1>> other1, RDD<scala.Tuple2<K,W2>> other2)
      Alias for cogroup.
    • groupWith

      public <W1, W2, W3> RDD<scala.Tuple2<K,scala.Tuple4<scala.collection.Iterable<V>,scala.collection.Iterable<W1>,scala.collection.Iterable<W2>,scala.collection.Iterable<W3>>>> groupWith(RDD<scala.Tuple2<K,W1>> other1, RDD<scala.Tuple2<K,W2>> other2, RDD<scala.Tuple2<K,W3>> other3)
      Alias for cogroup.
    • subtractByKey

      public <W> RDD<scala.Tuple2<K,V>> subtractByKey(RDD<scala.Tuple2<K,W>> other, scala.reflect.ClassTag<W> evidence$4)
      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 less than or equal to us.

      Parameters:
      other - (undocumented)
      evidence$4 - (undocumented)
      Returns:
      (undocumented)
    • subtractByKey

      public <W> RDD<scala.Tuple2<K,V>> subtractByKey(RDD<scala.Tuple2<K,W>> other, int numPartitions, scala.reflect.ClassTag<W> evidence$5)
      Return an RDD with the pairs from this whose keys are not in other.
      Parameters:
      other - (undocumented)
      numPartitions - (undocumented)
      evidence$5 - (undocumented)
      Returns:
      (undocumented)
    • subtractByKey

      public <W> RDD<scala.Tuple2<K,V>> subtractByKey(RDD<scala.Tuple2<K,W>> other, Partitioner p, scala.reflect.ClassTag<W> evidence$6)
      Return an RDD with the pairs from this whose keys are not in other.
      Parameters:
      other - (undocumented)
      p - (undocumented)
      evidence$6 - (undocumented)
      Returns:
      (undocumented)
    • lookup

      public scala.collection.Seq<V> lookup(K 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.
      Parameters:
      key - (undocumented)
      Returns:
      (undocumented)
    • saveAsHadoopFile

      public <F extends org.apache.hadoop.mapred.OutputFormat<K, V>> void saveAsHadoopFile(String path, scala.reflect.ClassTag<F> fm)
      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.
      Parameters:
      path - (undocumented)
      fm - (undocumented)
    • saveAsHadoopFile

      public <F extends org.apache.hadoop.mapred.OutputFormat<K, V>> void saveAsHadoopFile(String path, Class<? extends org.apache.hadoop.io.compress.CompressionCodec> codec, scala.reflect.ClassTag<F> fm)
      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.
      Parameters:
      path - (undocumented)
      codec - (undocumented)
      fm - (undocumented)
    • saveAsNewAPIHadoopFile

      public <F extends org.apache.hadoop.mapreduce.OutputFormat<K, V>> void saveAsNewAPIHadoopFile(String path, scala.reflect.ClassTag<F> fm)
      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.
      Parameters:
      path - (undocumented)
      fm - (undocumented)
    • saveAsNewAPIHadoopFile

      public void saveAsNewAPIHadoopFile(String path, Class<?> keyClass, Class<?> valueClass, Class<? extends org.apache.hadoop.mapreduce.OutputFormat<?,?>> outputFormatClass, org.apache.hadoop.conf.Configuration conf)
      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.
      Parameters:
      path - (undocumented)
      keyClass - (undocumented)
      valueClass - (undocumented)
      outputFormatClass - (undocumented)
      conf - (undocumented)
    • saveAsHadoopFile

      public void saveAsHadoopFile(String path, Class<?> keyClass, Class<?> valueClass, Class<? extends org.apache.hadoop.mapred.OutputFormat<?,?>> outputFormatClass, Class<? extends org.apache.hadoop.io.compress.CompressionCodec> codec)
      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.
      Parameters:
      path - (undocumented)
      keyClass - (undocumented)
      valueClass - (undocumented)
      outputFormatClass - (undocumented)
      codec - (undocumented)
    • saveAsHadoopFile

      public void saveAsHadoopFile(String path, Class<?> keyClass, Class<?> valueClass, Class<? extends org.apache.hadoop.mapred.OutputFormat<?,?>> outputFormatClass, org.apache.hadoop.mapred.JobConf conf, scala.Option<Class<? extends org.apache.hadoop.io.compress.CompressionCodec>> codec)
      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.

      Parameters:
      path - (undocumented)
      keyClass - (undocumented)
      valueClass - (undocumented)
      outputFormatClass - (undocumented)
      conf - (undocumented)
      codec - (undocumented)
      Note:
      We should make sure our tasks are idempotent when speculation is enabled, i.e. do not use output committer that writes data directly. There is an example in https://issues.apache.org/jira/browse/SPARK-10063 to show the bad result of using direct output committer with speculation enabled.
    • saveAsNewAPIHadoopDataset

      public void saveAsNewAPIHadoopDataset(org.apache.hadoop.conf.Configuration conf)
      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.

      Parameters:
      conf - (undocumented)
      Note:
      We should make sure our tasks are idempotent when speculation is enabled, i.e. do not use output committer that writes data directly. There is an example in https://issues.apache.org/jira/browse/SPARK-10063 to show the bad result of using direct output committer with speculation enabled.
    • saveAsHadoopDataset

      public void saveAsHadoopDataset(org.apache.hadoop.mapred.JobConf conf)
      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.
      Parameters:
      conf - (undocumented)
    • keys

      public RDD<K> keys()
      Return an RDD with the keys of each tuple.
      Returns:
      (undocumented)
    • values

      public RDD<V> values()
      Return an RDD with the values of each tuple.
      Returns:
      (undocumented)