Class PairRDDFunctions<K,V>
- All Implemented Interfaces:
Serializable
,org.apache.spark.internal.Logging
- See Also:
-
Nested Class Summary
Nested classes/interfaces inherited from interface org.apache.spark.internal.Logging
org.apache.spark.internal.Logging.LogStringContext, org.apache.spark.internal.Logging.SparkShellLoggingFilter
-
Constructor Summary
-
Method Summary
Modifier and TypeMethodDescriptionaggregateByKey
(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".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".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".For each key k inthis
orother
, return a resulting RDD that contains a tuple with the list of values for that key inthis
as well asother
.For each key k inthis
orother
, return a resulting RDD that contains a tuple with the list of values for that key inthis
as well asother
.cogroup
(RDD<scala.Tuple2<K, W>> other, Partitioner partitioner) For each key k inthis
orother
, return a resulting RDD that contains a tuple with the list of values for that key inthis
as well asother
.<W1,
W2> RDD<scala.Tuple2<K, scala.Tuple3<scala.collection.Iterable<V>, scala.collection.Iterable<W1>, scala.collection.Iterable<W2>>>> For each key k inthis
orother1
orother2
, return a resulting RDD that contains a tuple with the list of values for that key inthis
,other1
andother2
.<W1,
W2> RDD<scala.Tuple2<K, scala.Tuple3<scala.collection.Iterable<V>, scala.collection.Iterable<W1>, scala.collection.Iterable<W2>>>> For each key k inthis
orother1
orother2
, return a resulting RDD that contains a tuple with the list of values for that key inthis
,other1
andother2
.<W1,
W2> RDD<scala.Tuple2<K, scala.Tuple3<scala.collection.Iterable<V>, scala.collection.Iterable<W1>, scala.collection.Iterable<W2>>>> For each key k inthis
orother1
orother2
, return a resulting RDD that contains a tuple with the list of values for that key inthis
,other1
andother2
.<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 inthis
orother1
orother2
orother3
, return a resulting RDD that contains a tuple with the list of values for that key inthis
,other1
,other2
andother3
.<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 inthis
orother1
orother2
orother3
, return a resulting RDD that contains a tuple with the list of values for that key inthis
,other1
,other2
andother3
.<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 inthis
orother1
orother2
orother3
, return a resulting RDD that contains a tuple with the list of values for that key inthis
,other1
,other2
andother3
.Return the key-value pairs in this RDD to the master as a Map.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.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.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.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.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.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.countApproxDistinctByKey
(double relativeSD) Return approximate number of distinct values for each key in this RDD.countApproxDistinctByKey
(double relativeSD, int numPartitions) Return approximate number of distinct values for each key in this RDD.countApproxDistinctByKey
(double relativeSD, Partitioner partitioner) Return approximate number of distinct values for each key in this RDD.countApproxDistinctByKey
(int p, int sp, Partitioner partitioner) Return approximate number of distinct values for each key in this RDD.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.flatMapValues
(scala.Function1<V, scala.collection.IterableOnce<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.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.).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.).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.).fullOuterJoin
(RDD<scala.Tuple2<K, W>> other) Perform a full outer join ofthis
andother
.fullOuterJoin
(RDD<scala.Tuple2<K, W>> other, int numPartitions) Perform a full outer join ofthis
andother
.fullOuterJoin
(RDD<scala.Tuple2<K, W>> other, Partitioner partitioner) Perform a full outer join ofthis
andother
.Group the values for each key in the RDD into a single sequence.groupByKey
(int numPartitions) Group the values for each key in the RDD into a single sequence.groupByKey
(Partitioner partitioner) Group the values for each key in the RDD into a single sequence.Alias for cogroup.<W1,
W2> RDD<scala.Tuple2<K, scala.Tuple3<scala.collection.Iterable<V>, scala.collection.Iterable<W1>, scala.collection.Iterable<W2>>>> 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.Return an RDD containing all pairs of elements with matching keys inthis
andother
.Return an RDD containing all pairs of elements with matching keys inthis
andother
.join
(RDD<scala.Tuple2<K, W>> other, Partitioner partitioner) Return an RDD containing all pairs of elements with matching keys inthis
andother
.keys()
Return an RDD with the keys of each tuple.leftOuterJoin
(RDD<scala.Tuple2<K, W>> other) Perform a left outer join ofthis
andother
.leftOuterJoin
(RDD<scala.Tuple2<K, W>> other, int numPartitions) Perform a left outer join ofthis
andother
.leftOuterJoin
(RDD<scala.Tuple2<K, W>> other, Partitioner partitioner) Perform a left outer join ofthis
andother
.scala.collection.immutable.Seq<V>
Return the list of values in the RDD for keykey
.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.partitionBy
(Partitioner partitioner) Return a copy of the RDD partitioned using the specified partitioner.reduceByKey
(Partitioner partitioner, scala.Function2<V, V, V> func) Merge the values for each key using an associative and commutative reduce function.reduceByKey
(scala.Function2<V, V, V> func) Merge the values for each key using an associative and commutative reduce function.reduceByKey
(scala.Function2<V, V, V> func, int numPartitions) Merge the values for each key using an associative and commutative reduce function.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.rightOuterJoin
(RDD<scala.Tuple2<K, W>> other) Perform a right outer join ofthis
andother
.rightOuterJoin
(RDD<scala.Tuple2<K, W>> other, int numPartitions) Perform a right outer join ofthis
andother
.rightOuterJoin
(RDD<scala.Tuple2<K, W>> other, Partitioner partitioner) Perform a right outer join ofthis
andother
.sampleByKey
(boolean withReplacement, scala.collection.Map<K, Object> fractions, long seed) Return a subset of this RDD sampled by key (via stratified sampling).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 HadoopOutputFormat
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 HadoopOutputFormat
class supporting the key and value types K and V in this RDD.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 HadoopOutputFormat
class supporting the key and value types K and V in this RDD.saveAsHadoopFile
(String path, scala.reflect.ClassTag<F> fm) Output the RDD to any Hadoop-supported file system, using a HadoopOutputFormat
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 APIOutputFormat
(mapreduce.OutputFormat) object supporting the key and value types K and V in this RDD.saveAsNewAPIHadoopFile
(String path, scala.reflect.ClassTag<F> fm) Output the RDD to any Hadoop-supported file system, using a new Hadoop APIOutputFormat
(mapreduce.OutputFormat) object supporting the key and value types K and V in this RDD.subtractByKey
(RDD<scala.Tuple2<K, W>> other, int numPartitions, scala.reflect.ClassTag<W> evidence$5) Return an RDD with the pairs fromthis
whose keys are not inother
.subtractByKey
(RDD<scala.Tuple2<K, W>> other, Partitioner p, scala.reflect.ClassTag<W> evidence$6) Return an RDD with the pairs fromthis
whose keys are not inother
.subtractByKey
(RDD<scala.Tuple2<K, W>> other, scala.reflect.ClassTag<W> evidence$4) Return an RDD with the pairs fromthis
whose keys are not inother
.values()
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, logDebug, logDebug, logError, logError, logError, logError, logInfo, logInfo, logInfo, logInfo, logName, LogStringContext, logTrace, logTrace, logTrace, logTrace, logWarning, logWarning, logWarning, logWarning, org$apache$spark$internal$Logging$$log_, org$apache$spark$internal$Logging$$log__$eq, withLogContext
-
Constructor Details
-
PairRDDFunctions
-
-
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" CUsers 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.IterableOnce. 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.IterableOnce. 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.IterableOnce. 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
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 replacementfractions
- map of specific keys to sampling ratesseed
- 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 replacementfractions
- map of specific keys to sampling ratesseed
- seed for the random number generator- Returns:
- RDD containing the sampled subset
-
reduceByKey
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
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
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
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
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 millisecondsconfidence
- the desired statistical confidence in the result- Returns:
- a potentially incomplete result, with error bounds
-
countApproxDistinctByKey
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 thanp
) 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 andsp
ifsp
is not zero (32 max).sp
- The precision value for the sparse set, between 0 and 32. Ifsp
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
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
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
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
orPairRDDFunctions.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
Group the values for each key in the RDD into a single sequence. Hash-partitions the resulting RDD with intonumPartitions
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
orPairRDDFunctions.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
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, joinW>>> (RDD<scala.Tuple2<K, W>> other, Partitioner partitioner) Return an RDD containing all pairs of elements with matching keys inthis
andother
. Each pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is inthis
and (k, v2) is inother
. 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, leftOuterJoinscala.Option<W>>>> (RDD<scala.Tuple2<K, W>> other, Partitioner partitioner) Perform a left outer join ofthis
andother
. For each element (k, v) inthis
, the resulting RDD will either contain all pairs (k, (v, Some(w))) for w inother
, or the pair (k, (v, None)) if no elements inother
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>, rightOuterJoinW>>> (RDD<scala.Tuple2<K, W>> other, Partitioner partitioner) Perform a right outer join ofthis
andother
. For each element (k, w) inother
, the resulting RDD will either contain all pairs (k, (Some(v), w)) for v inthis
, or the pair (k, (None, w)) if no elements inthis
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>, fullOuterJoinscala.Option<W>>>> (RDD<scala.Tuple2<K, W>> other, Partitioner partitioner) Perform a full outer join ofthis
andother
. For each element (k, v) inthis
, the resulting RDD will either contain all pairs (k, (Some(v), Some(w))) for w inother
, or the pair (k, (Some(v), None)) if no elements inother
have key k. Similarly, for each element (k, w) inother
, the resulting RDD will either contain all pairs (k, (Some(v), Some(w))) for v inthis
, or the pair (k, (None, Some(w))) if no elements inthis
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
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
orPairRDDFunctions.reduceByKey
will provide much better performance.
-
join
Return an RDD containing all pairs of elements with matching keys inthis
andother
. Each pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is inthis
and (k, v2) is inother
. Performs a hash join across the cluster.- Parameters:
other
- (undocumented)- Returns:
- (undocumented)
-
join
public <W> RDD<scala.Tuple2<K,scala.Tuple2<V, joinW>>> (RDD<scala.Tuple2<K, W>> other, int numPartitions) Return an RDD containing all pairs of elements with matching keys inthis
andother
. Each pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is inthis
and (k, v2) is inother
. Performs a hash join across the cluster.- Parameters:
other
- (undocumented)numPartitions
- (undocumented)- Returns:
- (undocumented)
-
leftOuterJoin
public <W> RDD<scala.Tuple2<K,scala.Tuple2<V, leftOuterJoinscala.Option<W>>>> (RDD<scala.Tuple2<K, W>> other) Perform a left outer join ofthis
andother
. For each element (k, v) inthis
, the resulting RDD will either contain all pairs (k, (v, Some(w))) for w inother
, or the pair (k, (v, None)) if no elements inother
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, leftOuterJoinscala.Option<W>>>> (RDD<scala.Tuple2<K, W>> other, int numPartitions) Perform a left outer join ofthis
andother
. For each element (k, v) inthis
, the resulting RDD will either contain all pairs (k, (v, Some(w))) for w inother
, or the pair (k, (v, None)) if no elements inother
have key k. Hash-partitions the output intonumPartitions
partitions.- Parameters:
other
- (undocumented)numPartitions
- (undocumented)- Returns:
- (undocumented)
-
rightOuterJoin
public <W> RDD<scala.Tuple2<K,scala.Tuple2<scala.Option<V>, rightOuterJoinW>>> (RDD<scala.Tuple2<K, W>> other) Perform a right outer join ofthis
andother
. For each element (k, w) inother
, the resulting RDD will either contain all pairs (k, (Some(v), w)) for v inthis
, or the pair (k, (None, w)) if no elements inthis
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>, rightOuterJoinW>>> (RDD<scala.Tuple2<K, W>> other, int numPartitions) Perform a right outer join ofthis
andother
. For each element (k, w) inother
, the resulting RDD will either contain all pairs (k, (Some(v), w)) for v inthis
, or the pair (k, (None, w)) if no elements inthis
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>, fullOuterJoinscala.Option<W>>>> (RDD<scala.Tuple2<K, W>> other) Perform a full outer join ofthis
andother
. For each element (k, v) inthis
, the resulting RDD will either contain all pairs (k, (Some(v), Some(w))) for w inother
, or the pair (k, (Some(v), None)) if no elements inother
have key k. Similarly, for each element (k, w) inother
, the resulting RDD will either contain all pairs (k, (Some(v), Some(w))) for v inthis
, or the pair (k, (None, Some(w))) if no elements inthis
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>, fullOuterJoinscala.Option<W>>>> (RDD<scala.Tuple2<K, W>> other, int numPartitions) Perform a full outer join ofthis
andother
. For each element (k, v) inthis
, the resulting RDD will either contain all pairs (k, (Some(v), Some(w))) for w inother
, or the pair (k, (Some(v), None)) if no elements inother
have key k. Similarly, for each element (k, w) inother
, the resulting RDD will either contain all pairs (k, (Some(v), Some(w))) for v inthis
, or the pair (k, (None, Some(w))) if no elements inthis
have key k. Hash-partitions the resulting RDD into the given number of partitions.- Parameters:
other
- (undocumented)numPartitions
- (undocumented)- Returns:
- (undocumented)
-
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
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.IterableOnce<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, RDD<scala.Tuple2<K,W3> scala.Tuple4<scala.collection.Iterable<V>, cogroupscala.collection.Iterable<W1>, scala.collection.Iterable<W2>, scala.collection.Iterable<W3>>>> (RDD<scala.Tuple2<K, W1>> other1, RDD<scala.Tuple2<K, W2>> other2, RDD<scala.Tuple2<K, W3>> other3, Partitioner partitioner) For each key k inthis
orother1
orother2
orother3
, return a resulting RDD that contains a tuple with the list of values for that key inthis
,other1
,other2
andother3
.- Parameters:
other1
- (undocumented)other2
- (undocumented)other3
- (undocumented)partitioner
- (undocumented)- Returns:
- (undocumented)
-
cogroup
public <W> RDD<scala.Tuple2<K,scala.Tuple2<scala.collection.Iterable<V>, cogroupscala.collection.Iterable<W>>>> (RDD<scala.Tuple2<K, W>> other, Partitioner partitioner) For each key k inthis
orother
, return a resulting RDD that contains a tuple with the list of values for that key inthis
as well asother
.- Parameters:
other
- (undocumented)partitioner
- (undocumented)- Returns:
- (undocumented)
-
cogroup
public <W1,W2> RDD<scala.Tuple2<K,scala.Tuple3<scala.collection.Iterable<V>, cogroupscala.collection.Iterable<W1>, scala.collection.Iterable<W2>>>> (RDD<scala.Tuple2<K, W1>> other1, RDD<scala.Tuple2<K, W2>> other2, Partitioner partitioner) For each key k inthis
orother1
orother2
, return a resulting RDD that contains a tuple with the list of values for that key inthis
,other1
andother2
.- Parameters:
other1
- (undocumented)other2
- (undocumented)partitioner
- (undocumented)- Returns:
- (undocumented)
-
cogroup
public <W1,W2, RDD<scala.Tuple2<K,W3> scala.Tuple4<scala.collection.Iterable<V>, cogroupscala.collection.Iterable<W1>, scala.collection.Iterable<W2>, scala.collection.Iterable<W3>>>> (RDD<scala.Tuple2<K, W1>> other1, RDD<scala.Tuple2<K, W2>> other2, RDD<scala.Tuple2<K, W3>> other3) For each key k inthis
orother1
orother2
orother3
, return a resulting RDD that contains a tuple with the list of values for that key inthis
,other1
,other2
andother3
.- Parameters:
other1
- (undocumented)other2
- (undocumented)other3
- (undocumented)- Returns:
- (undocumented)
-
cogroup
public <W> RDD<scala.Tuple2<K,scala.Tuple2<scala.collection.Iterable<V>, cogroupscala.collection.Iterable<W>>>> (RDD<scala.Tuple2<K, W>> other) For each key k inthis
orother
, return a resulting RDD that contains a tuple with the list of values for that key inthis
as well asother
.- Parameters:
other
- (undocumented)- Returns:
- (undocumented)
-
cogroup
public <W1,W2> RDD<scala.Tuple2<K,scala.Tuple3<scala.collection.Iterable<V>, cogroupscala.collection.Iterable<W1>, scala.collection.Iterable<W2>>>> (RDD<scala.Tuple2<K, W1>> other1, RDD<scala.Tuple2<K, W2>> other2) For each key k inthis
orother1
orother2
, return a resulting RDD that contains a tuple with the list of values for that key inthis
,other1
andother2
.- Parameters:
other1
- (undocumented)other2
- (undocumented)- Returns:
- (undocumented)
-
cogroup
public <W> RDD<scala.Tuple2<K,scala.Tuple2<scala.collection.Iterable<V>, cogroupscala.collection.Iterable<W>>>> (RDD<scala.Tuple2<K, W>> other, int numPartitions) For each key k inthis
orother
, return a resulting RDD that contains a tuple with the list of values for that key inthis
as well asother
.- Parameters:
other
- (undocumented)numPartitions
- (undocumented)- Returns:
- (undocumented)
-
cogroup
public <W1,W2> RDD<scala.Tuple2<K,scala.Tuple3<scala.collection.Iterable<V>, cogroupscala.collection.Iterable<W1>, scala.collection.Iterable<W2>>>> (RDD<scala.Tuple2<K, W1>> other1, RDD<scala.Tuple2<K, W2>> other2, int numPartitions) For each key k inthis
orother1
orother2
, return a resulting RDD that contains a tuple with the list of values for that key inthis
,other1
andother2
.- Parameters:
other1
- (undocumented)other2
- (undocumented)numPartitions
- (undocumented)- Returns:
- (undocumented)
-
cogroup
public <W1,W2, RDD<scala.Tuple2<K,W3> scala.Tuple4<scala.collection.Iterable<V>, cogroupscala.collection.Iterable<W1>, scala.collection.Iterable<W2>, scala.collection.Iterable<W3>>>> (RDD<scala.Tuple2<K, W1>> other1, RDD<scala.Tuple2<K, W2>> other2, RDD<scala.Tuple2<K, W3>> other3, int numPartitions) For each key k inthis
orother1
orother2
orother3
, return a resulting RDD that contains a tuple with the list of values for that key inthis
,other1
,other2
andother3
.- Parameters:
other1
- (undocumented)other2
- (undocumented)other3
- (undocumented)numPartitions
- (undocumented)- Returns:
- (undocumented)
-
groupWith
public <W> RDD<scala.Tuple2<K,scala.Tuple2<scala.collection.Iterable<V>, groupWithscala.collection.Iterable<W>>>> (RDD<scala.Tuple2<K, W>> other) Alias for cogroup. -
groupWith
public <W1,W2> RDD<scala.Tuple2<K,scala.Tuple3<scala.collection.Iterable<V>, groupWithscala.collection.Iterable<W1>, scala.collection.Iterable<W2>>>> (RDD<scala.Tuple2<K, W1>> other1, RDD<scala.Tuple2<K, W2>> other2) Alias for cogroup. -
groupWith
public <W1,W2, RDD<scala.Tuple2<K,W3> scala.Tuple4<scala.collection.Iterable<V>, groupWithscala.collection.Iterable<W1>, scala.collection.Iterable<W2>, scala.collection.Iterable<W3>>>> (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 fromthis
whose keys are not inother
.Uses
this
partitioner/partition size, because even ifother
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 fromthis
whose keys are not inother
.- 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 fromthis
whose keys are not inother
.- Parameters:
other
- (undocumented)p
- (undocumented)evidence$6
- (undocumented)- Returns:
- (undocumented)
-
lookup
Return the list of values in the RDD for keykey
. 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 HadoopOutputFormat
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 HadoopOutputFormat
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 APIOutputFormat
(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 APIOutputFormat
(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 HadoopOutputFormat
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 HadoopOutputFormat
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
Return an RDD with the keys of each tuple.- Returns:
- (undocumented)
-
values
Return an RDD with the values of each tuple.- Returns:
- (undocumented)
-