Class JavaPairRDD<K,V>
- All Implemented Interfaces:
Serializable
,JavaRDDLike<scala.Tuple2<K,
,V>, JavaPairRDD<K, V>> scala.Serializable
- Direct Known Subclasses:
JavaHadoopRDD
,JavaNewHadoopRDD
- See Also:
-
Constructor Summary
-
Method Summary
Modifier and TypeMethodDescription<U> JavaPairRDD<K,
U> aggregateByKey
(U zeroValue, int numPartitions, Function2<U, V, U> seqFunc, Function2<U, U, U> combFunc) Aggregate the values of each key, using given combine functions and a neutral "zero value".<U> JavaPairRDD<K,
U> aggregateByKey
(U zeroValue, Function2<U, V, U> seqFunc, Function2<U, U, U> combFunc) Aggregate the values of each key, using given combine functions and a neutral "zero value".<U> JavaPairRDD<K,
U> aggregateByKey
(U zeroValue, Partitioner partitioner, Function2<U, V, U> seqFunc, Function2<U, U, U> combFunc) Aggregate the values of each key, using given combine functions and a neutral "zero value".cache()
Persist this RDD with the default storage level (MEMORY_ONLY
).classTag()
coalesce
(int numPartitions) Return a new RDD that is reduced intonumPartitions
partitions.coalesce
(int numPartitions, boolean shuffle) Return a new RDD that is reduced intonumPartitions
partitions.<W> JavaPairRDD<K,
scala.Tuple2<Iterable<V>, Iterable<W>>> cogroup
(JavaPairRDD<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
.<W> JavaPairRDD<K,
scala.Tuple2<Iterable<V>, Iterable<W>>> cogroup
(JavaPairRDD<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
.<W> JavaPairRDD<K,
scala.Tuple2<Iterable<V>, Iterable<W>>> cogroup
(JavaPairRDD<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
.cogroup
(JavaPairRDD<K, W1> other1, JavaPairRDD<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
.cogroup
(JavaPairRDD<K, W1> other1, JavaPairRDD<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
.cogroup
(JavaPairRDD<K, W1> other1, JavaPairRDD<K, W2> other2, JavaPairRDD<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
.cogroup
(JavaPairRDD<K, W1> other1, JavaPairRDD<K, W2> other2, JavaPairRDD<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
.cogroup
(JavaPairRDD<K, W1> other1, JavaPairRDD<K, W2> other2, JavaPairRDD<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
.cogroup
(JavaPairRDD<K, W1> other1, JavaPairRDD<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
.Return the key-value pairs in this RDD to the master as a Map.<C> JavaPairRDD<K,
C> combineByKey
(Function<V, C> createCombiner, Function2<C, V, C> mergeValue, Function2<C, C, C> mergeCombiners) Simplified version of combineByKey that hash-partitions the resulting RDD using the existing partitioner/parallelism level and using map-side aggregation.<C> JavaPairRDD<K,
C> combineByKey
(Function<V, C> createCombiner, Function2<C, V, C> mergeValue, Function2<C, C, C> mergeCombiners, int numPartitions) Simplified version of combineByKey that hash-partitions the output RDD and uses map-side aggregation.<C> JavaPairRDD<K,
C> combineByKey
(Function<V, C> createCombiner, Function2<C, V, C> mergeValue, Function2<C, C, C> mergeCombiners, Partitioner partitioner) Generic function to combine the elements for each key using a custom set of aggregation functions.<C> JavaPairRDD<K,
C> combineByKey
(Function<V, C> createCombiner, Function2<C, V, C> mergeValue, 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.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.Count the number of elements for each key, and return the result to the master as a Map.countByKeyApprox
(long timeout) Approximate version of countByKey that can return a partial result if it does not finish within a timeout.countByKeyApprox
(long timeout, double confidence) Approximate version of countByKey that can return a partial result if it does not finish within a timeout.distinct()
Return a new RDD containing the distinct elements in this RDD.distinct
(int numPartitions) Return a new RDD containing the distinct elements in this RDD.Return a new RDD containing only the elements that satisfy a predicate.filterByRange
(Comparator<K> comp, K lower, K upper) Return a RDD containing only the elements in the inclusive rangelower
toupper
.filterByRange
(K lower, K upper) Return a RDD containing only the elements in the inclusive rangelower
toupper
.first()
Return the first element in this RDD.<U> JavaPairRDD<K,
U> flatMapValues
(FlatMapFunction<V, 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.).static <K,
V> JavaPairRDD<K, V> fromJavaRDD
(JavaRDD<scala.Tuple2<K, V>> rdd) Convert a JavaRDD of key-value pairs to JavaPairRDD.static <K,
V> JavaPairRDD<K, V> fromRDD
(RDD<scala.Tuple2<K, V>> rdd, scala.reflect.ClassTag<K> evidence$5, scala.reflect.ClassTag<V> evidence$6) <W> JavaPairRDD<K,
scala.Tuple2<Optional<V>, Optional<W>>> fullOuterJoin
(JavaPairRDD<K, W> other) Perform a full outer join ofthis
andother
.<W> JavaPairRDD<K,
scala.Tuple2<Optional<V>, Optional<W>>> fullOuterJoin
(JavaPairRDD<K, W> other, int numPartitions) Perform a full outer join ofthis
andother
.<W> JavaPairRDD<K,
scala.Tuple2<Optional<V>, Optional<W>>> fullOuterJoin
(JavaPairRDD<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.<W> JavaPairRDD<K,
scala.Tuple2<Iterable<V>, Iterable<W>>> groupWith
(JavaPairRDD<K, W> other) Alias for cogroup.groupWith
(JavaPairRDD<K, W1> other1, JavaPairRDD<K, W2> other2) Alias for cogroup.groupWith
(JavaPairRDD<K, W1> other1, JavaPairRDD<K, W2> other2, JavaPairRDD<K, W3> other3) Alias for cogroup.intersection
(JavaPairRDD<K, V> other) Return the intersection of this RDD and another one.<W> JavaPairRDD<K,
scala.Tuple2<V, W>> join
(JavaPairRDD<K, W> other) Return an RDD containing all pairs of elements with matching keys inthis
andother
.<W> JavaPairRDD<K,
scala.Tuple2<V, W>> join
(JavaPairRDD<K, W> other, int numPartitions) Return an RDD containing all pairs of elements with matching keys inthis
andother
.<W> JavaPairRDD<K,
scala.Tuple2<V, W>> join
(JavaPairRDD<K, W> other, Partitioner partitioner) Return an RDD containing all pairs of elements with matching keys inthis
andother
.scala.reflect.ClassTag<K>
keys()
Return an RDD with the keys of each tuple.<W> JavaPairRDD<K,
scala.Tuple2<V, Optional<W>>> leftOuterJoin
(JavaPairRDD<K, W> other) Perform a left outer join ofthis
andother
.<W> JavaPairRDD<K,
scala.Tuple2<V, Optional<W>>> leftOuterJoin
(JavaPairRDD<K, W> other, int numPartitions) Perform a left outer join ofthis
andother
.<W> JavaPairRDD<K,
scala.Tuple2<V, Optional<W>>> leftOuterJoin
(JavaPairRDD<K, W> other, Partitioner partitioner) Perform a left outer join ofthis
andother
.Return the list of values in the RDD for keykey
.<U> JavaPairRDD<K,
U> Pass each value in the key-value pair RDD through a map function without changing the keys; this also retains the original RDD's partitioning.partitionBy
(Partitioner partitioner) Return a copy of the RDD partitioned using the specified partitioner.persist
(StorageLevel newLevel) Set this RDD's storage level to persist its values across operations after the first time it is computed.rdd()
reduceByKey
(Function2<V, V, V> func) Merge the values for each key using an associative and commutative reduce function.reduceByKey
(Function2<V, V, V> func, int numPartitions) Merge the values for each key using an associative and commutative reduce function.reduceByKey
(Partitioner partitioner, Function2<V, V, V> func) Merge the values for each key using an associative and commutative reduce function.reduceByKeyLocally
(Function2<V, V, V> func) Merge the values for each key using an associative and commutative reduce function, but return the result immediately to the master as a Map.repartition
(int numPartitions) Return a new RDD that has exactly numPartitions partitions.repartitionAndSortWithinPartitions
(Partitioner partitioner) Repartition the RDD according to the given partitioner and, within each resulting partition, sort records by their keys.repartitionAndSortWithinPartitions
(Partitioner partitioner, Comparator<K> comp) Repartition the RDD according to the given partitioner and, within each resulting partition, sort records by their keys.<W> JavaPairRDD<K,
scala.Tuple2<Optional<V>, W>> rightOuterJoin
(JavaPairRDD<K, W> other) Perform a right outer join ofthis
andother
.<W> JavaPairRDD<K,
scala.Tuple2<Optional<V>, W>> rightOuterJoin
(JavaPairRDD<K, W> other, int numPartitions) Perform a right outer join ofthis
andother
.<W> JavaPairRDD<K,
scala.Tuple2<Optional<V>, W>> rightOuterJoin
(JavaPairRDD<K, W> other, Partitioner partitioner) Perform a right outer join ofthis
andother
.sample
(boolean withReplacement, double fraction) Return a sampled subset of this RDD.sample
(boolean withReplacement, double fraction, long seed) Return a sampled subset of this RDD.sampleByKey
(boolean withReplacement, Map<K, Double> fractions) Return a subset of this RDD sampled by key (via stratified sampling).sampleByKey
(boolean withReplacement, Map<K, Double> fractions, long seed) Return a subset of this RDD sampled by key (via stratified sampling).sampleByKeyExact
(boolean withReplacement, Map<K, Double> fractions) 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).sampleByKeyExact
(boolean withReplacement, Map<K, Double> 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.<F extends org.apache.hadoop.mapred.OutputFormat<?,
?>>
voidsaveAsHadoopFile
(String path, Class<?> keyClass, Class<?> valueClass, Class<F> outputFormatClass) Output the RDD to any Hadoop-supported file system.<F extends org.apache.hadoop.mapred.OutputFormat<?,
?>>
voidsaveAsHadoopFile
(String path, Class<?> keyClass, Class<?> valueClass, Class<F> outputFormatClass, Class<? extends org.apache.hadoop.io.compress.CompressionCodec> codec) Output the RDD to any Hadoop-supported file system, compressing with the supplied codec.<F extends org.apache.hadoop.mapred.OutputFormat<?,
?>>
voidsaveAsHadoopFile
(String path, Class<?> keyClass, Class<?> valueClass, Class<F> outputFormatClass, org.apache.hadoop.mapred.JobConf conf) Output the RDD to any Hadoop-supported file system.void
saveAsNewAPIHadoopDataset
(org.apache.hadoop.conf.Configuration conf) Output the RDD to any Hadoop-supported storage system, using a Configuration object for that storage system.<F extends org.apache.hadoop.mapreduce.OutputFormat<?,
?>>
voidsaveAsNewAPIHadoopFile
(String path, Class<?> keyClass, Class<?> valueClass, Class<F> outputFormatClass) Output the RDD to any Hadoop-supported file system.<F extends org.apache.hadoop.mapreduce.OutputFormat<?,
?>>
voidsaveAsNewAPIHadoopFile
(String path, Class<?> keyClass, Class<?> valueClass, Class<F> outputFormatClass, org.apache.hadoop.conf.Configuration conf) Output the RDD to any Hadoop-supported file system.Assign a name to this RDDSort the RDD by key, so that each partition contains a sorted range of the elements in ascending order.sortByKey
(boolean ascending) Sort the RDD by key, so that each partition contains a sorted range of the elements.sortByKey
(boolean ascending, int numPartitions) Sort the RDD by key, so that each partition contains a sorted range of the elements.sortByKey
(Comparator<K> comp) Sort the RDD by key, so that each partition contains a sorted range of the elements.sortByKey
(Comparator<K> comp, boolean ascending) Sort the RDD by key, so that each partition contains a sorted range of the elements.sortByKey
(Comparator<K> comp, boolean ascending, int numPartitions) Sort the RDD by key, so that each partition contains a sorted range of the elements.subtract
(JavaPairRDD<K, V> other) Return an RDD with the elements fromthis
that are not inother
.subtract
(JavaPairRDD<K, V> other, int numPartitions) Return an RDD with the elements fromthis
that are not inother
.subtract
(JavaPairRDD<K, V> other, Partitioner p) Return an RDD with the elements fromthis
that are not inother
.<W> JavaPairRDD<K,
V> subtractByKey
(JavaPairRDD<K, W> other) Return an RDD with the pairs fromthis
whose keys are not inother
.<W> JavaPairRDD<K,
V> subtractByKey
(JavaPairRDD<K, W> other, int numPartitions) Return an RDD with the pairs fromthis
whose keys are not inother
.<W> JavaPairRDD<K,
V> subtractByKey
(JavaPairRDD<K, W> other, Partitioner p) Return an RDD with the pairs fromthis
whose keys are not inother
.static <K,
V> RDD<scala.Tuple2<K, V>> toRDD
(JavaPairRDD<K, V> rdd) union
(JavaPairRDD<K, V> other) Return the union of this RDD and another one.Mark the RDD as non-persistent, and remove all blocks for it from memory and disk.unpersist
(boolean blocking) Mark the RDD as non-persistent, and remove all blocks for it from memory and disk.values()
Return an RDD with the values of each tuple.scala.reflect.ClassTag<V>
Methods inherited from class java.lang.Object
equals, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
Methods inherited from interface org.apache.spark.api.java.JavaRDDLike
aggregate, cartesian, checkpoint, collect, collectAsync, collectPartitions, context, count, countApprox, countApprox, countApproxDistinct, countAsync, countByValue, countByValueApprox, countByValueApprox, flatMap, flatMapToDouble, flatMapToPair, fold, foreach, foreachAsync, foreachPartition, foreachPartitionAsync, getCheckpointFile, getNumPartitions, getStorageLevel, glom, groupBy, groupBy, id, isCheckpointed, isEmpty, iterator, keyBy, map, mapPartitions, mapPartitions, mapPartitionsToDouble, mapPartitionsToDouble, mapPartitionsToPair, mapPartitionsToPair, mapPartitionsWithIndex, mapToDouble, mapToPair, max, min, name, partitioner, partitions, pipe, pipe, pipe, pipe, pipe, reduce, saveAsObjectFile, saveAsTextFile, saveAsTextFile, take, takeAsync, takeOrdered, takeOrdered, takeSample, takeSample, toDebugString, toLocalIterator, top, top, treeAggregate, treeAggregate, treeAggregate, treeReduce, treeReduce, zip, zipPartitions, zipWithIndex, zipWithUniqueId
-
Constructor Details
-
JavaPairRDD
-
-
Method Details
-
fromRDD
public static <K,V> JavaPairRDD<K,V> fromRDD(RDD<scala.Tuple2<K, V>> rdd, scala.reflect.ClassTag<K> evidence$5, scala.reflect.ClassTag<V> evidence$6) -
toRDD
-
fromJavaRDD
Convert a JavaRDD of key-value pairs to JavaPairRDD. -
rdd
-
kClassTag
-
vClassTag
-
wrapRDD
-
classTag
-
cache
Persist this RDD with the default storage level (MEMORY_ONLY
).- Returns:
- (undocumented)
-
persist
Set this RDD's storage level to persist its values across operations after the first time it is computed. Can only be called once on each RDD.- Parameters:
newLevel
- (undocumented)- Returns:
- (undocumented)
-
unpersist
Mark the RDD as non-persistent, and remove all blocks for it from memory and disk. This method blocks until all blocks are deleted.- Returns:
- (undocumented)
-
unpersist
Mark the RDD as non-persistent, and remove all blocks for it from memory and disk.- Parameters:
blocking
- Whether to block until all blocks are deleted.- Returns:
- (undocumented)
-
distinct
Return a new RDD containing the distinct elements in this RDD.- Returns:
- (undocumented)
-
distinct
Return a new RDD containing the distinct elements in this RDD.- Parameters:
numPartitions
- (undocumented)- Returns:
- (undocumented)
-
filter
Return a new RDD containing only the elements that satisfy a predicate.- Parameters:
f
- (undocumented)- Returns:
- (undocumented)
-
coalesce
Return a new RDD that is reduced intonumPartitions
partitions.- Parameters:
numPartitions
- (undocumented)- Returns:
- (undocumented)
-
coalesce
Return a new RDD that is reduced intonumPartitions
partitions.- Parameters:
numPartitions
- (undocumented)shuffle
- (undocumented)- Returns:
- (undocumented)
-
repartition
Return a new RDD that has exactly numPartitions partitions.Can increase or decrease the level of parallelism in this RDD. Internally, this uses a shuffle to redistribute data.
If you are decreasing the number of partitions in this RDD, consider using
coalesce
, which can avoid performing a shuffle.- Parameters:
numPartitions
- (undocumented)- Returns:
- (undocumented)
-
sample
Return a sampled subset of this RDD.- Parameters:
withReplacement
- (undocumented)fraction
- (undocumented)- Returns:
- (undocumented)
-
sample
Return a sampled subset of this RDD.- Parameters:
withReplacement
- (undocumented)fraction
- (undocumented)seed
- (undocumented)- Returns:
- (undocumented)
-
sampleByKey
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
- (undocumented)fractions
- (undocumented)seed
- (undocumented)- Returns:
- (undocumented)
-
sampleByKey
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.Use Utils.random.nextLong as the default seed for the random number generator.
- Parameters:
withReplacement
- (undocumented)fractions
- (undocumented)- Returns:
- (undocumented)
-
sampleByKeyExact
public JavaPairRDD<K,V> sampleByKeyExact(boolean withReplacement, Map<K, Double> 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
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
- (undocumented)fractions
- (undocumented)seed
- (undocumented)- Returns:
- (undocumented)
-
sampleByKeyExact
Return a subset of this RDD sampled by key (via stratified sampling) containing exactly math.ceil(numItems * samplingRate) for each stratum (group of pairs with the same key).This method differs from
sampleByKey
in that we make additional passes over the RDD to create a sample size that's exactly equal to the sum of math.ceil(numItems * samplingRate) over all key values with a 99.99% confidence. When sampling without replacement, we need one additional pass over the RDD to guarantee sample size; when sampling with replacement, we need two additional passes.Use Utils.random.nextLong as the default seed for the random number generator.
- Parameters:
withReplacement
- (undocumented)fractions
- (undocumented)- Returns:
- (undocumented)
-
union
Return the union of this RDD and another one. Any identical elements will appear multiple times (use.distinct()
to eliminate them).- Parameters:
other
- (undocumented)- Returns:
- (undocumented)
-
intersection
Return the intersection of this RDD and another one. The output will not contain any duplicate elements, even if the input RDDs did.- Parameters:
other
- (undocumented)- Returns:
- (undocumented)
- Note:
- This method performs a shuffle internally.
-
first
Description copied from interface:JavaRDDLike
Return the first element in this RDD.- Returns:
- (undocumented)
-
combineByKey
public <C> JavaPairRDD<K,C> combineByKey(Function<V, C> createCombiner, Function2<C, V, C> mergeValue, 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. Turns a JavaPairRDD[(K, V)] into a result of type JavaPairRDD[(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, the serializer that is use for the shuffle, 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)- 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, List[Int]).
-
combineByKey
public <C> JavaPairRDD<K,C> combineByKey(Function<V, C> createCombiner, Function2<C, V, C> mergeValue, Function2<C, C, C> mergeCombiners, Partitioner partitioner) Generic function to combine the elements for each key using a custom set of aggregation functions. Turns a JavaPairRDD[(K, V)] into a result of type JavaPairRDD[(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. This method automatically uses map-side aggregation in shuffling the RDD.
- Parameters:
createCombiner
- (undocumented)mergeValue
- (undocumented)mergeCombiners
- (undocumented)partitioner
- (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, List[Int]).
-
combineByKey
public <C> JavaPairRDD<K,C> combineByKey(Function<V, C> createCombiner, Function2<C, V, C> mergeValue, Function2<C, C, C> mergeCombiners, int numPartitions) Simplified version of combineByKey that hash-partitions the output RDD and uses map-side aggregation.- Parameters:
createCombiner
- (undocumented)mergeValue
- (undocumented)mergeCombiners
- (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.- Parameters:
partitioner
- (undocumented)func
- (undocumented)- Returns:
- (undocumented)
-
reduceByKeyLocally
Merge the values for each key using an associative and commutative reduce function, but return the result 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, and return the result to the master as a Map. -
countByKeyApprox
Approximate version of countByKey that can return a partial result if it does not finish within a timeout.- Parameters:
timeout
- (undocumented)- Returns:
- (undocumented)
-
countByKeyApprox
Approximate version of countByKey that can return a partial result if it does not finish within a timeout.- Parameters:
timeout
- (undocumented)confidence
- (undocumented)- Returns:
- (undocumented)
-
aggregateByKey
public <U> JavaPairRDD<K,U> aggregateByKey(U zeroValue, Partitioner partitioner, Function2<U, V, U> seqFunc, Function2<U, U, U> combFunc) 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)seqFunc
- (undocumented)combFunc
- (undocumented)- Returns:
- (undocumented)
-
aggregateByKey
public <U> JavaPairRDD<K,U> aggregateByKey(U zeroValue, int numPartitions, Function2<U, V, U> seqFunc, Function2<U, U, U> combFunc) 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)seqFunc
- (undocumented)combFunc
- (undocumented)- Returns:
- (undocumented)
-
aggregateByKey
public <U> JavaPairRDD<K,U> aggregateByKey(U zeroValue, Function2<U, V, U> seqFunc, Function2<U, U, U> combFunc) 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. 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)seqFunc
- (undocumented)combFunc
- (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)partitioner
- (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)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)
-
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)
-
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.- Parameters:
partitioner
- (undocumented)- Returns:
- (undocumented)
- Note:
- If you are grouping in order to perform an aggregation (such as a sum or average) over
each key, using
JavaPairRDD.reduceByKey
orJavaPairRDD.combineByKey
will provide much better performance.
-
groupByKey
Group the values for each key in the RDD into a single sequence. Hash-partitions the resulting RDD with intonumPartitions
partitions.- Parameters:
numPartitions
- (undocumented)- Returns:
- (undocumented)
- Note:
- If you are grouping in order to perform an aggregation (such as a sum or average) over
each key, using
JavaPairRDD.reduceByKey
orJavaPairRDD.combineByKey
will provide much better performance.
-
subtract
Return an RDD with the elements fromthis
that are not inother
.Uses
this
partitioner/partition size, because even ifother
is huge, the resulting RDD will be <= us.- Parameters:
other
- (undocumented)- Returns:
- (undocumented)
-
subtract
Return an RDD with the elements fromthis
that are not inother
.- Parameters:
other
- (undocumented)numPartitions
- (undocumented)- Returns:
- (undocumented)
-
subtract
Return an RDD with the elements fromthis
that are not inother
.- Parameters:
other
- (undocumented)p
- (undocumented)- Returns:
- (undocumented)
-
subtractByKey
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 <= us.- Parameters:
other
- (undocumented)- Returns:
- (undocumented)
-
subtractByKey
Return an RDD with the pairs fromthis
whose keys are not inother
.- Parameters:
other
- (undocumented)numPartitions
- (undocumented)- Returns:
- (undocumented)
-
subtractByKey
Return an RDD with the pairs fromthis
whose keys are not inother
.- Parameters:
other
- (undocumented)p
- (undocumented)- Returns:
- (undocumented)
-
partitionBy
Return a copy of the RDD partitioned using the specified partitioner.- Parameters:
partitioner
- (undocumented)- Returns:
- (undocumented)
-
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
. Uses the given Partitioner to partition the output RDD.- Parameters:
other
- (undocumented)partitioner
- (undocumented)- Returns:
- (undocumented)
-
leftOuterJoin
public <W> JavaPairRDD<K,scala.Tuple2<V, leftOuterJoinOptional<W>>> (JavaPairRDD<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> JavaPairRDD<K,scala.Tuple2<Optional<V>, rightOuterJoinW>> (JavaPairRDD<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> JavaPairRDD<K,scala.Tuple2<Optional<V>, fullOuterJoinOptional<W>>> (JavaPairRDD<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> JavaPairRDD<K,C> combineByKey(Function<V, C> createCombiner, Function2<C, V, C> mergeValue, Function2<C, C, C> mergeCombiners) Simplified version of combineByKey that hash-partitions the resulting RDD using the existing partitioner/parallelism level and using map-side aggregation.- Parameters:
createCombiner
- (undocumented)mergeValue
- (undocumented)mergeCombiners
- (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)
-
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.- Returns:
- (undocumented)
- Note:
- If you are grouping in order to perform an aggregation (such as a sum or average) over
each key, using
JavaPairRDD.reduceByKey
orJavaPairRDD.combineByKey
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
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
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> JavaPairRDD<K,scala.Tuple2<V, leftOuterJoinOptional<W>>> (JavaPairRDD<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
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> JavaPairRDD<K,scala.Tuple2<Optional<V>, rightOuterJoinW>> (JavaPairRDD<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> JavaPairRDD<K,scala.Tuple2<Optional<V>, fullOuterJoinOptional<W>>> (JavaPairRDD<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> JavaPairRDD<K,scala.Tuple2<Optional<V>, fullOuterJoinOptional<W>>> (JavaPairRDD<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.- 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
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 <W> JavaPairRDD<K,scala.Tuple2<Iterable<V>, cogroupIterable<W>>> (JavaPairRDD<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> JavaPairRDD<K,scala.Tuple3<Iterable<V>, cogroupIterable<W1>, Iterable<W2>>> (JavaPairRDD<K, W1> other1, JavaPairRDD<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, JavaPairRDD<K,W3> scala.Tuple4<Iterable<V>, cogroupIterable<W1>, Iterable<W2>, Iterable<W3>>> (JavaPairRDD<K, W1> other1, JavaPairRDD<K, W2> other2, JavaPairRDD<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
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> JavaPairRDD<K,scala.Tuple3<Iterable<V>, cogroupIterable<W1>, Iterable<W2>>> (JavaPairRDD<K, W1> other1, JavaPairRDD<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 <W1,W2, JavaPairRDD<K,W3> scala.Tuple4<Iterable<V>, cogroupIterable<W1>, Iterable<W2>, Iterable<W3>>> (JavaPairRDD<K, W1> other1, JavaPairRDD<K, W2> other2, JavaPairRDD<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> JavaPairRDD<K,scala.Tuple2<Iterable<V>, cogroupIterable<W>>> (JavaPairRDD<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> JavaPairRDD<K,scala.Tuple3<Iterable<V>, cogroupIterable<W1>, Iterable<W2>>> (JavaPairRDD<K, W1> other1, JavaPairRDD<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, JavaPairRDD<K,W3> scala.Tuple4<Iterable<V>, cogroupIterable<W1>, Iterable<W2>, Iterable<W3>>> (JavaPairRDD<K, W1> other1, JavaPairRDD<K, W2> other2, JavaPairRDD<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
Alias for cogroup. -
groupWith
public <W1,W2> JavaPairRDD<K,scala.Tuple3<Iterable<V>, groupWithIterable<W1>, Iterable<W2>>> (JavaPairRDD<K, W1> other1, JavaPairRDD<K, W2> other2) Alias for cogroup. -
groupWith
public <W1,W2, JavaPairRDD<K,W3> scala.Tuple4<Iterable<V>, groupWithIterable<W1>, Iterable<W2>, Iterable<W3>>> (JavaPairRDD<K, W1> other1, JavaPairRDD<K, W2> other2, JavaPairRDD<K, W3> other3) Alias for cogroup. -
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<?,?>> void saveAsHadoopFile(String path, Class<?> keyClass, Class<?> valueClass, Class<F> outputFormatClass, org.apache.hadoop.mapred.JobConf conf) Output the RDD to any Hadoop-supported file system. -
saveAsHadoopFile
public <F extends org.apache.hadoop.mapred.OutputFormat<?,?>> void saveAsHadoopFile(String path, Class<?> keyClass, Class<?> valueClass, Class<F> outputFormatClass) Output the RDD to any Hadoop-supported file system. -
saveAsHadoopFile
public <F extends org.apache.hadoop.mapred.OutputFormat<?,?>> void saveAsHadoopFile(String path, Class<?> keyClass, Class<?> valueClass, Class<F> outputFormatClass, Class<? extends org.apache.hadoop.io.compress.CompressionCodec> codec) Output the RDD to any Hadoop-supported file system, compressing with the supplied codec. -
saveAsNewAPIHadoopFile
public <F extends org.apache.hadoop.mapreduce.OutputFormat<?,?>> void saveAsNewAPIHadoopFile(String path, Class<?> keyClass, Class<?> valueClass, Class<F> outputFormatClass, org.apache.hadoop.conf.Configuration conf) Output the RDD to any Hadoop-supported file system. -
saveAsNewAPIHadoopDataset
public void saveAsNewAPIHadoopDataset(org.apache.hadoop.conf.Configuration conf) Output the RDD to any Hadoop-supported storage system, using a Configuration object for that storage system.- Parameters:
conf
- (undocumented)
-
saveAsNewAPIHadoopFile
public <F extends org.apache.hadoop.mapreduce.OutputFormat<?,?>> void saveAsNewAPIHadoopFile(String path, Class<?> keyClass, Class<?> valueClass, Class<F> outputFormatClass) Output the RDD to any Hadoop-supported file system. -
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)
-
repartitionAndSortWithinPartitions
Repartition the RDD according to the given partitioner and, within each resulting partition, sort records by their keys.This is more efficient than calling
repartition
and then sorting within each partition because it can push the sorting down into the shuffle machinery.- Parameters:
partitioner
- (undocumented)- Returns:
- (undocumented)
-
repartitionAndSortWithinPartitions
public JavaPairRDD<K,V> repartitionAndSortWithinPartitions(Partitioner partitioner, Comparator<K> comp) Repartition the RDD according to the given partitioner and, within each resulting partition, sort records by their keys.This is more efficient than calling
repartition
and then sorting within each partition because it can push the sorting down into the shuffle machinery.- Parameters:
partitioner
- (undocumented)comp
- (undocumented)- Returns:
- (undocumented)
-
sortByKey
Sort the RDD by key, so that each partition contains a sorted range of the elements in ascending order. Callingcollect
orsave
on the resulting RDD will return or output an ordered list of records (in thesave
case, they will be written to multiplepart-X
files in the filesystem, in order of the keys).- Returns:
- (undocumented)
-
sortByKey
Sort the RDD by key, so that each partition contains a sorted range of the elements. Callingcollect
orsave
on the resulting RDD will return or output an ordered list of records (in thesave
case, they will be written to multiplepart-X
files in the filesystem, in order of the keys).- Parameters:
ascending
- (undocumented)- Returns:
- (undocumented)
-
sortByKey
Sort the RDD by key, so that each partition contains a sorted range of the elements. Callingcollect
orsave
on the resulting RDD will return or output an ordered list of records (in thesave
case, they will be written to multiplepart-X
files in the filesystem, in order of the keys).- Parameters:
ascending
- (undocumented)numPartitions
- (undocumented)- Returns:
- (undocumented)
-
sortByKey
Sort the RDD by key, so that each partition contains a sorted range of the elements. Callingcollect
orsave
on the resulting RDD will return or output an ordered list of records (in thesave
case, they will be written to multiplepart-X
files in the filesystem, in order of the keys).- Parameters:
comp
- (undocumented)- Returns:
- (undocumented)
-
sortByKey
Sort the RDD by key, so that each partition contains a sorted range of the elements. Callingcollect
orsave
on the resulting RDD will return or output an ordered list of records (in thesave
case, they will be written to multiplepart-X
files in the filesystem, in order of the keys).- Parameters:
comp
- (undocumented)ascending
- (undocumented)- Returns:
- (undocumented)
-
sortByKey
Sort the RDD by key, so that each partition contains a sorted range of the elements. Callingcollect
orsave
on the resulting RDD will return or output an ordered list of records (in thesave
case, they will be written to multiplepart-X
files in the filesystem, in order of the keys).- Parameters:
comp
- (undocumented)ascending
- (undocumented)numPartitions
- (undocumented)- Returns:
- (undocumented)
-
filterByRange
Return a RDD containing only the elements in the inclusive rangelower
toupper
. If the RDD has been partitioned using aRangePartitioner
, then this operation can be performed efficiently by only scanning the partitions that might contain matching elements. Otherwise, a standardfilter
is applied to all partitions.- Parameters:
lower
- (undocumented)upper
- (undocumented)- Returns:
- (undocumented)
- Since:
- 3.1.0
-
filterByRange
Return a RDD containing only the elements in the inclusive rangelower
toupper
. If the RDD has been partitioned using aRangePartitioner
, then this operation can be performed efficiently by only scanning the partitions that might contain matching elements. Otherwise, a standardfilter
is applied to all partitions.- Parameters:
comp
- (undocumented)lower
- (undocumented)upper
- (undocumented)- Returns:
- (undocumented)
- Since:
- 3.1.0
-
keys
Return an RDD with the keys of each tuple.- Returns:
- (undocumented)
-
values
Return an RDD with the values of each tuple.- 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.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)
-
setName
Assign a name to this RDD
-