From 7e07c8211222a565d65894c2b50846e81e248358 Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Tue, 12 Jul 2022 16:57:53 +0200 Subject: [PATCH 01/16] wip seq and java rdd key value --- .../spark/extensions/KSparkExtensions.scala | 14 ++++ .../kotlinx/spark/api/Conversions.kt | 2 + .../org/jetbrains/kotlinx/spark/api/RDD.kt | 79 +++++++++++++++++++ .../org/jetbrains/kotlinx/spark/api/Seq.kt | 22 ++++++ 4 files changed, 117 insertions(+) create mode 100644 kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RDD.kt create mode 100644 kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Seq.kt diff --git a/core/src/main/scala/org/jetbrains/kotlinx/spark/extensions/KSparkExtensions.scala b/core/src/main/scala/org/jetbrains/kotlinx/spark/extensions/KSparkExtensions.scala index 2092d83e..9ad2f8ba 100644 --- a/core/src/main/scala/org/jetbrains/kotlinx/spark/extensions/KSparkExtensions.scala +++ b/core/src/main/scala/org/jetbrains/kotlinx/spark/extensions/KSparkExtensions.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql._ import java.util import scala.collection.JavaConverters +import scala.reflect.ClassTag object KSparkExtensions { @@ -53,4 +54,17 @@ object KSparkExtensions { } def sparkContext(s: SparkSession): SparkContext = s.sparkContext + + /** + * Produces a ClassTag[T], which is actually just a casted ClassTag[AnyRef]. + * + * This method is used to keep ClassTags out of the external Java API, as the Java compiler + * cannot produce them automatically. While this ClassTag-faking does please the compiler, + * it can cause problems at runtime if the Scala API relies on ClassTags for correctness. + * + * Often, though, a ClassTag[AnyRef] will not lead to incorrect behavior, just worse performance + * or security issues. For instance, an Array[AnyRef] can hold any type T, but may lose primitive + * specialization. + */ + def fakeClassTag[T]: ClassTag[T] = ClassTag.AnyRef.asInstanceOf[ClassTag[T]] } diff --git a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt index b9b5d306..22ab4143 100644 --- a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt +++ b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt @@ -44,6 +44,8 @@ import scala.collection.mutable.Map as ScalaMutableMap import scala.collection.mutable.Seq as ScalaMutableSeq import scala.collection.mutable.Set as ScalaMutableSet import org.apache.spark.streaming.State +import scala.collection.`Seq$` + /** Returns state value if it exists, else `null`. */ fun State.getOrNull(): T? = if (exists()) get() else null diff --git a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RDD.kt b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RDD.kt new file mode 100644 index 00000000..da88e786 --- /dev/null +++ b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RDD.kt @@ -0,0 +1,79 @@ +package org.jetbrains.kotlinx.spark.api + +import org.apache.spark.Partitioner +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.rdd.PairRDDFunctions +import org.apache.spark.serializer.Serializer +import org.jetbrains.kotlinx.spark.api.tuples.* +import org.jetbrains.kotlinx.spark.extensions.`KSparkExtensions$` +import scala.Tuple2 +import scala.reflect.ClassTag + +fun main() = withSpark { + val ds = sc.parallelize( + listOf(1 X "a", 2 X "b", 3 X "c", 4 X "d", 5 X "e", 6 X "f", 7 X "g", 8 X "h", 9 X "i", 10 X "j") + ).toPairRDD() + + + + +} + +/** + * 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). + * + * @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]). + */ +fun JavaRDD>.combineByKey( + createCombiner: (V) -> C, + mergeValue: (C, V) -> C, + mergeCombiners: (C, C) -> C, + partitioner: Partitioner, + mapSideCombine: Boolean = true, + serializer: Serializer? = null, +): JavaRDD> = + toPairRDD() + .combineByKey(createCombiner, mergeValue, mergeCombiners, partitioner, mapSideCombine, serializer) + .toTupleRDD() + + +/** + * Simplified version of combineByKey that hash-partitions the output RDD and uses map-side + * aggregation. + */ +fun JavaRDD>.combineByKey( + createCombiner: (V) -> C, + mergeValue: (C, V) -> C, + mergeCombiners: (C, C) -> C, + numPartitions: Int = rdd().sparkContext().defaultParallelism(), +): JavaRDD> = + toPairRDD() + .combineByKey(createCombiner, mergeValue, mergeCombiners, numPartitions) + .toTupleRDD() + +/** + * 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. + */ +fun JavaRDD>.reduceByKey( + partitioner: Partitioner = Partitioner.defaultPartitioner(rdd(), emptyImmutableSeq()), + func: (V, V) -> V, +): JavaRDD> = + toPairRDD() + .reduceByKey(partitioner, func) + .toTupleRDD() + diff --git a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Seq.kt b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Seq.kt new file mode 100644 index 00000000..14ac09e9 --- /dev/null +++ b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Seq.kt @@ -0,0 +1,22 @@ +package org.jetbrains.kotlinx.spark.api + + +fun emptySeq(): scala.collection.Seq = scala.collection.`Seq$`.`MODULE$`.empty() + +fun seqOf(vararg elements: T): scala.collection.Seq = + if (elements.isEmpty()) emptySeq() + else scala.collection.`Seq$`.`MODULE$`.from(elements.asIterable().asScalaIterable()) + +fun emptyImmutableSeq(): scala.collection.immutable.Seq = + scala.collection.immutable.`Seq$`.`MODULE$`.empty() + +fun immutableSeqOf(vararg elements: T): scala.collection.immutable.Seq = + if (elements.isEmpty()) emptyImmutableSeq() + else scala.collection.immutable.`Seq$`.`MODULE$`.from(elements.asIterable().asScalaIterable()) + +fun emptyMutableSeq(): scala.collection.mutable.Seq = + scala.collection.mutable.`Seq$`.`MODULE$`.empty() + +fun mutableSeqOf(vararg elements: T): scala.collection.mutable.Seq = + if (elements.isEmpty()) emptyMutableSeq() + else scala.collection.mutable.`Seq$`.`MODULE$`.from(elements.asIterable().asScalaIterable()) From 6e2deff551c05677bed9d2b83d600e7bc70d72f8 Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Tue, 12 Jul 2022 16:58:10 +0200 Subject: [PATCH 02/16] wip seq and java rdd key value --- .../main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt | 1 - 1 file changed, 1 deletion(-) diff --git a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt index 22ab4143..5212403e 100644 --- a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt +++ b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt @@ -44,7 +44,6 @@ import scala.collection.mutable.Map as ScalaMutableMap import scala.collection.mutable.Seq as ScalaMutableSeq import scala.collection.mutable.Set as ScalaMutableSet import org.apache.spark.streaming.State -import scala.collection.`Seq$` /** Returns state value if it exists, else `null`. */ From 38661ac1a379027de8d813c4702e6c15195a57aa Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Wed, 13 Jul 2022 17:40:58 +0200 Subject: [PATCH 03/16] will need tests, ported most/all javaPairRDD and PairRDDFunctions --- .../org/jetbrains/kotlinx/spark/api/RDD.kt | 901 +++++++++++++++++- .../org/jetbrains/kotlinx/spark/api/Seq.kt | 22 - 2 files changed, 858 insertions(+), 65 deletions(-) delete mode 100644 kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Seq.kt diff --git a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RDD.kt b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RDD.kt index da88e786..e33e9077 100644 --- a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RDD.kt +++ b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RDD.kt @@ -1,41 +1,27 @@ package org.jetbrains.kotlinx.spark.api +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.io.compress.CompressionCodec +import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.mapred.OutputFormat import org.apache.spark.Partitioner import org.apache.spark.api.java.JavaRDD -import org.apache.spark.rdd.PairRDDFunctions +import org.apache.spark.api.java.Optional +import org.apache.spark.partial.BoundedDouble +import org.apache.spark.partial.PartialResult import org.apache.spark.serializer.Serializer -import org.jetbrains.kotlinx.spark.api.tuples.* -import org.jetbrains.kotlinx.spark.extensions.`KSparkExtensions$` import scala.Tuple2 -import scala.reflect.ClassTag - -fun main() = withSpark { - val ds = sc.parallelize( - listOf(1 X "a", 2 X "b", 3 X "c", 4 X "d", 5 X "e", 6 X "f", 7 X "g", 8 X "h", 9 X "i", 10 X "j") - ).toPairRDD() - - - - -} +import scala.Tuple3 +import scala.Tuple4 +import kotlin.random.Random +import org.apache.hadoop.mapreduce.OutputFormat as NewOutputFormat /** * 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. + * functions. This method is here for backward compatibility. It does not provide combiner + * classtag information to the shuffle. * - * 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). - * - * @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]). + * @see `combineByKeyWithClassTag` */ fun JavaRDD>.combineByKey( createCombiner: (V) -> C, @@ -44,25 +30,161 @@ fun JavaRDD>.combineByKey( partitioner: Partitioner, mapSideCombine: Boolean = true, serializer: Serializer? = null, -): JavaRDD> = - toPairRDD() - .combineByKey(createCombiner, mergeValue, mergeCombiners, partitioner, mapSideCombine, serializer) - .toTupleRDD() - +): JavaRDD> = toPairRDD() + .combineByKey(createCombiner, mergeValue, mergeCombiners, partitioner, mapSideCombine, serializer) + .toTupleRDD() /** - * Simplified version of combineByKey that hash-partitions the output RDD and uses map-side - * aggregation. + * 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. + * + * @see `combineByKeyWithClassTag` */ fun JavaRDD>.combineByKey( createCombiner: (V) -> C, mergeValue: (C, V) -> C, mergeCombiners: (C, C) -> C, - numPartitions: Int = rdd().sparkContext().defaultParallelism(), -): JavaRDD> = - toPairRDD() - .combineByKey(createCombiner, mergeValue, mergeCombiners, numPartitions) - .toTupleRDD() + numPartitions: Int, +): JavaRDD> = toPairRDD() + .combineByKey(createCombiner, mergeValue, mergeCombiners, numPartitions) + .toTupleRDD() + + +/** + * 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. + */ +fun JavaRDD>.aggregateByKey( + zeroValue: U, + partitioner: Partitioner, + seqOp: (U, V) -> U, + combOp: (U, U) -> U, +): JavaRDD> = toPairRDD() + .aggregateByKey(zeroValue, partitioner, seqOp, combOp) + .toTupleRDD() + +/** + * 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. + */ +fun JavaRDD>.aggregateByKey( + zeroValue: U, + numPartitions: Int, + seqOp: (U, V) -> U, + combOp: (U, U) -> U, +): JavaRDD> = toPairRDD() + .aggregateByKey(zeroValue, numPartitions, seqOp, combOp) + .toTupleRDD() + +/** + * 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. + */ +fun JavaRDD>.aggregateByKey( + zeroValue: U, + seqOp: (U, V) -> U, + combOp: (U, U) -> U, +): JavaRDD> = toPairRDD() + .aggregateByKey(zeroValue, seqOp, combOp) + .toTupleRDD() + +/** + * 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.). + */ +fun JavaRDD>.foldByKey( + zeroValue: V, + partitioner: Partitioner, + func: (V, V) -> V, +): JavaRDD> = toPairRDD() + .foldByKey(zeroValue, partitioner, func) + .toTupleRDD() + +/** + * 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.). + */ +fun JavaRDD>.foldByKey( + zeroValue: V, + numPartitions: Int, + func: (V, V) -> V, +): JavaRDD> = toPairRDD() + .foldByKey(zeroValue, numPartitions, func) + .toTupleRDD() + +/** + * 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.). + */ +fun JavaRDD>.foldByKey( + zeroValue: V, + func: (V, V) -> V, +): JavaRDD> = toPairRDD() + .foldByKey(zeroValue, func) + .toTupleRDD() + +/** + * 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. + * + * @param withReplacement whether to sample with or without replacement + * @param fractions map of specific keys to sampling rates + * @param seed seed for the random number generator + * @return RDD containing the sampled subset + */ +fun JavaRDD>.sampleByKey( + withReplacement: Boolean, + fractions: Map, + seed: Long = Random.nextLong(), +): JavaRDD> = toPairRDD() + .sampleByKey(withReplacement, fractions, seed) + .toTupleRDD() + +/** + * 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. + * + * @param withReplacement whether to sample with or without replacement + * @param fractions map of specific keys to sampling rates + * @param seed seed for the random number generator + * @return RDD containing the sampled subset + */ +fun JavaRDD>.sampleByKeyExact( + withReplacement: Boolean, + fractions: Map, + seed: Long = Random.nextLong() +): JavaRDD> = toPairRDD() + .sampleByKeyExact(withReplacement, fractions, seed) + .toTupleRDD() /** * Merge the values for each key using an associative and commutative reduce function. This will @@ -70,10 +192,703 @@ fun JavaRDD>.combineByKey( * to a "combiner" in MapReduce. */ fun JavaRDD>.reduceByKey( - partitioner: Partitioner = Partitioner.defaultPartitioner(rdd(), emptyImmutableSeq()), + partitioner: Partitioner, func: (V, V) -> V, -): JavaRDD> = +): JavaRDD> = toPairRDD() + .reduceByKey(partitioner, func) + .toTupleRDD() + +/** + * 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. + */ +fun JavaRDD>.reduceByKey( + func: (V, V) -> V, + numPartitions: Int, +): JavaRDD> = toPairRDD() + .reduceByKey(func, numPartitions) + .toTupleRDD() + +/** + * 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. + */ +fun JavaRDD>.reduceByKey( + func: (V, V) -> V, +): JavaRDD> = toPairRDD() + .reduceByKey(func) + .toTupleRDD() + +/** + * 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. + */ +fun JavaRDD>.reduceByKeyLocally( + func: (V, V) -> V, +): Map = toPairRDD() + .reduceByKeyLocally(func) + +/** + * Count the number of elements for each key, collecting the results to a local Map. + * + * @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 instead of a map. + */ +fun JavaRDD>.countByKey(): Map = + toPairRDD() + .countByKey() + +/** + * 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. + * + * @param timeout maximum time to wait for the job, in milliseconds + * @param confidence the desired statistical confidence in the result + * @return a potentially incomplete result, with error bounds + */ +fun JavaRDD>.countByKeyApprox( + timeout: Long, + confidence: Double = 0.95, +): PartialResult> = toPairRDD() + .countByKeyApprox(timeout, confidence) + +/** + * Group the values for each key in the RDD into a single sequence. Allows controlling the + * partitioning of the resulting key-value pair RDD by passing a Partitioner. + * The ordering of elements within each group is not guaranteed, and may even differ + * each time the resulting RDD is evaluated. + * + * @note This operation may be very expensive. If you are grouping in order to perform an + * aggregation (such as a sum or average) over each key, using `PairRDDFunctions.aggregateByKey` + * or `PairRDDFunctions.reduceByKey` will provide much better performance. + * + * @note 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`. + */ +fun JavaRDD>.groupByKey( + partitioner: Partitioner, +): JavaRDD>> = toPairRDD() + .groupByKey(partitioner) + .toTupleRDD() + +/** + * Group the values for each key in the RDD into a single sequence. Hash-partitions the + * resulting RDD with into `numPartitions` partitions. The ordering of elements within + * each group is not guaranteed, and may even differ each time the resulting RDD is evaluated. + * + * @note This operation may be very expensive. If you are grouping in order to perform an + * aggregation (such as a sum or average) over each key, using `PairRDDFunctions.aggregateByKey` + * or `PairRDDFunctions.reduceByKey` will provide much better performance. + * + * @note 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`. + */ +fun JavaRDD>.groupByKey( + numPartitions: Int, +): JavaRDD>> = toPairRDD() + .groupByKey(numPartitions) + .toTupleRDD() + +/** + * Return a copy of the RDD partitioned using the specified partitioner. + */ +fun JavaRDD>.partitionBy( + partitioner: Partitioner, +): JavaRDD> = toPairRDD() + .partitionBy(partitioner) + .toTupleRDD() + +/** + * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each + * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and + * (k, v2) is in `other`. Uses the given Partitioner to partition the output RDD. + */ +fun JavaRDD>.join( + other: JavaRDD>, + partitioner: Partitioner, +): JavaRDD>> = toPairRDD() + .join(other.toPairRDD(), partitioner) + .toTupleRDD() + +/** + * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the + * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the + * pair (k, (v, None)) if no elements in `other` have key k. Uses the given Partitioner to + * partition the output RDD. + */ +fun JavaRDD>.leftOuterJoin( + other: JavaRDD>, + partitioner: Partitioner +): JavaRDD>>> = toPairRDD() + .leftOuterJoin(other.toPairRDD(), partitioner) + .toTupleRDD() + +/** + * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the + * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the + * pair (k, (None, w)) if no elements in `this` have key k. Uses the given Partitioner to + * partition the output RDD. + */ +fun JavaRDD>.rightOuterJoin( + other: JavaRDD>, + partitioner: Partitioner, +): JavaRDD, W>>> = toPairRDD() + .rightOuterJoin(other.toPairRDD(), partitioner) + .toTupleRDD() + +/** + * Perform a full outer join of `this` and `other`. For each element (k, v) in `this`, the + * resulting RDD will either contain all pairs (k, (Some(v), Some(w))) for w in `other`, or + * the pair (k, (Some(v), None)) if no elements in `other` have key k. Similarly, for each + * element (k, w) in `other`, the resulting RDD will either contain all pairs + * (k, (Some(v), Some(w))) for v in `this`, or the pair (k, (None, Some(w))) if no elements + * in `this` have key k. Uses the given Partitioner to partition the output RDD. + */ +fun JavaRDD>.fullOuterJoin( + other: JavaRDD>, + partitioner: Partitioner, +): JavaRDD, Optional>>> = toPairRDD() + .fullOuterJoin(other.toPairRDD(), partitioner) + .toTupleRDD() + +/** + * 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. + * + * @see `combineByKeyWithClassTag` + */ +fun JavaRDD>.combineByKey( + createCombiner: (V) -> C, + mergeValue: (C, V) -> C, + mergeCombiners: (C, C) -> C +): JavaRDD> = toPairRDD() + .combineByKey(createCombiner, mergeValue, mergeCombiners) + .toTupleRDD() + + +/** + * Group the values for each key in the RDD into a single sequence. Hash-partitions the + * resulting RDD with the existing partitioner/parallelism level. The ordering of elements + * within each group is not guaranteed, and may even differ each time the resulting RDD is + * evaluated. + * + * @note This operation may be very expensive. If you are grouping in order to perform an + * aggregation (such as a sum or average) over each key, using `PairRDDFunctions.aggregateByKey` + * or `PairRDDFunctions.reduceByKey` will provide much better performance. + */ +fun JavaRDD>.groupByKey(): JavaRDD>> = + toPairRDD() + .groupByKey() + .toTupleRDD() + +/** + * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each + * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and + * (k, v2) is in `other`. Performs a hash join across the cluster. + */ +fun JavaRDD>.join(other: JavaRDD>): JavaRDD>> = + toPairRDD() + .join(other.toPairRDD()) + .toTupleRDD() + +/** + * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each + * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and + * (k, v2) is in `other`. Performs a hash join across the cluster. + */ +fun JavaRDD>.join( + other: JavaRDD>, + numPartitions: Int +): JavaRDD>> = + toPairRDD() + .join(other.toPairRDD(), numPartitions) + .toTupleRDD() + +/** + * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the + * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the + * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output + * using the existing partitioner/parallelism level. + */ +fun JavaRDD>.leftOuterJoin(other: JavaRDD>): JavaRDD>>> = + toPairRDD() + .leftOuterJoin(other.toPairRDD()) + .toTupleRDD() + +/** + * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the + * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the + * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output + * into `numPartitions` partitions. + */ +fun JavaRDD>.leftOuterJoin( + other: JavaRDD>, + numPartitions: Int +): JavaRDD>>> = toPairRDD() + .leftOuterJoin(other.toPairRDD(), numPartitions) + .toTupleRDD() + +/** + * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the + * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the + * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting + * RDD using the existing partitioner/parallelism level. + */ +fun JavaRDD>.rightOuterJoin(other: JavaRDD>): JavaRDD, W>>> = toPairRDD() - .reduceByKey(partitioner, func) + .rightOuterJoin(other.toPairRDD()) .toTupleRDD() +/** + * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the + * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the + * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting + * RDD into the given number of partitions. + */ +fun JavaRDD>.rightOuterJoin( + other: JavaRDD>, + numPartitions: Int, +): JavaRDD, W>>> = toPairRDD() + .rightOuterJoin(other.toPairRDD(), numPartitions) + .toTupleRDD() + +/** + * Perform a full outer join of `this` and `other`. For each element (k, v) in `this`, the + * resulting RDD will either contain all pairs (k, (Some(v), Some(w))) for w in `other`, or + * the pair (k, (Some(v), None)) if no elements in `other` have key k. Similarly, for each + * element (k, w) in `other`, the resulting RDD will either contain all pairs + * (k, (Some(v), Some(w))) for v in `this`, or the pair (k, (None, Some(w))) if no elements + * in `this` have key k. Hash-partitions the resulting RDD using the existing partitioner/ + * parallelism level. + */ +fun JavaRDD>.fullOuterJoin( + other: JavaRDD>, +): JavaRDD, Optional>>> = + toPairRDD() + .fullOuterJoin(other.toPairRDD()) + .toTupleRDD() + +/** + * Perform a full outer join of `this` and `other`. For each element (k, v) in `this`, the + * resulting RDD will either contain all pairs (k, (Some(v), Some(w))) for w in `other`, or + * the pair (k, (Some(v), None)) if no elements in `other` have key k. Similarly, for each + * element (k, w) in `other`, the resulting RDD will either contain all pairs + * (k, (Some(v), Some(w))) for v in `this`, or the pair (k, (None, Some(w))) if no elements + * in `this` have key k. Hash-partitions the resulting RDD into the given number of partitions. + */ +fun JavaRDD>.fullOuterJoin( + other: JavaRDD>, + numPartitions: Int, +): JavaRDD, Optional>>> = + toPairRDD() + .fullOuterJoin(other.toPairRDD(), numPartitions) + .toTupleRDD() + +/** + * 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) + * + * @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. + */ +fun JavaRDD>.collectAsMap(): Map = toPairRDD().collectAsMap() + +/** + * 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. + */ +fun JavaRDD>.mapValues(f: (V) -> U): JavaRDD> = + toPairRDD().mapValues(f).toTupleRDD() + +/** + * 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. + */ +fun JavaRDD>.flatMapValues(f: (V) -> Iterator): JavaRDD> = + toPairRDD().flatMapValues(f).toTupleRDD() + +/** + * For each key k in `this` or `other1` or `other2` or `other3`, + * return a resulting RDD that contains a tuple with the list of values + * for that key in `this`, `other1`, `other2` and `other3`. + */ +fun JavaRDD>.cogroup( + other1: JavaRDD>, + other2: JavaRDD>, + other3: JavaRDD>, + partitioner: Partitioner, +): JavaRDD, Iterable, Iterable, Iterable>>> = + toPairRDD().cogroup(other1.toPairRDD(), other2.toPairRDD(), other3.toPairRDD(), partitioner).toTupleRDD() + +/** + * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the + * list of values for that key in `this` as well as `other`. + */ +fun JavaRDD>.cogroup( + other: JavaRDD>, + partitioner: Partitioner, +): JavaRDD, Iterable>>> = toPairRDD() + .cogroup(other.toPairRDD(), partitioner) + .toTupleRDD() + +/** + * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a + * tuple with the list of values for that key in `this`, `other1` and `other2`. + */ +fun JavaRDD>.cogroup( + other1: JavaRDD>, + other2: JavaRDD>, + partitioner: Partitioner, +): JavaRDD, Iterable, Iterable>>> = + toPairRDD() + .cogroup(other1.toPairRDD(), other2.toPairRDD(), partitioner) + .toTupleRDD() + +/** + * For each key k in `this` or `other1` or `other2` or `other3`, + * return a resulting RDD that contains a tuple with the list of values + * for that key in `this`, `other1`, `other2` and `other3`. + */ +fun JavaRDD>.cogroup( + other1: JavaRDD>, + other2: JavaRDD>, + other3: JavaRDD>, +): JavaRDD, Iterable, Iterable, Iterable>>> = + toPairRDD() + .cogroup(other1.toPairRDD(), other2.toPairRDD(), other3.toPairRDD()) + .toTupleRDD() + +/** + * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the + * list of values for that key in `this` as well as `other`. + */ +fun JavaRDD>.cogroup( + other: JavaRDD>, +): JavaRDD, Iterable>>> = + toPairRDD().cogroup(other.toPairRDD()).toTupleRDD() + +/** + * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a + * tuple with the list of values for that key in `this`, `other1` and `other2`. + */ +fun JavaRDD>.cogroup( + other1: JavaRDD>, + other2: JavaRDD>, +): JavaRDD, Iterable, Iterable>>> = + toPairRDD() + .cogroup(other1.toPairRDD(), other2.toPairRDD()) + .toTupleRDD() + +/** + * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the + * list of values for that key in `this` as well as `other`. + */ +fun JavaRDD>.cogroup( + other: JavaRDD>, + numPartitions: Int +): JavaRDD, Iterable>>> = + toPairRDD().cogroup(other.toPairRDD(), numPartitions).toTupleRDD() + +/** + * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a + * tuple with the list of values for that key in `this`, `other1` and `other2`. + */ +fun JavaRDD>.cogroup( + other1: JavaRDD>, + other2: JavaRDD>, + numPartitions: Int, +): JavaRDD, Iterable, Iterable>>> = + toPairRDD() + .cogroup(other1.toPairRDD(), other2.toPairRDD(), numPartitions) + .toTupleRDD() + +/** + * For each key k in `this` or `other1` or `other2` or `other3`, + * return a resulting RDD that contains a tuple with the list of values + * for that key in `this`, `other1`, `other2` and `other3`. + */ +fun JavaRDD>.cogroup( + other1: JavaRDD>, + other2: JavaRDD>, + other3: JavaRDD>, + numPartitions: Int, +): JavaRDD, Iterable, Iterable, Iterable>>> = + toPairRDD() + .cogroup(other1.toPairRDD(), other2.toPairRDD(), other3.toPairRDD(), numPartitions) + .toTupleRDD() + + +/** Alias for cogroup. */ +fun JavaRDD>.groupWith( + other: JavaRDD>, +): JavaRDD, Iterable>>> = + toPairRDD().groupWith(other.toPairRDD()).toTupleRDD() + +/** Alias for cogroup. */ +fun JavaRDD>.groupWith( + other1: JavaRDD>, + other2: JavaRDD>, +): JavaRDD, Iterable, Iterable>>> = + toPairRDD().groupWith(other1.toPairRDD(), other2.toPairRDD()).toTupleRDD() + +/** Alias for cogroup. */ +fun JavaRDD>.groupWith( + other1: JavaRDD>, + other2: JavaRDD>, + other3: JavaRDD>, +): JavaRDD, Iterable, Iterable, Iterable>>> = + toPairRDD().groupWith(other1.toPairRDD(), other2.toPairRDD(), other3.toPairRDD()).toTupleRDD() + +/** + * Return an RDD with the pairs from `this` whose keys are not in `other`. + * + * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting + * RDD will be less than or equal to us. + */ +fun JavaRDD>.subtractByKey(other: JavaRDD>): JavaRDD> = + toPairRDD().subtractByKey(other.toPairRDD()).toTupleRDD() + +/** + * Return an RDD with the pairs from `this` whose keys are not in `other`. + */ +fun JavaRDD>.subtractByKey( + other: JavaRDD>, + numPartitions: Int, +): JavaRDD> = toPairRDD() + .subtractByKey(other.toPairRDD(), numPartitions) + .toTupleRDD() + +/** + * Return an RDD with the pairs from `this` whose keys are not in `other`. + */ +fun JavaRDD>.subtractByKey( + other: JavaRDD>, + p: Partitioner, +): JavaRDD> = toPairRDD() + .subtractByKey(other.toPairRDD(), p) + .toTupleRDD() + +/** + * Return the list of values in the RDD for key `key`. This operation is done efficiently if the + * RDD has a known partitioner by only searching the partition that the key maps to. + */ +fun JavaRDD>.lookup(key: K): List = toPairRDD().lookup(key) + +/** Output the RDD to any Hadoop-supported file system. */ +fun > JavaRDD>.saveAsHadoopFile( + path: String, + keyClass: Class<*>, + valueClass: Class<*>, + outputFormatClass: Class, + conf: JobConf, +): Unit = toPairRDD().saveAsHadoopFile(path, keyClass, valueClass, outputFormatClass, conf) + +/** Output the RDD to any Hadoop-supported file system. */ +fun > JavaRDD>.saveAsHadoopFile( + path: String, + keyClass: Class<*>, + valueClass: Class<*>, + outputFormatClass: Class, +): Unit = toPairRDD().saveAsHadoopFile(path, keyClass, valueClass, outputFormatClass) + +/** Output the RDD to any Hadoop-supported file system, compressing with the supplied codec. */ +fun > JavaRDD>.saveAsHadoopFile( + path: String, + keyClass: Class<*>, + valueClass: Class<*>, + outputFormatClass: Class, + codec: Class, +): Unit = toPairRDD().saveAsHadoopFile(path, keyClass, valueClass, outputFormatClass, codec) + +/** Output the RDD to any Hadoop-supported file system. */ +fun > JavaRDD>.saveAsNewAPIHadoopFile( + path: String, + keyClass: Class<*>, + valueClass: Class<*>, + outputFormatClass: Class, + conf: Configuration, +): Unit = toPairRDD().saveAsNewAPIHadoopFile(path, keyClass, valueClass, outputFormatClass, conf) + +/** + * Output the RDD to any Hadoop-supported storage system, using + * a Configuration object for that storage system. + */ +fun JavaRDD>.saveAsNewAPIHadoopDataset(conf: Configuration): Unit = + toPairRDD().saveAsNewAPIHadoopDataset(conf) + +/** Output the RDD to any Hadoop-supported file system. */ +fun > JavaRDD>.saveAsNewAPIHadoopFile( + path: String, + keyClass: Class<*>, + valueClass: Class<*>, + outputFormatClass: Class, +): Unit = toPairRDD().saveAsNewAPIHadoopFile(path, keyClass, valueClass, outputFormatClass) + +/** + * 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. + */ +fun JavaRDD>.saveAsHadoopDataset(conf: JobConf): Unit = + toPairRDD().saveAsHadoopDataset(conf) + +/** + * 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. + */ +fun JavaRDD>.repartitionAndSortWithinPartitions(partitioner: Partitioner): JavaRDD> = + toPairRDD().repartitionAndSortWithinPartitions(partitioner).toTupleRDD() + +/** + * 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. + */ +fun JavaRDD>.repartitionAndSortWithinPartitions( + partitioner: Partitioner, + comp: Comparator, +): JavaRDD> = toPairRDD().repartitionAndSortWithinPartitions(partitioner, comp).toTupleRDD() + +/** + * Sort the RDD by key, so that each partition contains a sorted range of the elements. Calling + * `collect` or `save` on the resulting RDD will return or output an ordered list of records + * (in the `save` case, they will be written to multiple `part-X` files in the filesystem, in + * order of the keys). + */ +fun JavaRDD>.sortByKey(ascending: Boolean = true): JavaRDD> = + toPairRDD().sortByKey(ascending).toTupleRDD() + +/** + * Sort the RDD by key, so that each partition contains a sorted range of the elements. Calling + * `collect` or `save` on the resulting RDD will return or output an ordered list of records + * (in the `save` case, they will be written to multiple `part-X` files in the filesystem, in + * order of the keys). + */ +fun JavaRDD>.sortByKey(ascending: Boolean, numPartitions: Int): JavaRDD> = + toPairRDD().sortByKey(ascending, numPartitions).toTupleRDD() + +/** + * Sort the RDD by key, so that each partition contains a sorted range of the elements. Calling + * `collect` or `save` on the resulting RDD will return or output an ordered list of records + * (in the `save` case, they will be written to multiple `part-X` files in the filesystem, in + * order of the keys). + */ +fun JavaRDD>.sortByKey(comp: Comparator, ascending: Boolean = true): JavaRDD> = + toPairRDD().sortByKey(comp, ascending).toTupleRDD() + +/** + * Sort the RDD by key, so that each partition contains a sorted range of the elements. Calling + * `collect` or `save` on the resulting RDD will return or output an ordered list of records + * (in the `save` case, they will be written to multiple `part-X` files in the filesystem, in + * order of the keys). + */ +fun JavaRDD>.sortByKey( + comp: Comparator, + ascending: Boolean, + numPartitions: Int, +): JavaRDD> = toPairRDD().sortByKey(comp, ascending, numPartitions).toTupleRDD() + +//#if sparkMinor >= 3.1 +/** + * Return a RDD containing only the elements in the inclusive range `lower` to `upper`. + * If the RDD has been partitioned using a `RangePartitioner`, then this operation can be + * performed efficiently by only scanning the partitions that might contain matching elements. + * Otherwise, a standard `filter` is applied to all partitions. + * + * @since 3.1.0 + */ +fun JavaRDD>.filterByRange(lower: K, upper: K): JavaRDD> = + toPairRDD().filterByRange(lower, upper).toTupleRDD() + +/** + * Return a RDD containing only the elements in the inclusive range `lower` to `upper`. + * If the RDD has been partitioned using a `RangePartitioner`, then this operation can be + * performed efficiently by only scanning the partitions that might contain matching elements. + * Otherwise, a standard `filter` is applied to all partitions. + * + * @since 3.1.0 + */ +fun JavaRDD>.filterByRange( + comp: Comparator, + lower: K, + upper: K, +): JavaRDD> = toPairRDD() + .filterByRange(comp, lower, upper) + .toTupleRDD() +//#endif + +/** + * Return an RDD with the keys of each tuple. + */ +fun JavaRDD>.keys(): JavaRDD = toPairRDD().keys() + +/** + * Return an RDD with the values of each tuple. + */ +fun JavaRDD>.values(): JavaRDD = toPairRDD().values() + +/** + * 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. + * + * @param relativeSD Relative accuracy. Smaller values create counters that require more space. + * It must be greater than 0.000017. + * @param partitioner partitioner of the resulting RDD. + */ +fun JavaRDD>.countApproxDistinctByKey( + relativeSD: Double, + partitioner: Partitioner, +): JavaRDD> = toPairRDD().countApproxDistinctByKey(relativeSD, partitioner).toTupleRDD() + +/** + * 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. + * + * @param relativeSD Relative accuracy. Smaller values create counters that require more space. + * It must be greater than 0.000017. + * @param numPartitions number of partitions of the resulting RDD. + */ +fun JavaRDD>.countApproxDistinctByKey( + relativeSD: Double, + numPartitions: Int, +): JavaRDD> = toPairRDD().countApproxDistinctByKey(relativeSD, numPartitions).toTupleRDD() + +/** + * 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. + * + * @param relativeSD Relative accuracy. Smaller values create counters that require more space. + * It must be greater than 0.000017. + */ +fun JavaRDD>.countApproxDistinctByKey(relativeSD: Double): JavaRDD> = + toPairRDD().countApproxDistinctByKey(relativeSD).toTupleRDD() \ No newline at end of file diff --git a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Seq.kt b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Seq.kt deleted file mode 100644 index 14ac09e9..00000000 --- a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Seq.kt +++ /dev/null @@ -1,22 +0,0 @@ -package org.jetbrains.kotlinx.spark.api - - -fun emptySeq(): scala.collection.Seq = scala.collection.`Seq$`.`MODULE$`.empty() - -fun seqOf(vararg elements: T): scala.collection.Seq = - if (elements.isEmpty()) emptySeq() - else scala.collection.`Seq$`.`MODULE$`.from(elements.asIterable().asScalaIterable()) - -fun emptyImmutableSeq(): scala.collection.immutable.Seq = - scala.collection.immutable.`Seq$`.`MODULE$`.empty() - -fun immutableSeqOf(vararg elements: T): scala.collection.immutable.Seq = - if (elements.isEmpty()) emptyImmutableSeq() - else scala.collection.immutable.`Seq$`.`MODULE$`.from(elements.asIterable().asScalaIterable()) - -fun emptyMutableSeq(): scala.collection.mutable.Seq = - scala.collection.mutable.`Seq$`.`MODULE$`.empty() - -fun mutableSeqOf(vararg elements: T): scala.collection.mutable.Seq = - if (elements.isEmpty()) emptyMutableSeq() - else scala.collection.mutable.`Seq$`.`MODULE$`.from(elements.asIterable().asScalaIterable()) From 66fd0e4bb64cf975fcc59fef23d3e9e0196e6eab Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Wed, 13 Jul 2022 18:14:19 +0200 Subject: [PATCH 04/16] ported JavaDoubleRDD and added rdd helper creation functions --- .../org/jetbrains/kotlinx/spark/api/Rdd.kt | 15 +++ .../jetbrains/kotlinx/spark/api/RddDouble.kt | 106 ++++++++++++++++++ .../spark/api/{RDD.kt => RddKeyValue.kt} | 10 +- .../kotlinx/spark/api/SparkSession.kt | 15 +++ .../kotlinx/spark/api/StreamingKeyValues.kt | 7 -- 5 files changed, 145 insertions(+), 8 deletions(-) create mode 100644 kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Rdd.kt create mode 100644 kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddDouble.kt rename kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/{RDD.kt => RddKeyValue.kt} (99%) diff --git a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Rdd.kt b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Rdd.kt new file mode 100644 index 00000000..0f82a69a --- /dev/null +++ b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Rdd.kt @@ -0,0 +1,15 @@ +package org.jetbrains.kotlinx.spark.api + +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.api.java.JavaSparkContext + + +fun JavaSparkContext.rddOf( + vararg elements: T, + numSlices: Int = defaultParallelism(), +): JavaRDD = parallelize(elements.toList(), numSlices) + +fun JavaSparkContext.toRDD( + elements: List, + numSlices: Int = defaultParallelism(), +): JavaRDD = parallelize(elements, numSlices) diff --git a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddDouble.kt b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddDouble.kt new file mode 100644 index 00000000..83489891 --- /dev/null +++ b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddDouble.kt @@ -0,0 +1,106 @@ +package org.jetbrains.kotlinx.spark.api + +import org.apache.spark.api.java.JavaDoubleRDD +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.partial.BoundedDouble +import org.apache.spark.partial.PartialResult +import org.apache.spark.rdd.RDD +import org.apache.spark.util.StatCounter +import scala.Tuple2 + +@Suppress("UNCHECKED_CAST") +fun JavaRDD.toJavaDoubleRDD(): JavaDoubleRDD = + JavaDoubleRDD.fromRDD(rdd() as RDD) + +@Suppress("UNCHECKED_CAST") +fun JavaDoubleRDD.toDoubleRDD(): JavaRDD = + JavaDoubleRDD.toRDD(this).toJavaRDD() as JavaRDD + +/** Add up the elements in this RDD. */ +fun JavaRDD.sum(): Double = toJavaDoubleRDD().sum() + +/** + * Return a [[org.apache.spark.util.StatCounter]] object that captures the mean, variance and + * count of the RDD's elements in one operation. + */ +fun JavaRDD.stats(): StatCounter = toJavaDoubleRDD().stats() + +/** Compute the mean of this RDD's elements. */ +fun JavaRDD.mean(): Double = toJavaDoubleRDD().mean() + +/** Compute the population variance of this RDD's elements. */ +fun JavaRDD.variance(): Double = toJavaDoubleRDD().variance() + +/** Compute the population standard deviation of this RDD's elements. */ +fun JavaRDD.stdev(): Double = toJavaDoubleRDD().stdev() + +/** + * Compute the sample standard deviation of this RDD's elements (which corrects for bias in + * estimating the standard deviation by dividing by N-1 instead of N). + */ +fun JavaRDD.sampleStdev(): Double = toJavaDoubleRDD().sampleStdev() + +/** + * Compute the sample variance of this RDD's elements (which corrects for bias in + * estimating the variance by dividing by N-1 instead of N). + */ +fun JavaRDD.sampleVariance(): Double = toJavaDoubleRDD().sampleVariance() + +/** + * Compute the population standard deviation of this RDD's elements. + */ +fun JavaRDD.popStdev(): Double = toJavaDoubleRDD().popStdev() + +/** + * Compute the population variance of this RDD's elements. + */ +fun JavaRDD.popVariance(): Double = toJavaDoubleRDD().popVariance() + +/** + * Approximate operation to return the mean within a timeout. + */ +fun JavaRDD.meanApprox( + timeout: Long, + confidence: Double = 0.95, +): PartialResult = toJavaDoubleRDD().meanApprox(timeout, confidence) + +/** + * Approximate operation to return the sum within a timeout. + */ +fun JavaRDD.sumApprox( + timeout: Long, + confidence: Double = 0.95, +): PartialResult = toJavaDoubleRDD().sumApprox(timeout, confidence) + +/** + * Compute a histogram of the data using bucketCount number of buckets evenly + * spaced between the minimum and maximum of the RDD. For example if the min + * value is 0 and the max is 100 and there are two buckets the resulting + * buckets will be [0, 50) [50, 100]. bucketCount must be at least 1 + * If the RDD contains infinity, NaN throws an exception + * If the elements in RDD do not vary (max == min) always returns a single bucket. + */ +fun JavaRDD.histogram(bucketCount: Int): Tuple2 = + toJavaDoubleRDD().histogram(bucketCount) + +/** + * Compute a histogram using the provided buckets. The buckets are all open + * to the right except for the last which is closed. + * e.g. for the array + * [1, 10, 20, 50] the buckets are [1, 10) [10, 20) [20, 50] + * e.g {@code <=x<10, 10<=x<20, 20<=x<=50} + * And on the input of 1 and 50 we would have a histogram of 1, 0, 1 + * + * @note If your histogram is evenly spaced (e.g. [0, 10, 20, 30]) this can be switched + * from an O(log n) insertion to O(1) per element. (where n = # buckets) if you set evenBuckets + * to true. + * buckets must be sorted and not contain any duplicates. + * buckets array must be at least two elements + * All NaN entries are treated the same. If you have a NaN bucket it must be + * the maximum value of the last position and all NaN entries will be counted + * in that bucket. + */ +fun JavaRDD.histogram( + buckets: Array, + evenBuckets: Boolean = false, +): LongArray = toJavaDoubleRDD().histogram(buckets, evenBuckets) \ No newline at end of file diff --git a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RDD.kt b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddKeyValue.kt similarity index 99% rename from kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RDD.kt rename to kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddKeyValue.kt index e33e9077..5e6584e3 100644 --- a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RDD.kt +++ b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddKeyValue.kt @@ -5,7 +5,9 @@ import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapred.OutputFormat import org.apache.spark.Partitioner +import org.apache.spark.api.java.JavaPairRDD import org.apache.spark.api.java.JavaRDD +import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.api.java.Optional import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.PartialResult @@ -16,6 +18,12 @@ import scala.Tuple4 import kotlin.random.Random import org.apache.hadoop.mapreduce.OutputFormat as NewOutputFormat +fun JavaRDD>.toPairRDD(): JavaPairRDD = + JavaPairRDD.fromJavaRDD(this) + +fun JavaPairRDD.toTupleRDD(): JavaRDD> = + JavaPairRDD.toRDD(this).toJavaRDD() + /** * 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 @@ -745,7 +753,7 @@ fun > JavaRDD>.saveAsNewAPIHadoopFi * (e.g. a table name to write to) in the same way as it would be configured for a Hadoop * MapReduce job. */ -fun JavaRDD>.saveAsHadoopDataset(conf: JobConf): Unit = +fun JavaRDD>.saveAsHadoopDataset(conf: JobConf): Unit = toPairRDD().saveAsHadoopDataset(conf) /** diff --git a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt index 27a7294c..95cc4a09 100644 --- a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt +++ b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt @@ -30,6 +30,7 @@ package org.jetbrains.kotlinx.spark.api import org.apache.hadoop.conf.Configuration import org.apache.spark.SparkConf import org.apache.spark.SparkContext +import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.java.JavaRDDLike import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.broadcast.Broadcast @@ -101,6 +102,20 @@ class KSparkSession(val spark: SparkSession) { */ inline fun JavaRDDLike.toDF(vararg colNames: String): Dataset = toDF(spark, *colNames) + /** + * Utility method to create an RDD from a list. + * NOTE: [T] must be [Serializable]. + */ + inline fun List.toRDD(numSlices: Int = sc.defaultParallelism()): JavaRDD = + sc.toRDD(this, numSlices) + + /** + * Utility method to create an RDD from a list. + * NOTE: [T] must be [Serializable]. + */ + inline fun rddOf(vararg elements: T, numSlices: Int = sc.defaultParallelism()): JavaRDD = + sc.toRDD(elements.toList(), numSlices) + /** * A collection of methods for registering user-defined functions (UDF). * diff --git a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/StreamingKeyValues.kt b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/StreamingKeyValues.kt index 692bb4c5..ebb431ca 100644 --- a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/StreamingKeyValues.kt +++ b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/StreamingKeyValues.kt @@ -40,13 +40,6 @@ fun JavaDStream>.toPairDStream(): JavaPairDStream = fun JavaPairDStream.toTupleDStream(): JavaDStream> = toJavaDStream() -fun JavaRDD>.toPairRDD(): JavaPairRDD = - JavaPairRDD.fromJavaRDD(this) - -fun JavaPairRDD.toTupleRDD(): JavaRDD> = - JavaPairRDD.toRDD(this).toJavaRDD() - - /** * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to * generate the RDDs with `numPartitions` partitions. From c43bd81bbe43eafd6a403afc292eed9f73d57681 Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Thu, 14 Jul 2022 17:04:25 +0200 Subject: [PATCH 05/16] fixing docs --- .../jetbrains/kotlinx/spark/api/RddDouble.kt | 10 +- .../kotlinx/spark/api/RddKeyValue.kt | 246 +++++++++--------- .../kotlinx/spark/api/StreamingKeyValues.kt | 1 - settings.gradle.kts | 2 +- 4 files changed, 127 insertions(+), 132 deletions(-) diff --git a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddDouble.kt b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddDouble.kt index 83489891..bdd02236 100644 --- a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddDouble.kt +++ b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddDouble.kt @@ -20,7 +20,7 @@ fun JavaDoubleRDD.toDoubleRDD(): JavaRDD = fun JavaRDD.sum(): Double = toJavaDoubleRDD().sum() /** - * Return a [[org.apache.spark.util.StatCounter]] object that captures the mean, variance and + * Return a [org.apache.spark.util.StatCounter] object that captures the mean, variance and * count of the RDD's elements in one operation. */ fun JavaRDD.stats(): StatCounter = toJavaDoubleRDD().stats() @@ -76,7 +76,7 @@ fun JavaRDD.sumApprox( * Compute a histogram of the data using bucketCount number of buckets evenly * spaced between the minimum and maximum of the RDD. For example if the min * value is 0 and the max is 100 and there are two buckets the resulting - * buckets will be [0, 50) [50, 100]. bucketCount must be at least 1 + * buckets will be `[0, 50)` `[50, 100]`. bucketCount must be at least 1 * If the RDD contains infinity, NaN throws an exception * If the elements in RDD do not vary (max == min) always returns a single bucket. */ @@ -87,11 +87,11 @@ fun JavaRDD.histogram(bucketCount: Int): Tuple2 * Compute a histogram using the provided buckets. The buckets are all open * to the right except for the last which is closed. * e.g. for the array - * [1, 10, 20, 50] the buckets are [1, 10) [10, 20) [20, 50] - * e.g {@code <=x<10, 10<=x<20, 20<=x<=50} + * `[1, 10, 20, 50]` the buckets are `[1, 10) [10, 20) [20, 50]` + * e.g. ` <=x<10, 10<=x<20, 20<=x<=50` * And on the input of 1 and 50 we would have a histogram of 1, 0, 1 * - * @note If your histogram is evenly spaced (e.g. [0, 10, 20, 30]) this can be switched + * Note: If your histogram is evenly spaced (e.g. `[0, 10, 20, 30]`) this can be switched * from an O(log n) insertion to O(1) per element. (where n = # buckets) if you set evenBuckets * to true. * buckets must be sorted and not contain any duplicates. diff --git a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddKeyValue.kt b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddKeyValue.kt index 5e6584e3..f43cf7a8 100644 --- a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddKeyValue.kt +++ b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddKeyValue.kt @@ -1,3 +1,5 @@ +@file:Suppress("unused") + package org.jetbrains.kotlinx.spark.api import org.apache.hadoop.conf.Configuration @@ -5,9 +7,9 @@ import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapred.OutputFormat import org.apache.spark.Partitioner +import org.apache.spark.RangePartitioner import org.apache.spark.api.java.JavaPairRDD import org.apache.spark.api.java.JavaRDD -import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.api.java.Optional import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.PartialResult @@ -28,8 +30,6 @@ fun JavaPairRDD.toTupleRDD(): JavaRDD> = * 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. - * - * @see `combineByKeyWithClassTag` */ fun JavaRDD>.combineByKey( createCombiner: (V) -> C, @@ -46,8 +46,6 @@ fun JavaRDD>.combineByKey( * 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. - * - * @see `combineByKeyWithClassTag` */ fun JavaRDD>.combineByKey( createCombiner: (V) -> C, @@ -61,12 +59,12 @@ fun JavaRDD>.combineByKey( /** * 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, + * 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. + * instead of creating a new [U]. */ fun JavaRDD>.aggregateByKey( zeroValue: U, @@ -79,12 +77,12 @@ fun JavaRDD>.aggregateByKey( /** * 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, + * 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. + * instead of creating a new [U]. */ fun JavaRDD>.aggregateByKey( zeroValue: U, @@ -97,12 +95,12 @@ fun JavaRDD>.aggregateByKey( /** * 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, + * 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. + * instead of creating a new [U]. */ fun JavaRDD>.aggregateByKey( zeroValue: U, @@ -115,7 +113,7 @@ fun JavaRDD>.aggregateByKey( /** * 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.). + * (e.g., [emptyList] for list concatenation, 0 for addition, or 1 for multiplication.). */ fun JavaRDD>.foldByKey( zeroValue: V, @@ -128,7 +126,7 @@ fun JavaRDD>.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.). + * (e.g., [emptyList] for list concatenation, 0 for addition, or 1 for multiplication.). */ fun JavaRDD>.foldByKey( zeroValue: V, @@ -141,7 +139,7 @@ fun JavaRDD>.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.). + * (e.g., [emptyList] for list concatenation, 0 for addition, or 1 for multiplication.). */ fun JavaRDD>.foldByKey( zeroValue: V, @@ -154,7 +152,7 @@ fun JavaRDD>.foldByKey( * 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 + * [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. * @@ -175,7 +173,7 @@ fun JavaRDD>.sampleByKey( * 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 + * 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 @@ -243,10 +241,10 @@ fun JavaRDD>.reduceByKeyLocally( /** * Count the number of elements for each key, collecting the results to a local Map. * - * @note This method should only be used if the resulting map is expected to be small, as + * 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 instead of a map. + * To handle very large results, consider using `rdd.mapValues { 1L }.reduceByKey(Long::plus)`, which + * returns an [RDD] instead of a map. */ fun JavaRDD>.countByKey(): Map = toPairRDD() @@ -278,12 +276,12 @@ fun JavaRDD>.countByKeyApprox( * The ordering of elements within each group is not guaranteed, and may even differ * each time the resulting RDD is evaluated. * - * @note This operation may be very expensive. If you are grouping in order to perform an - * aggregation (such as a sum or average) over each key, using `PairRDDFunctions.aggregateByKey` - * or `PairRDDFunctions.reduceByKey` will provide much better performance. + * 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 [aggregateByKey] + * or [reduceByKey] will provide much better performance. * - * @note 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`. + * Note: 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]. */ fun JavaRDD>.groupByKey( partitioner: Partitioner, @@ -293,15 +291,15 @@ fun JavaRDD>.groupByKey( /** * Group the values for each key in the RDD into a single sequence. Hash-partitions the - * resulting RDD with into `numPartitions` partitions. The ordering of elements within + * resulting RDD with into [numPartitions] partitions. The ordering of elements within * each group is not guaranteed, and may even differ each time the resulting RDD is evaluated. * - * @note This operation may be very expensive. If you are grouping in order to perform an - * aggregation (such as a sum or average) over each key, using `PairRDDFunctions.aggregateByKey` - * or `PairRDDFunctions.reduceByKey` will provide much better performance. + * 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 [aggregateByKey] + * or [reduceByKey] will provide much better performance. * - * @note 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`. + * Note: 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]. */ fun JavaRDD>.groupByKey( numPartitions: Int, @@ -319,9 +317,9 @@ fun JavaRDD>.partitionBy( .toTupleRDD() /** - * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each - * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and - * (k, v2) is in `other`. Uses the given Partitioner to partition the output RDD. + * Return an RDD containing all pairs of elements with matching keys in [this] and [other]. Each + * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in [this] and + * (k, v2) is in [other]. Uses the given Partitioner to partition the output RDD. */ fun JavaRDD>.join( other: JavaRDD>, @@ -331,9 +329,9 @@ fun JavaRDD>.join( .toTupleRDD() /** - * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the - * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the - * pair (k, (v, None)) if no elements in `other` have key k. Uses the given Partitioner to + * Perform a left outer join of [this] and [other]. For each element (k, v) in [this], the + * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in [other], or the + * pair (k, (v, None)) if no elements in [other] have key k. Uses the given Partitioner to * partition the output RDD. */ fun JavaRDD>.leftOuterJoin( @@ -344,9 +342,9 @@ fun JavaRDD>.leftOuterJoin( .toTupleRDD() /** - * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the - * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the - * pair (k, (None, w)) if no elements in `this` have key k. Uses the given Partitioner to + * Perform a right outer join of [this] and [other]. For each element (k, w) in [other], the + * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in [this], or the + * pair (k, (None, w)) if no elements in [this] have key k. Uses the given Partitioner to * partition the output RDD. */ fun JavaRDD>.rightOuterJoin( @@ -357,12 +355,12 @@ fun JavaRDD>.rightOuterJoin( .toTupleRDD() /** - * Perform a full outer join of `this` and `other`. For each element (k, v) in `this`, the - * resulting RDD will either contain all pairs (k, (Some(v), Some(w))) for w in `other`, or - * the pair (k, (Some(v), None)) if no elements in `other` have key k. Similarly, for each - * element (k, w) in `other`, the resulting RDD will either contain all pairs - * (k, (Some(v), Some(w))) for v in `this`, or the pair (k, (None, Some(w))) if no elements - * in `this` have key k. Uses the given Partitioner to partition the output RDD. + * Perform a full outer join of [this] and [other]. For each element (k, v) in [this], the + * resulting RDD will either contain all pairs (k, (Some(v), Some(w))) for w in [other], or + * the pair (k, (Some(v), None)) if no elements in [other] have key k. Similarly, for each + * element (k, w) in [other], the resulting RDD will either contain all pairs + * (k, (Some(v), Some(w))) for v in [this], or the pair (k, (None, Some(w))) if no elements + * in [this] have key k. Uses the given Partitioner to partition the output RDD. */ fun JavaRDD>.fullOuterJoin( other: JavaRDD>, @@ -375,8 +373,6 @@ fun JavaRDD>.fullOuterJoin( * 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. - * - * @see `combineByKeyWithClassTag` */ fun JavaRDD>.combineByKey( createCombiner: (V) -> C, @@ -393,9 +389,9 @@ fun JavaRDD>.combineByKey( * within each group is not guaranteed, and may even differ each time the resulting RDD is * evaluated. * - * @note This operation may be very expensive. If you are grouping in order to perform an - * aggregation (such as a sum or average) over each key, using `PairRDDFunctions.aggregateByKey` - * or `PairRDDFunctions.reduceByKey` will provide much better performance. + * 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 [aggregateByKey] + * or [reduceByKey] will provide much better performance. */ fun JavaRDD>.groupByKey(): JavaRDD>> = toPairRDD() @@ -403,9 +399,9 @@ fun JavaRDD>.groupByKey(): JavaRDD>> = .toTupleRDD() /** - * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each - * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and - * (k, v2) is in `other`. Performs a hash join across the cluster. + * Return an RDD containing all pairs of elements with matching keys in [this] and [other]. Each + * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in [this] and + * (k, v2) is in [other]. Performs a hash join across the cluster. */ fun JavaRDD>.join(other: JavaRDD>): JavaRDD>> = toPairRDD() @@ -413,9 +409,9 @@ fun JavaRDD>.join(other: JavaRDD>): JavaRDD< .toTupleRDD() /** - * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each - * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and - * (k, v2) is in `other`. Performs a hash join across the cluster. + * Return an RDD containing all pairs of elements with matching keys in [this] and [other]. Each + * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in [this] and + * (k, v2) is in [other]. Performs a hash join across the cluster. */ fun JavaRDD>.join( other: JavaRDD>, @@ -426,9 +422,9 @@ fun JavaRDD>.join( .toTupleRDD() /** - * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the - * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the - * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output + * Perform a left outer join of [this] and [other]. For each element (k, v) in [this], the + * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in [other], or the + * pair (k, (v, None)) if no elements in [other] have key k. Hash-partitions the output * using the existing partitioner/parallelism level. */ fun JavaRDD>.leftOuterJoin(other: JavaRDD>): JavaRDD>>> = @@ -437,10 +433,10 @@ fun JavaRDD>.leftOuterJoin(other: JavaRDD>): .toTupleRDD() /** - * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the - * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the - * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output - * into `numPartitions` partitions. + * Perform a left outer join of [this] and [other]. For each element (k, v) in [this], the + * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in [other], or the + * pair (k, (v, None)) if no elements in [other] have key k. Hash-partitions the output + * into [numPartitions] partitions. */ fun JavaRDD>.leftOuterJoin( other: JavaRDD>, @@ -450,9 +446,9 @@ fun JavaRDD>.leftOuterJoin( .toTupleRDD() /** - * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the - * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the - * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting + * Perform a right outer join of [this] and [other]. For each element (k, w) in [other], the + * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in [this], or the + * pair (k, (None, w)) if no elements in [this] have key k. Hash-partitions the resulting * RDD using the existing partitioner/parallelism level. */ fun JavaRDD>.rightOuterJoin(other: JavaRDD>): JavaRDD, W>>> = @@ -461,9 +457,9 @@ fun JavaRDD>.rightOuterJoin(other: JavaRDD>) .toTupleRDD() /** - * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the - * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the - * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting + * Perform a right outer join of [this] and [other]. For each element (k, w) in [other], the + * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in [this], or the + * pair (k, (None, w)) if no elements in [this] have key k. Hash-partitions the resulting * RDD into the given number of partitions. */ fun JavaRDD>.rightOuterJoin( @@ -474,12 +470,12 @@ fun JavaRDD>.rightOuterJoin( .toTupleRDD() /** - * Perform a full outer join of `this` and `other`. For each element (k, v) in `this`, the - * resulting RDD will either contain all pairs (k, (Some(v), Some(w))) for w in `other`, or - * the pair (k, (Some(v), None)) if no elements in `other` have key k. Similarly, for each - * element (k, w) in `other`, the resulting RDD will either contain all pairs - * (k, (Some(v), Some(w))) for v in `this`, or the pair (k, (None, Some(w))) if no elements - * in `this` have key k. Hash-partitions the resulting RDD using the existing partitioner/ + * Perform a full outer join of [this] and [other]. For each element (k, v) in [this], the + * resulting RDD will either contain all pairs (k, (Some(v), Some(w))) for w in [other], or + * the pair (k, (Some(v), None)) if no elements in [other] have key k. Similarly, for each + * element (k, w) in [other], the resulting RDD will either contain all pairs + * (k, (Some(v), Some(w))) for v in [this], or the pair (k, (None, Some(w))) if no elements + * in [this] have key k. Hash-partitions the resulting RDD using the existing partitioner/ * parallelism level. */ fun JavaRDD>.fullOuterJoin( @@ -490,12 +486,12 @@ fun JavaRDD>.fullOuterJoin( .toTupleRDD() /** - * Perform a full outer join of `this` and `other`. For each element (k, v) in `this`, the - * resulting RDD will either contain all pairs (k, (Some(v), Some(w))) for w in `other`, or - * the pair (k, (Some(v), None)) if no elements in `other` have key k. Similarly, for each - * element (k, w) in `other`, the resulting RDD will either contain all pairs - * (k, (Some(v), Some(w))) for v in `this`, or the pair (k, (None, Some(w))) if no elements - * in `this` have key k. Hash-partitions the resulting RDD into the given number of partitions. + * Perform a full outer join of [this] and [other]. For each element (k, v) in [this], the + * resulting RDD will either contain all pairs (k, (Some(v), Some(w))) for w in [other], or + * the pair (k, (Some(v), None)) if no elements in [other] have key k. Similarly, for each + * element (k, w) in [other], the resulting RDD will either contain all pairs + * (k, (Some(v), Some(w))) for v in [this], or the pair (k, (None, Some(w))) if no elements + * in [this] have key k. Hash-partitions the resulting RDD into the given number of partitions. */ fun JavaRDD>.fullOuterJoin( other: JavaRDD>, @@ -511,7 +507,7 @@ fun JavaRDD>.fullOuterJoin( * 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) * - * @note this method should only be used if the resulting data is expected to be small, as + * 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. */ fun JavaRDD>.collectAsMap(): Map = toPairRDD().collectAsMap() @@ -531,9 +527,9 @@ fun JavaRDD>.flatMapValues(f: (V) -> Iterator): JavaRD toPairRDD().flatMapValues(f).toTupleRDD() /** - * For each key k in `this` or `other1` or `other2` or `other3`, + * For each key k in [this] or [other1] or [other2] or [other3], * return a resulting RDD that contains a tuple with the list of values - * for that key in `this`, `other1`, `other2` and `other3`. + * for that key in [this], [other1], [other2] and [other3]. */ fun JavaRDD>.cogroup( other1: JavaRDD>, @@ -544,8 +540,8 @@ fun JavaRDD>.cogroup( toPairRDD().cogroup(other1.toPairRDD(), other2.toPairRDD(), other3.toPairRDD(), partitioner).toTupleRDD() /** - * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the - * list of values for that key in `this` as well as `other`. + * For each key k in [this] or [other], return a resulting RDD that contains a tuple with the + * list of values for that key in [this] as well as [other]. */ fun JavaRDD>.cogroup( other: JavaRDD>, @@ -555,8 +551,8 @@ fun JavaRDD>.cogroup( .toTupleRDD() /** - * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a - * tuple with the list of values for that key in `this`, `other1` and `other2`. + * For each key k in [this] or [other1] or [other2], return a resulting RDD that contains a + * tuple with the list of values for that key in [this], [other1] and [other2]. */ fun JavaRDD>.cogroup( other1: JavaRDD>, @@ -568,9 +564,9 @@ fun JavaRDD>.cogroup( .toTupleRDD() /** - * For each key k in `this` or `other1` or `other2` or `other3`, + * For each key k in [this] or [other1] or [other2] or [other3], * return a resulting RDD that contains a tuple with the list of values - * for that key in `this`, `other1`, `other2` and `other3`. + * for that key in [this], [other1], [other2] and [other3]. */ fun JavaRDD>.cogroup( other1: JavaRDD>, @@ -582,8 +578,8 @@ fun JavaRDD>.cogroup( .toTupleRDD() /** - * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the - * list of values for that key in `this` as well as `other`. + * For each key k in [this] or [other], return a resulting RDD that contains a tuple with the + * list of values for that key in [this] as well as [other]. */ fun JavaRDD>.cogroup( other: JavaRDD>, @@ -591,8 +587,8 @@ fun JavaRDD>.cogroup( toPairRDD().cogroup(other.toPairRDD()).toTupleRDD() /** - * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a - * tuple with the list of values for that key in `this`, `other1` and `other2`. + * For each key k in [this] or [other1] or [other2], return a resulting RDD that contains a + * tuple with the list of values for that key in [this], [other1] and [other2]. */ fun JavaRDD>.cogroup( other1: JavaRDD>, @@ -603,8 +599,8 @@ fun JavaRDD>.cogroup( .toTupleRDD() /** - * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the - * list of values for that key in `this` as well as `other`. + * For each key k in [this] or [other], return a resulting RDD that contains a tuple with the + * list of values for that key in [this] as well as [other]. */ fun JavaRDD>.cogroup( other: JavaRDD>, @@ -613,8 +609,8 @@ fun JavaRDD>.cogroup( toPairRDD().cogroup(other.toPairRDD(), numPartitions).toTupleRDD() /** - * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a - * tuple with the list of values for that key in `this`, `other1` and `other2`. + * For each key k in [this] or [other1] or [other2], return a resulting RDD that contains a + * tuple with the list of values for that key in [this], [other1] and [other2]. */ fun JavaRDD>.cogroup( other1: JavaRDD>, @@ -626,9 +622,9 @@ fun JavaRDD>.cogroup( .toTupleRDD() /** - * For each key k in `this` or `other1` or `other2` or `other3`, + * For each key k in [this] or [other1] or [other2] or [other3], * return a resulting RDD that contains a tuple with the list of values - * for that key in `this`, `other1`, `other2` and `other3`. + * for that key in [this], [other1], [other2] and [other3]. */ fun JavaRDD>.cogroup( other1: JavaRDD>, @@ -641,20 +637,20 @@ fun JavaRDD>.cogroup( .toTupleRDD() -/** Alias for cogroup. */ +/** Alias for [cogroup]. */ fun JavaRDD>.groupWith( other: JavaRDD>, ): JavaRDD, Iterable>>> = toPairRDD().groupWith(other.toPairRDD()).toTupleRDD() -/** Alias for cogroup. */ +/** Alias for [cogroup]. */ fun JavaRDD>.groupWith( other1: JavaRDD>, other2: JavaRDD>, ): JavaRDD, Iterable, Iterable>>> = toPairRDD().groupWith(other1.toPairRDD(), other2.toPairRDD()).toTupleRDD() -/** Alias for cogroup. */ +/** Alias for [cogroup]. */ fun JavaRDD>.groupWith( other1: JavaRDD>, other2: JavaRDD>, @@ -663,16 +659,16 @@ fun JavaRDD>.groupWith( toPairRDD().groupWith(other1.toPairRDD(), other2.toPairRDD(), other3.toPairRDD()).toTupleRDD() /** - * Return an RDD with the pairs from `this` whose keys are not in `other`. + * Return an RDD with the pairs from [this] whose keys are not in [other]. * - * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting + * Uses [this] partitioner/partition size, because even if [other] is huge, the resulting * RDD will be less than or equal to us. */ fun JavaRDD>.subtractByKey(other: JavaRDD>): JavaRDD> = toPairRDD().subtractByKey(other.toPairRDD()).toTupleRDD() /** - * Return an RDD with the pairs from `this` whose keys are not in `other`. + * Return an RDD with the pairs from [this] whose keys are not in [other]. */ fun JavaRDD>.subtractByKey( other: JavaRDD>, @@ -682,7 +678,7 @@ fun JavaRDD>.subtractByKey( .toTupleRDD() /** - * Return an RDD with the pairs from `this` whose keys are not in `other`. + * Return an RDD with the pairs from [this] whose keys are not in [other]. */ fun JavaRDD>.subtractByKey( other: JavaRDD>, @@ -692,7 +688,7 @@ fun JavaRDD>.subtractByKey( .toTupleRDD() /** - * Return the list of values in the RDD for key `key`. This operation is done efficiently if the + * Return the list of values in the RDD for key [key]. This operation is done efficiently if the * RDD has a known partitioner by only searching the partition that the key maps to. */ fun JavaRDD>.lookup(key: K): List = toPairRDD().lookup(key) @@ -760,7 +756,7 @@ fun JavaRDD>.saveAsHadoopDataset(conf: JobConf): Unit = * 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 + * This is more efficient than calling [JavaRDD.repartition] and then sorting within each partition * because it can push the sorting down into the shuffle machinery. */ fun JavaRDD>.repartitionAndSortWithinPartitions(partitioner: Partitioner): JavaRDD> = @@ -770,7 +766,7 @@ fun JavaRDD>.repartitionAndSortWithinPartitions(partitioner: * 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 + * This is more efficient than calling [JavaRDD.repartition] and then sorting within each partition * because it can push the sorting down into the shuffle machinery. */ fun JavaRDD>.repartitionAndSortWithinPartitions( @@ -780,7 +776,7 @@ fun JavaRDD>.repartitionAndSortWithinPartitions( /** * Sort the RDD by key, so that each partition contains a sorted range of the elements. Calling - * `collect` or `save` on the resulting RDD will return or output an ordered list of records + * [JavaRDD.collect] or `save` on the resulting RDD will return or output an ordered list of records * (in the `save` case, they will be written to multiple `part-X` files in the filesystem, in * order of the keys). */ @@ -789,7 +785,7 @@ fun JavaRDD>.sortByKey(ascending: Boolean = true): JavaRDD JavaRDD>.sortByKey(ascending: Boolean, numPartitions: In /** * Sort the RDD by key, so that each partition contains a sorted range of the elements. Calling - * `collect` or `save` on the resulting RDD will return or output an ordered list of records + * [JavaRDD.collect] or `save` on the resulting RDD will return or output an ordered list of records * (in the `save` case, they will be written to multiple `part-X` files in the filesystem, in * order of the keys). */ @@ -807,7 +803,7 @@ fun JavaRDD>.sortByKey(comp: Comparator, ascending: Boole /** * Sort the RDD by key, so that each partition contains a sorted range of the elements. Calling - * `collect` or `save` on the resulting RDD will return or output an ordered list of records + * [JavaRDD.collect] or `save` on the resulting RDD will return or output an ordered list of records * (in the `save` case, they will be written to multiple `part-X` files in the filesystem, in * order of the keys). */ @@ -819,10 +815,10 @@ fun JavaRDD>.sortByKey( //#if sparkMinor >= 3.1 /** - * Return a RDD containing only the elements in the inclusive range `lower` to `upper`. - * If the RDD has been partitioned using a `RangePartitioner`, then this operation can be + * Return a RDD containing only the elements in the inclusive range [lower] to [upper]. + * If the RDD has been partitioned using a [RangePartitioner], then this operation can be * performed efficiently by only scanning the partitions that might contain matching elements. - * Otherwise, a standard `filter` is applied to all partitions. + * Otherwise, a standard [filter] is applied to all partitions. * * @since 3.1.0 */ @@ -830,10 +826,10 @@ fun JavaRDD>.filterByRange(lower: K, upper: K): JavaRDD JavaRDD>.countApproxDistinctByKey( * * 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. + * [here](https://doi.org/10.1145/2452376.2452456). * * @param relativeSD Relative accuracy. Smaller values create counters that require more space. * It must be greater than 0.000017. @@ -893,7 +889,7 @@ fun JavaRDD>.countApproxDistinctByKey( * * 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. + * [here](https://doi.org/10.1145/2452376.2452456). * * @param relativeSD Relative accuracy. Smaller values create counters that require more space. * It must be greater than 0.000017. diff --git a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/StreamingKeyValues.kt b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/StreamingKeyValues.kt index ebb431ca..f0066d70 100644 --- a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/StreamingKeyValues.kt +++ b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/StreamingKeyValues.kt @@ -23,7 +23,6 @@ package org.jetbrains.kotlinx.spark.api import org.apache.spark.HashPartitioner import org.apache.spark.Partitioner -import org.apache.spark.api.java.JavaPairRDD import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.java.Optional import org.apache.spark.streaming.Duration diff --git a/settings.gradle.kts b/settings.gradle.kts index fcb82334..d0aa217b 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -41,4 +41,4 @@ buildCache { local { removeUnusedEntriesAfterDays = 30 } -} \ No newline at end of file +} From 7f37f50ebdf55d49ffe38ee4eb85644c5053f8ca Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Fri, 15 Jul 2022 14:33:23 +0200 Subject: [PATCH 06/16] min/max for rdds, small fixes and starting with tests --- .../org/jetbrains/kotlinx/spark/api/Rdd.kt | 25 ++++++++++++++ .../kotlinx/spark/api/RddKeyValue.kt | 22 +++++++----- .../jetbrains/kotlinx/spark/api/RddTest.kt | 34 +++++++++++++++++++ 3 files changed, 72 insertions(+), 9 deletions(-) create mode 100644 kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/RddTest.kt diff --git a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Rdd.kt b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Rdd.kt index 0f82a69a..d7db6249 100644 --- a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Rdd.kt +++ b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Rdd.kt @@ -2,6 +2,7 @@ package org.jetbrains.kotlinx.spark.api import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.java.JavaSparkContext +import java.io.Serializable fun JavaSparkContext.rddOf( @@ -13,3 +14,27 @@ fun JavaSparkContext.toRDD( elements: List, numSlices: Int = defaultParallelism(), ): JavaRDD = parallelize(elements, numSlices) + +/** + * Returns the minimum element from this RDD as defined by the specified + * [Comparator]. + * + * @return the minimum of the RDD + */ +fun > JavaRDD.min(): T = min( + object : Comparator, Serializable { + override fun compare(o1: T, o2: T): Int = o1.compareTo(o2) + } +) + +/** + * Returns the maximum element from this RDD as defined by the specified + * [Comparator]. + * + * @return the maximum of the RDD + */ +fun > JavaRDD.max(): T = max( + object : Comparator, Serializable { + override fun compare(o1: T, o2: T): Int = o1.compareTo(o2) + } +) diff --git a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddKeyValue.kt b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddKeyValue.kt index f43cf7a8..5bc86fde 100644 --- a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddKeyValue.kt +++ b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddKeyValue.kt @@ -187,7 +187,7 @@ fun JavaRDD>.sampleByKey( fun JavaRDD>.sampleByKeyExact( withReplacement: Boolean, fractions: Map, - seed: Long = Random.nextLong() + seed: Long = Random.nextLong(), ): JavaRDD> = toPairRDD() .sampleByKeyExact(withReplacement, fractions, seed) .toTupleRDD() @@ -210,8 +210,8 @@ fun JavaRDD>.reduceByKey( * to a "combiner" in MapReduce. Output will be hash-partitioned with numPartitions partitions. */ fun JavaRDD>.reduceByKey( - func: (V, V) -> V, numPartitions: Int, + func: (V, V) -> V, ): JavaRDD> = toPairRDD() .reduceByKey(func, numPartitions) .toTupleRDD() @@ -336,7 +336,7 @@ fun JavaRDD>.join( */ fun JavaRDD>.leftOuterJoin( other: JavaRDD>, - partitioner: Partitioner + partitioner: Partitioner, ): JavaRDD>>> = toPairRDD() .leftOuterJoin(other.toPairRDD(), partitioner) .toTupleRDD() @@ -377,7 +377,7 @@ fun JavaRDD>.fullOuterJoin( fun JavaRDD>.combineByKey( createCombiner: (V) -> C, mergeValue: (C, V) -> C, - mergeCombiners: (C, C) -> C + mergeCombiners: (C, C) -> C, ): JavaRDD> = toPairRDD() .combineByKey(createCombiner, mergeValue, mergeCombiners) .toTupleRDD() @@ -415,7 +415,7 @@ fun JavaRDD>.join(other: JavaRDD>): JavaRDD< */ fun JavaRDD>.join( other: JavaRDD>, - numPartitions: Int + numPartitions: Int, ): JavaRDD>> = toPairRDD() .join(other.toPairRDD(), numPartitions) @@ -427,7 +427,9 @@ fun JavaRDD>.join( * pair (k, (v, None)) if no elements in [other] have key k. Hash-partitions the output * using the existing partitioner/parallelism level. */ -fun JavaRDD>.leftOuterJoin(other: JavaRDD>): JavaRDD>>> = +fun JavaRDD>.leftOuterJoin( + other: JavaRDD>, +): JavaRDD>>> = toPairRDD() .leftOuterJoin(other.toPairRDD()) .toTupleRDD() @@ -440,7 +442,7 @@ fun JavaRDD>.leftOuterJoin(other: JavaRDD>): */ fun JavaRDD>.leftOuterJoin( other: JavaRDD>, - numPartitions: Int + numPartitions: Int, ): JavaRDD>>> = toPairRDD() .leftOuterJoin(other.toPairRDD(), numPartitions) .toTupleRDD() @@ -451,7 +453,9 @@ fun JavaRDD>.leftOuterJoin( * pair (k, (None, w)) if no elements in [this] have key k. Hash-partitions the resulting * RDD using the existing partitioner/parallelism level. */ -fun JavaRDD>.rightOuterJoin(other: JavaRDD>): JavaRDD, W>>> = +fun JavaRDD>.rightOuterJoin( + other: JavaRDD>, +): JavaRDD, W>>> = toPairRDD() .rightOuterJoin(other.toPairRDD()) .toTupleRDD() @@ -604,7 +608,7 @@ fun JavaRDD>.cogroup( */ fun JavaRDD>.cogroup( other: JavaRDD>, - numPartitions: Int + numPartitions: Int, ): JavaRDD, Iterable>>> = toPairRDD().cogroup(other.toPairRDD(), numPartitions).toTupleRDD() diff --git a/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/RddTest.kt b/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/RddTest.kt new file mode 100644 index 00000000..d0eab2a2 --- /dev/null +++ b/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/RddTest.kt @@ -0,0 +1,34 @@ +package org.jetbrains.kotlinx.spark.api + +import io.kotest.core.spec.style.ShouldSpec +import io.kotest.matchers.shouldBe +import org.jetbrains.kotlinx.spark.api.tuples.* + +class RddTest : ShouldSpec({ + context("RDD extension functions") { + + withSpark(logLevel = SparkLogLevel.DEBUG) { + + context("Key/value") { + should("work with spark example") { + val data = listOf(1, 1, 2, 2, 2, 3).map(Int::toString) + val rdd = sc.parallelize(data) + + val pairs = rdd.map { it X 1 } + val counts = pairs.reduceByKey { a, b -> a + b } + val list = counts.collect().toList() + list shouldBe listOf("1" X 2, "2" X 3, "3" X 1) + } + } + + context("Double") { + should("get max/min") { + val data = listOf(1, 1, 2, 2, 2, 3).map(Int::toDouble) + val rdd = sc.parallelize(data) + rdd.max() shouldBe 3.0 + rdd.min() shouldBe 1.0 + } + } + } + } +}) \ No newline at end of file From 2e73f7e7b327c849a3a7940bb307c62c2a1a05c9 Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Wed, 13 Jul 2022 13:15:28 +0200 Subject: [PATCH 07/16] added seqOf functions and tests --- .../org/jetbrains/kotlinx/spark/api/Seq.kt | 40 +++++++++++++++++++ .../jetbrains/kotlinx/spark/api/ApiTest.kt | 12 ++++++ 2 files changed, 52 insertions(+) create mode 100644 kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Seq.kt diff --git a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Seq.kt b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Seq.kt new file mode 100644 index 00000000..2cb44ca2 --- /dev/null +++ b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Seq.kt @@ -0,0 +1,40 @@ +package org.jetbrains.kotlinx.spark.api + +import scala.collection.immutable.`Seq$`.`MODULE$` as Seq +import scala.collection.immutable.Seq as Seq +import scala.collection.mutable.`Seq$`.`MODULE$` as MutableSeq +import scala.collection.mutable.Seq as MutableSeq + +/** + * Returns a new empty immutable Seq. + */ +fun emptySeq(): Seq = Seq.empty() as Seq + +/** + * Returns a new immutable Seq with the given elements. + */ +fun seqOf(vararg elements: T): Seq = + if (elements.isEmpty()) + emptySeq() + else + Seq.newBuilder().apply { + for (it in elements) + `$plus$eq`(it) + }.result() as Seq + +/** + * Returns a new mutable Seq with the given elements. + */ +fun emptyMutableSeq(): MutableSeq = MutableSeq.empty() as MutableSeq + +/** + * Returns a new mutable Seq with the given elements. + */ +fun mutableSeqOf(vararg elements: T): MutableSeq = + if (elements.isEmpty()) + emptyMutableSeq() + else + MutableSeq.newBuilder().apply { + for (it in elements) + `$plus$eq`(it) + }.result() as MutableSeq diff --git a/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/ApiTest.kt b/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/ApiTest.kt index c3362e91..20044807 100644 --- a/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/ApiTest.kt +++ b/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/ApiTest.kt @@ -33,6 +33,18 @@ class ApiTest : ShouldSpec({ context("miscellaneous integration tests") { withSpark(props = mapOf("spark.sql.codegen.comments" to true)) { + should("Create Seqs") { + spark.createDataset(seqOf(1, 2, 3), encoder()) + .collectAsList() shouldBe listOf(1, 2, 3) + + + seqOf(1, 2, 3) shouldBe seqOf(1, 2, 3) + mutableSeqOf(1, 2, 3) shouldBe mutableSeqOf(1, 2, 3) + + seqOf() shouldBe emptySeq() + mutableSeqOf() shouldBe emptyMutableSeq() + } + @OptIn(ExperimentalStdlibApi::class) should("broadcast variables") { val largeList = (1..15).map { SomeClass(a = (it..15).toList().toIntArray(), b = it) } From bf241a9fe9c96966978be152925791acb8b475c6 Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Fri, 15 Jul 2022 15:27:45 +0200 Subject: [PATCH 08/16] rdd test? --- .../test/kotlin/org/jetbrains/kotlinx/spark/api/RddTest.kt | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/RddTest.kt b/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/RddTest.kt index d0eab2a2..b1a3e36e 100644 --- a/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/RddTest.kt +++ b/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/RddTest.kt @@ -1,6 +1,9 @@ package org.jetbrains.kotlinx.spark.api import io.kotest.core.spec.style.ShouldSpec +import io.kotest.matchers.collections.shouldContain +import io.kotest.matchers.collections.shouldContainAll +import io.kotest.matchers.should import io.kotest.matchers.shouldBe import org.jetbrains.kotlinx.spark.api.tuples.* @@ -17,7 +20,7 @@ class RddTest : ShouldSpec({ val pairs = rdd.map { it X 1 } val counts = pairs.reduceByKey { a, b -> a + b } val list = counts.collect().toList() - list shouldBe listOf("1" X 2, "2" X 3, "3" X 1) + list.shouldContainAll("1" X 2, "2" X 3, "3" X 1) } } From ba75c4847974c5616da51108b3105cf5847ce148 Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Mon, 18 Jul 2022 17:19:28 +0200 Subject: [PATCH 09/16] updated jupyter with rdd funs, added rdd index groups generation example, renamed toPairRDD to toJavaPairRDD for clarity --- .../spark/examples/RddGroupCalculation.kt | 232 +++++++ .../streaming/KotlinStatefulNetworkCount.kt | 11 +- .../spark/api/jupyter/SparkIntegration.kt | 4 + .../kotlinx/spark/api/jupyter/JupyterTests.kt | 24 +- .../kotlinx/spark/api/Conversions.kt | 589 ++++++++++++++++-- .../jetbrains/kotlinx/spark/api/RddDouble.kt | 35 +- .../kotlinx/spark/api/RddKeyValue.kt | 205 +++--- .../kotlinx/spark/api/SparkSession.kt | 4 +- .../kotlinx/spark/api/StreamingKeyValues.kt | 4 +- .../jetbrains/kotlinx/spark/api/RddTest.kt | 50 +- 10 files changed, 971 insertions(+), 187 deletions(-) create mode 100644 examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/RddGroupCalculation.kt diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/RddGroupCalculation.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/RddGroupCalculation.kt new file mode 100644 index 00000000..134603ed --- /dev/null +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/RddGroupCalculation.kt @@ -0,0 +1,232 @@ +package org.jetbrains.kotlinx.spark.examples + +import org.apache.spark.sql.Dataset +import org.jetbrains.kotlinx.spark.api.* +import org.jetbrains.kotlinx.spark.api.tuples.X +import org.jetbrains.kotlinx.spark.examples.GroupCalculation.getAllPossibleGroups +import scala.Tuple2 +import java.util.* +import kotlin.math.pow + +/** + * Gets all the possible, unique, non repeating groups of indices for a list. + * + * Example by Jolanrensen. + */ + +fun main() = withSpark { + val groupIndices = getAllPossibleGroups(listSize = 10, groupSize = 3) + .sort("value") + groupIndices.showDS(numRows = groupIndices.count().toInt()) +} + +object GroupCalculation { + + /** + * Get all the possible, unique, non repeating groups (of size [groupSize]) of indices for a list of + * size [listSize]. + * + * + * The workload is evenly distributed by [listSize] and [groupSize] + * + * @param listSize the size of the list for which to calculate the indices + * @param groupSize the size of a group of indices + * @return all the possible, unique non repeating groups of indices + */ + fun KSparkSession.getAllPossibleGroups( + listSize: Int, + groupSize: Int, + ): Dataset { + val indices = (0 until listSize).toList().toRDD() + + // for a groupSize of 1, no pairing up is needed, so just return the indices converted to IntArrays + if (groupSize == 1) { + return indices + .mapPartitions { + it.map { intArrayOf(it) } + } + .toDS() + + } + + val keys = indices + .mapPartitions { // this converts all indices to (number in table, index) + + // key is key (item in table), value is index in list + val keyValues = ArrayList>() + while (it.hasNext()) { + val listIndex = it.next() + + // for each dimension loop over the other dimensions using addTuples + for (dimension in 0 until groupSize) { + addTuples( + groupSize = groupSize, + keyValues = keyValues, + value = listIndex, + listSize = listSize, + currentDimension = 0, + indexTuple = IntArray(groupSize), + skipDimension = dimension, + ) + } + } + keyValues.iterator() + } + + // each number in table occurs for each dimension as key. + // The values of those two will be a tuple of (key, indices as list) + val allPossibleGroups = keys.aggregateByKey( + zeroValue = IntArray(groupSize) { -1 }, + seqFunc = { base: IntArray, listIndex: Int -> + // put listIndex in the first empty spot in base + base[base.indexOfFirst { it < 0 }] = listIndex + + base + }, + + // how to merge partially filled up int arrays + combFunc = { a: IntArray, b: IntArray -> + // merge a and b + var j = 0 + for (i in a.indices) { + if (a[i] < 0) { + while (b[j] < 0) { + j++ + if (j == b.size) return@aggregateByKey a + } + a[i] = b[j] + j++ + } + } + a + }, + ) + .values() // finally just take the values + + return allPossibleGroups.toDS() + } + + /** + * Simple method to give each index of x dimensions a unique number. + * + * @param indexTuple IntArray (can be seen as Tuple) of size x with all values < listSize. The index for which to return the number + * @param listSize The size of the list, aka the max width, height etc. of the table + * @return the unique number for this [indexTuple] + */ + private fun getTupleValue(indexTuple: IntArray, listSize: Int): Int = + indexTuple.indices.sumOf { + indexTuple[it] * listSize.toDouble().pow(it).toInt() + } + + + /** + * To make sure that every tuple is only picked once, this method returns true only if the indices are in the right + * corner of the matrix. This works for any number of dimensions > 1. Here is an example for 2-D: + * + * + * - 0 1 2 3 4 5 6 7 8 9 + * -------------------------------- + * 0| x ✓ ✓ ✓ ✓ ✓ ✓ ✓ ✓ ✓ + * 1| x x ✓ ✓ ✓ ✓ ✓ ✓ ✓ ✓ + * 2| x x x ✓ ✓ ✓ ✓ ✓ ✓ ✓ + * 3| x x x x ✓ ✓ ✓ ✓ ✓ ✓ + * 4| x x x x x ✓ ✓ ✓ ✓ ✓ + * 5| x x x x x x ✓ ✓ ✓ ✓ + * 6| x x x x x x x ✓ ✓ ✓ + * 7| x x x x x x x x ✓ ✓ + * 8| x x x x x x x x x ✓ + * 9| x x x x x x x x x x + * + * @param indexTuple a tuple of indices in the form of an IntArray + * @return true if this tuple is in the right corner and should be included + */ + private fun isValidIndexTuple(indexTuple: IntArray): Boolean { + // x - y > 0; 2d + // (x - y) > 0 && (x - z) > 0 && (y - z) > 0; 3d + // (x - y) > 0 && (x - z) > 0 && (x - a) > 0 && (y - z) > 0 && (y - a) > 0 && (z - a) > 0; 4d + require(indexTuple.size >= 2) { "not a tuple" } + for (i in 0 until indexTuple.size - 1) { + for (j in i + 1 until indexTuple.size) { + if (indexTuple[i] - indexTuple[j] <= 0) return false + } + } + return true + } + + /** + * Recursive method that for [skipDimension] loops over all the other dimensions and stores + * in [keyValues] all results from [getTupleValue] as key and [value] as value. + * In the end, [keyValues] will have, for each key in the table below, a value for the key's column, row etc. + * + * + * This is an example for 2D. The letters will be int indices as well (a = 0, b = 1, ..., [listSize]), but help for clarification. + * The numbers we don't want are filtered out using [isValidIndexTuple]. + * The actual value of the number in the table comes from [getTupleValue]. + * + * + * + * + * - a b c d e f g h i j + * -------------------------------- + * a| - 1 2 3 4 5 6 7 8 9 + * b| - - 12 13 14 15 16 17 18 19 + * c| - - - 23 24 25 26 27 28 29 + * d| - - - - 34 35 36 37 38 39 + * e| - - - - - 45 46 47 48 49 + * f| - - - - - - 56 57 58 59 + * g| - - - - - - - 67 68 69 + * h| - - - - - - - - 78 79 + * i| - - - - - - - - - 89 + * j| - - - - - - - - - - + * + * + * @param groupSize the size of index tuples to form + * @param keyValues the MutableList in which to store the key-value results + * @param value the current index to work from (can be seen as a letter in the table above) + * @param listSize the size of the list to make + * @param currentDimension the indicator for which dimension we're currently calculating for (and how deep in the recursion we are) + * @param indexTuple the array (or tuple) in which to store the current indices + * @param skipDimension the current dimension that will have a set value [value] while looping over the other dimensions + */ + private fun addTuples( + groupSize: Int, + keyValues: MutableList>, + value: Int, + listSize: Int, + currentDimension: Int, + indexTuple: IntArray, + skipDimension: Int, + ) { + if (currentDimension >= groupSize) { // base case + if (isValidIndexTuple(indexTuple)) { + keyValues += getTupleValue(indexTuple, listSize) X value + } + return + } + if (currentDimension == skipDimension) { + indexTuple[currentDimension] = value + addTuples( + groupSize = groupSize, + keyValues = keyValues, + value = value, + listSize = listSize, + currentDimension = currentDimension + 1, + indexTuple = indexTuple, + skipDimension = skipDimension, + ) + } else { + for (i in 0 until listSize) { + indexTuple[currentDimension] = i + addTuples( + groupSize = groupSize, + keyValues = keyValues, + value = value, + listSize = listSize, + currentDimension = currentDimension + 1, + indexTuple = indexTuple, + skipDimension = skipDimension, + ) + } + } + } +} diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/KotlinStatefulNetworkCount.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/KotlinStatefulNetworkCount.kt index 2c938ead..07034919 100644 --- a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/KotlinStatefulNetworkCount.kt +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/KotlinStatefulNetworkCount.kt @@ -24,11 +24,8 @@ import org.apache.spark.api.java.StorageLevels import org.apache.spark.streaming.Durations import org.apache.spark.streaming.State import org.apache.spark.streaming.StateSpec -import org.jetbrains.kotlinx.spark.api.getOrElse -import org.jetbrains.kotlinx.spark.api.mapWithState -import org.jetbrains.kotlinx.spark.api.toPairRDD +import org.jetbrains.kotlinx.spark.api.* import org.jetbrains.kotlinx.spark.api.tuples.X -import org.jetbrains.kotlinx.spark.api.withSparkStreaming import java.util.regex.Pattern import kotlin.system.exitProcess @@ -71,8 +68,8 @@ object KotlinStatefulNetworkCount { ) { // Initial state RDD input to mapWithState - val tuples = listOf("hello" X 1, "world" X 1) - val initialRDD = ssc.sparkContext().parallelize(tuples) + val tuples = arrayOf("hello" X 1, "world" X 1) + val initialRDD = ssc.sparkContext().rddOf(*tuples) val lines = ssc.socketTextStream( args.getOrElse(0) { DEFAULT_HOSTNAME }, @@ -95,7 +92,7 @@ object KotlinStatefulNetworkCount { val stateDstream = wordsDstream.mapWithState( StateSpec .function(mappingFunc) - .initialState(initialRDD.toPairRDD()) + .initialState(initialRDD.toJavaPairRDD()) ) stateDstream.print() diff --git a/jupyter/src/main/kotlin/org/jetbrains/kotlinx/spark/api/jupyter/SparkIntegration.kt b/jupyter/src/main/kotlin/org/jetbrains/kotlinx/spark/api/jupyter/SparkIntegration.kt index 3752fdf9..3f5ae1b7 100644 --- a/jupyter/src/main/kotlin/org/jetbrains/kotlinx/spark/api/jupyter/SparkIntegration.kt +++ b/jupyter/src/main/kotlin/org/jetbrains/kotlinx/spark/api/jupyter/SparkIntegration.kt @@ -77,6 +77,10 @@ internal class SparkIntegration : Integration() { inline fun RDD.toDF(vararg colNames: String): Dataset = toDF(spark, *colNames)""".trimIndent(), """ inline fun JavaRDDLike.toDF(vararg colNames: String): Dataset = toDF(spark, *colNames)""".trimIndent(), + """ + fun List.toRDD(numSlices: Int = sc.defaultParallelism()): JavaRDD = sc.toRDD(this, numSlices)""".trimIndent(), + """ + fun rddOf(vararg elements: T, numSlices: Int = sc.defaultParallelism()): JavaRDD = sc.toRDD(elements.toList(), numSlices)""".trimIndent(), """ val udf: UDFRegistration get() = spark.udf()""".trimIndent(), ).map(::execute) diff --git a/jupyter/src/test/kotlin/org/jetbrains/kotlinx/spark/api/jupyter/JupyterTests.kt b/jupyter/src/test/kotlin/org/jetbrains/kotlinx/spark/api/jupyter/JupyterTests.kt index b4b750ee..3b4632db 100644 --- a/jupyter/src/test/kotlin/org/jetbrains/kotlinx/spark/api/jupyter/JupyterTests.kt +++ b/jupyter/src/test/kotlin/org/jetbrains/kotlinx/spark/api/jupyter/JupyterTests.kt @@ -118,10 +118,10 @@ class JupyterTests : ShouldSpec({ @Language("kts") val html = execHtml( """ - val rdd: JavaRDD> = sc.parallelize(listOf( + val rdd: JavaRDD> = listOf( listOf(1, 2, 3), listOf(4, 5, 6), - )) + ).toRDD() rdd """.trimIndent() ) @@ -135,10 +135,10 @@ class JupyterTests : ShouldSpec({ @Language("kts") val html = execHtml( """ - val rdd: JavaRDD = sc.parallelize(listOf( + val rdd: JavaRDD = rddOf( intArrayOf(1, 2, 3), intArrayOf(4, 5, 6), - )) + ) rdd """.trimIndent() ) @@ -164,12 +164,12 @@ class JupyterTests : ShouldSpec({ @Language("kts") val html = execHtml( """ - val rdd = sc.parallelize( + val rdd = listOf( Test("aaaaaaaaa", longArrayOf(1L, 100000L, 24L), mapOf(1 to "one", 2 to "two")), Test("aaaaaaaaa", longArrayOf(1L, 100000L, 24L), mapOf(1 to "one", 2 to "two")), - ) - ) + ).toRDD() + rdd """.trimIndent() ) @@ -180,10 +180,10 @@ class JupyterTests : ShouldSpec({ @Language("kts") val html = execHtml( """ - val rdd: JavaPairRDD = sc.parallelizePairs(listOf( + val rdd: JavaPairRDD = rddOf( c(1, 2).toTuple(), c(3, 4).toTuple(), - )) + ).toJavaPairRDD() rdd """.trimIndent() ) @@ -197,7 +197,7 @@ class JupyterTests : ShouldSpec({ @Language("kts") val html = execHtml( """ - val rdd: JavaDoubleRDD = sc.parallelizeDoubles(listOf(1.0, 2.0, 3.0, 4.0,)) + val rdd: JavaDoubleRDD = rddOf(1.0, 2.0, 3.0, 4.0,).toJavaDoubleRDD() rdd """.trimIndent() ) @@ -213,10 +213,10 @@ class JupyterTests : ShouldSpec({ @Language("kts") val html = execHtml( """ - val rdd: RDD> = sc.parallelize(listOf( + val rdd: RDD> = rddOf( listOf(1, 2, 3), listOf(4, 5, 6), - )).rdd() + ).rdd() rdd """.trimIndent() ) diff --git a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt index 5212403e..73ec0a6e 100644 --- a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt +++ b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt @@ -45,7 +45,6 @@ import scala.collection.mutable.Seq as ScalaMutableSeq import scala.collection.mutable.Set as ScalaMutableSet import org.apache.spark.streaming.State - /** Returns state value if it exists, else `null`. */ fun State.getOrNull(): T? = if (exists()) get() else null @@ -249,115 +248,375 @@ fun Tuple3.toArity(): Arity3 = Arity3 Tuple4.toArity(): Arity4 = Arity4(this._1(), this._2(), this._3(), this._4()) +fun Tuple4.toArity(): Arity4 = + Arity4(this._1(), this._2(), this._3(), this._4()) /** * Returns a new Arity5 based on this Tuple5. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Tuple5.toArity(): Arity5 = Arity5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple5.toArity(): Arity5 = + Arity5(this._1(), this._2(), this._3(), this._4(), this._5()) /** * Returns a new Arity6 based on this Tuple6. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Tuple6.toArity(): Arity6 = Arity6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple6.toArity(): Arity6 = + Arity6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) /** * Returns a new Arity7 based on this Tuple7. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Tuple7.toArity(): Arity7 = Arity7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple7.toArity(): Arity7 = + Arity7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) /** * Returns a new Arity8 based on this Tuple8. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Tuple8.toArity(): Arity8 = Arity8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple8.toArity(): Arity8 = + Arity8( + this._1(), + this._2(), + this._3(), + this._4(), + this._5(), + this._6(), + this._7(), + this._8() + ) /** * Returns a new Arity9 based on this Tuple9. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Tuple9.toArity(): Arity9 = Arity9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple9.toArity(): Arity9 = + Arity9( + this._1(), + this._2(), + this._3(), + this._4(), + this._5(), + this._6(), + this._7(), + this._8(), + this._9() + ) /** * Returns a new Arity10 based on this Tuple10. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Tuple10.toArity(): Arity10 = Arity10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple10.toArity(): Arity10 = + Arity10( + this._1(), + this._2(), + this._3(), + this._4(), + this._5(), + this._6(), + this._7(), + this._8(), + this._9(), + this._10() + ) /** * Returns a new Arity11 based on this Tuple11. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Tuple11.toArity(): Arity11 = Arity11(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11()) +fun Tuple11.toArity(): Arity11 = + Arity11( + this._1(), + this._2(), + this._3(), + this._4(), + this._5(), + this._6(), + this._7(), + this._8(), + this._9(), + this._10(), + this._11() + ) /** * Returns a new Arity12 based on this Tuple12. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Tuple12.toArity(): Arity12 = Arity12(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12()) +fun Tuple12.toArity(): Arity12 = + Arity12( + this._1(), + this._2(), + this._3(), + this._4(), + this._5(), + this._6(), + this._7(), + this._8(), + this._9(), + this._10(), + this._11(), + this._12() + ) /** * Returns a new Arity13 based on this Tuple13. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Tuple13.toArity(): Arity13 = Arity13(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13()) +fun Tuple13.toArity(): Arity13 = + Arity13( + this._1(), + this._2(), + this._3(), + this._4(), + this._5(), + this._6(), + this._7(), + this._8(), + this._9(), + this._10(), + this._11(), + this._12(), + this._13() + ) /** * Returns a new Arity14 based on this Tuple14. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Tuple14.toArity(): Arity14 = Arity14(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14()) +fun Tuple14.toArity(): Arity14 = + Arity14( + this._1(), + this._2(), + this._3(), + this._4(), + this._5(), + this._6(), + this._7(), + this._8(), + this._9(), + this._10(), + this._11(), + this._12(), + this._13(), + this._14() + ) /** * Returns a new Arity15 based on this Tuple15. - **/@Deprecated("Use Scala tuples instead.", ReplaceWith("")) + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Tuple15.toArity(): Arity15 = Arity15(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15()) +fun Tuple15.toArity(): Arity15 = + Arity15( + this._1(), + this._2(), + this._3(), + this._4(), + this._5(), + this._6(), + this._7(), + this._8(), + this._9(), + this._10(), + this._11(), + this._12(), + this._13(), + this._14(), + this._15() + ) /** * Returns a new Arity16 based on this Tuple16. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Tuple16.toArity(): Arity16 = Arity16(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16()) +fun Tuple16.toArity(): Arity16 = + Arity16( + this._1(), + this._2(), + this._3(), + this._4(), + this._5(), + this._6(), + this._7(), + this._8(), + this._9(), + this._10(), + this._11(), + this._12(), + this._13(), + this._14(), + this._15(), + this._16() + ) /** * Returns a new Arity17 based on this Tuple17. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Tuple17.toArity(): Arity17 = Arity17(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17()) +fun Tuple17.toArity(): Arity17 = + Arity17( + this._1(), + this._2(), + this._3(), + this._4(), + this._5(), + this._6(), + this._7(), + this._8(), + this._9(), + this._10(), + this._11(), + this._12(), + this._13(), + this._14(), + this._15(), + this._16(), + this._17() + ) /** * Returns a new Arity18 based on this Tuple18. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Tuple18.toArity(): Arity18 = Arity18(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18()) +fun Tuple18.toArity(): Arity18 = + Arity18( + this._1(), + this._2(), + this._3(), + this._4(), + this._5(), + this._6(), + this._7(), + this._8(), + this._9(), + this._10(), + this._11(), + this._12(), + this._13(), + this._14(), + this._15(), + this._16(), + this._17(), + this._18() + ) /** * Returns a new Arity19 based on this Tuple19. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Tuple19.toArity(): Arity19 = Arity19(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19()) +fun Tuple19.toArity(): Arity19 = + Arity19( + this._1(), + this._2(), + this._3(), + this._4(), + this._5(), + this._6(), + this._7(), + this._8(), + this._9(), + this._10(), + this._11(), + this._12(), + this._13(), + this._14(), + this._15(), + this._16(), + this._17(), + this._18(), + this._19() + ) /** * Returns a new Arity20 based on this Tuple20. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Tuple20.toArity(): Arity20 = Arity20(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20()) +fun Tuple20.toArity(): Arity20 = + Arity20( + this._1(), + this._2(), + this._3(), + this._4(), + this._5(), + this._6(), + this._7(), + this._8(), + this._9(), + this._10(), + this._11(), + this._12(), + this._13(), + this._14(), + this._15(), + this._16(), + this._17(), + this._18(), + this._19(), + this._20() + ) /** * Returns a new Arity21 based on this Tuple21. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Tuple21.toArity(): Arity21 = Arity21(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20(), this._21()) +fun Tuple21.toArity(): Arity21 = + Arity21( + this._1(), + this._2(), + this._3(), + this._4(), + this._5(), + this._6(), + this._7(), + this._8(), + this._9(), + this._10(), + this._11(), + this._12(), + this._13(), + this._14(), + this._15(), + this._16(), + this._17(), + this._18(), + this._19(), + this._20(), + this._21() + ) /** * Returns a new Arity22 based on this Tuple22. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Tuple22.toArity(): Arity22 = Arity22(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20(), this._21(), this._22()) +fun Tuple22.toArity(): Arity22 = + Arity22( + this._1(), + this._2(), + this._3(), + this._4(), + this._5(), + this._6(), + this._7(), + this._8(), + this._9(), + this._10(), + this._11(), + this._12(), + this._13(), + this._14(), + this._15(), + this._16(), + this._17(), + this._18(), + this._19(), + this._20(), + this._21(), + this._22() + ) /** * Returns a new Tuple1 based on this Arity1. @@ -381,112 +640,362 @@ fun Arity3.toTuple(): Tuple3 = Tuple3 Arity4.toTuple(): Tuple4 = Tuple4(this._1, this._2, this._3, this._4) +fun Arity4.toTuple(): Tuple4 = + Tuple4(this._1, this._2, this._3, this._4) /** * Returns a new Tuple5 based on this Arity5. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Arity5.toTuple(): Tuple5 = Tuple5(this._1, this._2, this._3, this._4, this._5) +fun Arity5.toTuple(): Tuple5 = + Tuple5(this._1, this._2, this._3, this._4, this._5) /** * Returns a new Tuple6 based on this Arity6. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Arity6.toTuple(): Tuple6 = Tuple6(this._1, this._2, this._3, this._4, this._5, this._6) +fun Arity6.toTuple(): Tuple6 = + Tuple6(this._1, this._2, this._3, this._4, this._5, this._6) /** * Returns a new Tuple7 based on this Arity7. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Arity7.toTuple(): Tuple7 = Tuple7(this._1, this._2, this._3, this._4, this._5, this._6, this._7) +fun Arity7.toTuple(): Tuple7 = + Tuple7(this._1, this._2, this._3, this._4, this._5, this._6, this._7) /** * Returns a new Tuple8 based on this Arity8. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Arity8.toTuple(): Tuple8 = Tuple8(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8) +fun Arity8.toTuple(): Tuple8 = + Tuple8(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8) /** * Returns a new Tuple9 based on this Arity9. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Arity9.toTuple(): Tuple9 = Tuple9(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9) +fun Arity9.toTuple(): Tuple9 = + Tuple9( + this._1, + this._2, + this._3, + this._4, + this._5, + this._6, + this._7, + this._8, + this._9 + ) /** * Returns a new Tuple10 based on this Arity10. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Arity10.toTuple(): Tuple10 = Tuple10(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10) +fun Arity10.toTuple(): Tuple10 = + Tuple10( + this._1, + this._2, + this._3, + this._4, + this._5, + this._6, + this._7, + this._8, + this._9, + this._10 + ) /** * Returns a new Tuple11 based on this Arity11. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Arity11.toTuple(): Tuple11 = Tuple11(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11) +fun Arity11.toTuple(): Tuple11 = + Tuple11( + this._1, + this._2, + this._3, + this._4, + this._5, + this._6, + this._7, + this._8, + this._9, + this._10, + this._11 + ) /** * Returns a new Tuple12 based on this Arity12. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Arity12.toTuple(): Tuple12 = Tuple12(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, this._12) +fun Arity12.toTuple(): Tuple12 = + Tuple12( + this._1, + this._2, + this._3, + this._4, + this._5, + this._6, + this._7, + this._8, + this._9, + this._10, + this._11, + this._12 + ) /** * Returns a new Tuple13 based on this Arity13. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Arity13.toTuple(): Tuple13 = Tuple13(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, this._12, this._13) +fun Arity13.toTuple(): Tuple13 = + Tuple13( + this._1, + this._2, + this._3, + this._4, + this._5, + this._6, + this._7, + this._8, + this._9, + this._10, + this._11, + this._12, + this._13 + ) /** * Returns a new Tuple14 based on this Arity14. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Arity14.toTuple(): Tuple14 = Tuple14(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, this._12, this._13, this._14) +fun Arity14.toTuple(): Tuple14 = + Tuple14( + this._1, + this._2, + this._3, + this._4, + this._5, + this._6, + this._7, + this._8, + this._9, + this._10, + this._11, + this._12, + this._13, + this._14 + ) /** * Returns a new Tuple15 based on this Arity15. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Arity15.toTuple(): Tuple15 = Tuple15(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, this._12, this._13, this._14, this._15) +fun Arity15.toTuple(): Tuple15 = + Tuple15( + this._1, + this._2, + this._3, + this._4, + this._5, + this._6, + this._7, + this._8, + this._9, + this._10, + this._11, + this._12, + this._13, + this._14, + this._15 + ) /** * Returns a new Tuple16 based on this Arity16. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Arity16.toTuple(): Tuple16 = Tuple16(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, this._12, this._13, this._14, this._15, this._16) +fun Arity16.toTuple(): Tuple16 = + Tuple16( + this._1, + this._2, + this._3, + this._4, + this._5, + this._6, + this._7, + this._8, + this._9, + this._10, + this._11, + this._12, + this._13, + this._14, + this._15, + this._16 + ) /** * Returns a new Tuple17 based on this Arity17. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Arity17.toTuple(): Tuple17 = Tuple17(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, this._12, this._13, this._14, this._15, this._16, this._17) +fun Arity17.toTuple(): Tuple17 = + Tuple17( + this._1, + this._2, + this._3, + this._4, + this._5, + this._6, + this._7, + this._8, + this._9, + this._10, + this._11, + this._12, + this._13, + this._14, + this._15, + this._16, + this._17 + ) /** * Returns a new Tuple18 based on this Arity18. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Arity18.toTuple(): Tuple18 = Tuple18(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, this._12, this._13, this._14, this._15, this._16, this._17, this._18) +fun Arity18.toTuple(): Tuple18 = + Tuple18( + this._1, + this._2, + this._3, + this._4, + this._5, + this._6, + this._7, + this._8, + this._9, + this._10, + this._11, + this._12, + this._13, + this._14, + this._15, + this._16, + this._17, + this._18 + ) /** * Returns a new Tuple19 based on this Arity19. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Arity19.toTuple(): Tuple19 = Tuple19(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, this._12, this._13, this._14, this._15, this._16, this._17, this._18, this._19) +fun Arity19.toTuple(): Tuple19 = + Tuple19( + this._1, + this._2, + this._3, + this._4, + this._5, + this._6, + this._7, + this._8, + this._9, + this._10, + this._11, + this._12, + this._13, + this._14, + this._15, + this._16, + this._17, + this._18, + this._19 + ) /** * Returns a new Tuple20 based on this Arity20. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Arity20.toTuple(): Tuple20 = Tuple20(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, this._12, this._13, this._14, this._15, this._16, this._17, this._18, this._19, this._20) +fun Arity20.toTuple(): Tuple20 = + Tuple20( + this._1, + this._2, + this._3, + this._4, + this._5, + this._6, + this._7, + this._8, + this._9, + this._10, + this._11, + this._12, + this._13, + this._14, + this._15, + this._16, + this._17, + this._18, + this._19, + this._20 + ) /** * Returns a new Tuple21 based on this Arity21. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Arity21.toTuple(): Tuple21 = Tuple21(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, this._12, this._13, this._14, this._15, this._16, this._17, this._18, this._19, this._20, this._21) +fun Arity21.toTuple(): Tuple21 = + Tuple21( + this._1, + this._2, + this._3, + this._4, + this._5, + this._6, + this._7, + this._8, + this._9, + this._10, + this._11, + this._12, + this._13, + this._14, + this._15, + this._16, + this._17, + this._18, + this._19, + this._20, + this._21 + ) /** * Returns a new Tuple22 based on this Arity22. **/ @Deprecated("Use Scala tuples instead.", ReplaceWith("")) -fun Arity22.toTuple(): Tuple22 = Tuple22(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, this._12, this._13, this._14, this._15, this._16, this._17, this._18, this._19, this._20, this._21, this._22) +fun Arity22.toTuple(): Tuple22 = + Tuple22( + this._1, + this._2, + this._3, + this._4, + this._5, + this._6, + this._7, + this._8, + this._9, + this._10, + this._11, + this._12, + this._13, + this._14, + this._15, + this._16, + this._17, + this._18, + this._19, + this._20, + this._21, + this._22 + ) diff --git a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddDouble.kt b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddDouble.kt index bdd02236..199337d9 100644 --- a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddDouble.kt +++ b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddDouble.kt @@ -9,57 +9,62 @@ import org.apache.spark.util.StatCounter import scala.Tuple2 @Suppress("UNCHECKED_CAST") -fun JavaRDD.toJavaDoubleRDD(): JavaDoubleRDD = - JavaDoubleRDD.fromRDD(rdd() as RDD) +inline fun JavaRDD.toJavaDoubleRDD(): JavaDoubleRDD = + JavaDoubleRDD.fromRDD( + when (T::class) { + Double::class -> this + else -> map(Number::toDouble) + }.rdd() as RDD + ) @Suppress("UNCHECKED_CAST") fun JavaDoubleRDD.toDoubleRDD(): JavaRDD = JavaDoubleRDD.toRDD(this).toJavaRDD() as JavaRDD /** Add up the elements in this RDD. */ -fun JavaRDD.sum(): Double = toJavaDoubleRDD().sum() +inline fun JavaRDD.sum(): Double = toJavaDoubleRDD().sum() /** * Return a [org.apache.spark.util.StatCounter] object that captures the mean, variance and * count of the RDD's elements in one operation. */ -fun JavaRDD.stats(): StatCounter = toJavaDoubleRDD().stats() +inline fun JavaRDD.stats(): StatCounter = toJavaDoubleRDD().stats() /** Compute the mean of this RDD's elements. */ -fun JavaRDD.mean(): Double = toJavaDoubleRDD().mean() +inline fun JavaRDD.mean(): Double = toJavaDoubleRDD().mean() /** Compute the population variance of this RDD's elements. */ -fun JavaRDD.variance(): Double = toJavaDoubleRDD().variance() +inline fun JavaRDD.variance(): Double = toJavaDoubleRDD().variance() /** Compute the population standard deviation of this RDD's elements. */ -fun JavaRDD.stdev(): Double = toJavaDoubleRDD().stdev() +inline fun JavaRDD.stdev(): Double = toJavaDoubleRDD().stdev() /** * Compute the sample standard deviation of this RDD's elements (which corrects for bias in * estimating the standard deviation by dividing by N-1 instead of N). */ -fun JavaRDD.sampleStdev(): Double = toJavaDoubleRDD().sampleStdev() +inline fun JavaRDD.sampleStdev(): Double = toJavaDoubleRDD().sampleStdev() /** * Compute the sample variance of this RDD's elements (which corrects for bias in * estimating the variance by dividing by N-1 instead of N). */ -fun JavaRDD.sampleVariance(): Double = toJavaDoubleRDD().sampleVariance() +inline fun JavaRDD.sampleVariance(): Double = toJavaDoubleRDD().sampleVariance() /** * Compute the population standard deviation of this RDD's elements. */ -fun JavaRDD.popStdev(): Double = toJavaDoubleRDD().popStdev() +inline fun JavaRDD.popStdev(): Double = toJavaDoubleRDD().popStdev() /** * Compute the population variance of this RDD's elements. */ -fun JavaRDD.popVariance(): Double = toJavaDoubleRDD().popVariance() +inline fun JavaRDD.popVariance(): Double = toJavaDoubleRDD().popVariance() /** * Approximate operation to return the mean within a timeout. */ -fun JavaRDD.meanApprox( +inline fun JavaRDD.meanApprox( timeout: Long, confidence: Double = 0.95, ): PartialResult = toJavaDoubleRDD().meanApprox(timeout, confidence) @@ -67,7 +72,7 @@ fun JavaRDD.meanApprox( /** * Approximate operation to return the sum within a timeout. */ -fun JavaRDD.sumApprox( +inline fun JavaRDD.sumApprox( timeout: Long, confidence: Double = 0.95, ): PartialResult = toJavaDoubleRDD().sumApprox(timeout, confidence) @@ -80,7 +85,7 @@ fun JavaRDD.sumApprox( * If the RDD contains infinity, NaN throws an exception * If the elements in RDD do not vary (max == min) always returns a single bucket. */ -fun JavaRDD.histogram(bucketCount: Int): Tuple2 = +inline fun JavaRDD.histogram(bucketCount: Int): Tuple2 = toJavaDoubleRDD().histogram(bucketCount) /** @@ -100,7 +105,7 @@ fun JavaRDD.histogram(bucketCount: Int): Tuple2 * the maximum value of the last position and all NaN entries will be counted * in that bucket. */ -fun JavaRDD.histogram( +inline fun JavaRDD.histogram( buckets: Array, evenBuckets: Boolean = false, ): LongArray = toJavaDoubleRDD().histogram(buckets, evenBuckets) \ No newline at end of file diff --git a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddKeyValue.kt b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddKeyValue.kt index 5bc86fde..3d6c5c80 100644 --- a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddKeyValue.kt +++ b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddKeyValue.kt @@ -20,7 +20,7 @@ import scala.Tuple4 import kotlin.random.Random import org.apache.hadoop.mapreduce.OutputFormat as NewOutputFormat -fun JavaRDD>.toPairRDD(): JavaPairRDD = +fun JavaRDD>.toJavaPairRDD(): JavaPairRDD = JavaPairRDD.fromJavaRDD(this) fun JavaPairRDD.toTupleRDD(): JavaRDD> = @@ -38,7 +38,7 @@ fun JavaRDD>.combineByKey( partitioner: Partitioner, mapSideCombine: Boolean = true, serializer: Serializer? = null, -): JavaRDD> = toPairRDD() +): JavaRDD> = toJavaPairRDD() .combineByKey(createCombiner, mergeValue, mergeCombiners, partitioner, mapSideCombine, serializer) .toTupleRDD() @@ -52,7 +52,7 @@ fun JavaRDD>.combineByKey( mergeValue: (C, V) -> C, mergeCombiners: (C, C) -> C, numPartitions: Int, -): JavaRDD> = toPairRDD() +): JavaRDD> = toJavaPairRDD() .combineByKey(createCombiner, mergeValue, mergeCombiners, numPartitions) .toTupleRDD() @@ -69,10 +69,10 @@ fun JavaRDD>.combineByKey( fun JavaRDD>.aggregateByKey( zeroValue: U, partitioner: Partitioner, - seqOp: (U, V) -> U, - combOp: (U, U) -> U, -): JavaRDD> = toPairRDD() - .aggregateByKey(zeroValue, partitioner, seqOp, combOp) + seqFunc: (U, V) -> U, + combFunc: (U, U) -> U, +): JavaRDD> = toJavaPairRDD() + .aggregateByKey(zeroValue, partitioner, seqFunc, combFunc) .toTupleRDD() /** @@ -87,10 +87,10 @@ fun JavaRDD>.aggregateByKey( fun JavaRDD>.aggregateByKey( zeroValue: U, numPartitions: Int, - seqOp: (U, V) -> U, - combOp: (U, U) -> U, -): JavaRDD> = toPairRDD() - .aggregateByKey(zeroValue, numPartitions, seqOp, combOp) + seqFunc: (U, V) -> U, + combFunc: (U, U) -> U, +): JavaRDD> = toJavaPairRDD() + .aggregateByKey(zeroValue, numPartitions, seqFunc, combFunc) .toTupleRDD() /** @@ -104,10 +104,10 @@ fun JavaRDD>.aggregateByKey( */ fun JavaRDD>.aggregateByKey( zeroValue: U, - seqOp: (U, V) -> U, - combOp: (U, U) -> U, -): JavaRDD> = toPairRDD() - .aggregateByKey(zeroValue, seqOp, combOp) + seqFunc: (U, V) -> U, + combFunc: (U, U) -> U, +): JavaRDD> = toJavaPairRDD() + .aggregateByKey(zeroValue, seqFunc, combFunc) .toTupleRDD() /** @@ -119,7 +119,7 @@ fun JavaRDD>.foldByKey( zeroValue: V, partitioner: Partitioner, func: (V, V) -> V, -): JavaRDD> = toPairRDD() +): JavaRDD> = toJavaPairRDD() .foldByKey(zeroValue, partitioner, func) .toTupleRDD() @@ -132,7 +132,7 @@ fun JavaRDD>.foldByKey( zeroValue: V, numPartitions: Int, func: (V, V) -> V, -): JavaRDD> = toPairRDD() +): JavaRDD> = toJavaPairRDD() .foldByKey(zeroValue, numPartitions, func) .toTupleRDD() @@ -144,7 +144,7 @@ fun JavaRDD>.foldByKey( fun JavaRDD>.foldByKey( zeroValue: V, func: (V, V) -> V, -): JavaRDD> = toPairRDD() +): JavaRDD> = toJavaPairRDD() .foldByKey(zeroValue, func) .toTupleRDD() @@ -165,7 +165,7 @@ fun JavaRDD>.sampleByKey( withReplacement: Boolean, fractions: Map, seed: Long = Random.nextLong(), -): JavaRDD> = toPairRDD() +): JavaRDD> = toJavaPairRDD() .sampleByKey(withReplacement, fractions, seed) .toTupleRDD() @@ -188,7 +188,7 @@ fun JavaRDD>.sampleByKeyExact( withReplacement: Boolean, fractions: Map, seed: Long = Random.nextLong(), -): JavaRDD> = toPairRDD() +): JavaRDD> = toJavaPairRDD() .sampleByKeyExact(withReplacement, fractions, seed) .toTupleRDD() @@ -200,7 +200,7 @@ fun JavaRDD>.sampleByKeyExact( fun JavaRDD>.reduceByKey( partitioner: Partitioner, func: (V, V) -> V, -): JavaRDD> = toPairRDD() +): JavaRDD> = toJavaPairRDD() .reduceByKey(partitioner, func) .toTupleRDD() @@ -212,7 +212,7 @@ fun JavaRDD>.reduceByKey( fun JavaRDD>.reduceByKey( numPartitions: Int, func: (V, V) -> V, -): JavaRDD> = toPairRDD() +): JavaRDD> = toJavaPairRDD() .reduceByKey(func, numPartitions) .toTupleRDD() @@ -224,7 +224,7 @@ fun JavaRDD>.reduceByKey( */ fun JavaRDD>.reduceByKey( func: (V, V) -> V, -): JavaRDD> = toPairRDD() +): JavaRDD> = toJavaPairRDD() .reduceByKey(func) .toTupleRDD() @@ -235,7 +235,7 @@ fun JavaRDD>.reduceByKey( */ fun JavaRDD>.reduceByKeyLocally( func: (V, V) -> V, -): Map = toPairRDD() +): Map = toJavaPairRDD() .reduceByKeyLocally(func) /** @@ -247,7 +247,7 @@ fun JavaRDD>.reduceByKeyLocally( * returns an [RDD] instead of a map. */ fun JavaRDD>.countByKey(): Map = - toPairRDD() + toJavaPairRDD() .countByKey() /** @@ -267,7 +267,7 @@ fun JavaRDD>.countByKey(): Map = fun JavaRDD>.countByKeyApprox( timeout: Long, confidence: Double = 0.95, -): PartialResult> = toPairRDD() +): PartialResult> = toJavaPairRDD() .countByKeyApprox(timeout, confidence) /** @@ -285,7 +285,7 @@ fun JavaRDD>.countByKeyApprox( */ fun JavaRDD>.groupByKey( partitioner: Partitioner, -): JavaRDD>> = toPairRDD() +): JavaRDD>> = toJavaPairRDD() .groupByKey(partitioner) .toTupleRDD() @@ -303,7 +303,7 @@ fun JavaRDD>.groupByKey( */ fun JavaRDD>.groupByKey( numPartitions: Int, -): JavaRDD>> = toPairRDD() +): JavaRDD>> = toJavaPairRDD() .groupByKey(numPartitions) .toTupleRDD() @@ -312,7 +312,7 @@ fun JavaRDD>.groupByKey( */ fun JavaRDD>.partitionBy( partitioner: Partitioner, -): JavaRDD> = toPairRDD() +): JavaRDD> = toJavaPairRDD() .partitionBy(partitioner) .toTupleRDD() @@ -324,8 +324,8 @@ fun JavaRDD>.partitionBy( fun JavaRDD>.join( other: JavaRDD>, partitioner: Partitioner, -): JavaRDD>> = toPairRDD() - .join(other.toPairRDD(), partitioner) +): JavaRDD>> = toJavaPairRDD() + .join(other.toJavaPairRDD(), partitioner) .toTupleRDD() /** @@ -337,8 +337,8 @@ fun JavaRDD>.join( fun JavaRDD>.leftOuterJoin( other: JavaRDD>, partitioner: Partitioner, -): JavaRDD>>> = toPairRDD() - .leftOuterJoin(other.toPairRDD(), partitioner) +): JavaRDD>>> = toJavaPairRDD() + .leftOuterJoin(other.toJavaPairRDD(), partitioner) .toTupleRDD() /** @@ -350,8 +350,8 @@ fun JavaRDD>.leftOuterJoin( fun JavaRDD>.rightOuterJoin( other: JavaRDD>, partitioner: Partitioner, -): JavaRDD, W>>> = toPairRDD() - .rightOuterJoin(other.toPairRDD(), partitioner) +): JavaRDD, W>>> = toJavaPairRDD() + .rightOuterJoin(other.toJavaPairRDD(), partitioner) .toTupleRDD() /** @@ -365,8 +365,8 @@ fun JavaRDD>.rightOuterJoin( fun JavaRDD>.fullOuterJoin( other: JavaRDD>, partitioner: Partitioner, -): JavaRDD, Optional>>> = toPairRDD() - .fullOuterJoin(other.toPairRDD(), partitioner) +): JavaRDD, Optional>>> = toJavaPairRDD() + .fullOuterJoin(other.toJavaPairRDD(), partitioner) .toTupleRDD() /** @@ -378,7 +378,7 @@ fun JavaRDD>.combineByKey( createCombiner: (V) -> C, mergeValue: (C, V) -> C, mergeCombiners: (C, C) -> C, -): JavaRDD> = toPairRDD() +): JavaRDD> = toJavaPairRDD() .combineByKey(createCombiner, mergeValue, mergeCombiners) .toTupleRDD() @@ -394,7 +394,7 @@ fun JavaRDD>.combineByKey( * or [reduceByKey] will provide much better performance. */ fun JavaRDD>.groupByKey(): JavaRDD>> = - toPairRDD() + toJavaPairRDD() .groupByKey() .toTupleRDD() @@ -404,8 +404,8 @@ fun JavaRDD>.groupByKey(): JavaRDD>> = * (k, v2) is in [other]. Performs a hash join across the cluster. */ fun JavaRDD>.join(other: JavaRDD>): JavaRDD>> = - toPairRDD() - .join(other.toPairRDD()) + toJavaPairRDD() + .join(other.toJavaPairRDD()) .toTupleRDD() /** @@ -417,8 +417,8 @@ fun JavaRDD>.join( other: JavaRDD>, numPartitions: Int, ): JavaRDD>> = - toPairRDD() - .join(other.toPairRDD(), numPartitions) + toJavaPairRDD() + .join(other.toJavaPairRDD(), numPartitions) .toTupleRDD() /** @@ -430,8 +430,8 @@ fun JavaRDD>.join( fun JavaRDD>.leftOuterJoin( other: JavaRDD>, ): JavaRDD>>> = - toPairRDD() - .leftOuterJoin(other.toPairRDD()) + toJavaPairRDD() + .leftOuterJoin(other.toJavaPairRDD()) .toTupleRDD() /** @@ -443,8 +443,8 @@ fun JavaRDD>.leftOuterJoin( fun JavaRDD>.leftOuterJoin( other: JavaRDD>, numPartitions: Int, -): JavaRDD>>> = toPairRDD() - .leftOuterJoin(other.toPairRDD(), numPartitions) +): JavaRDD>>> = toJavaPairRDD() + .leftOuterJoin(other.toJavaPairRDD(), numPartitions) .toTupleRDD() /** @@ -456,8 +456,8 @@ fun JavaRDD>.leftOuterJoin( fun JavaRDD>.rightOuterJoin( other: JavaRDD>, ): JavaRDD, W>>> = - toPairRDD() - .rightOuterJoin(other.toPairRDD()) + toJavaPairRDD() + .rightOuterJoin(other.toJavaPairRDD()) .toTupleRDD() /** @@ -469,8 +469,8 @@ fun JavaRDD>.rightOuterJoin( fun JavaRDD>.rightOuterJoin( other: JavaRDD>, numPartitions: Int, -): JavaRDD, W>>> = toPairRDD() - .rightOuterJoin(other.toPairRDD(), numPartitions) +): JavaRDD, W>>> = toJavaPairRDD() + .rightOuterJoin(other.toJavaPairRDD(), numPartitions) .toTupleRDD() /** @@ -485,8 +485,8 @@ fun JavaRDD>.rightOuterJoin( fun JavaRDD>.fullOuterJoin( other: JavaRDD>, ): JavaRDD, Optional>>> = - toPairRDD() - .fullOuterJoin(other.toPairRDD()) + toJavaPairRDD() + .fullOuterJoin(other.toJavaPairRDD()) .toTupleRDD() /** @@ -501,8 +501,8 @@ fun JavaRDD>.fullOuterJoin( other: JavaRDD>, numPartitions: Int, ): JavaRDD, Optional>>> = - toPairRDD() - .fullOuterJoin(other.toPairRDD(), numPartitions) + toJavaPairRDD() + .fullOuterJoin(other.toJavaPairRDD(), numPartitions) .toTupleRDD() /** @@ -514,21 +514,21 @@ fun JavaRDD>.fullOuterJoin( * 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. */ -fun JavaRDD>.collectAsMap(): Map = toPairRDD().collectAsMap() +fun JavaRDD>.collectAsMap(): Map = toJavaPairRDD().collectAsMap() /** * 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. */ fun JavaRDD>.mapValues(f: (V) -> U): JavaRDD> = - toPairRDD().mapValues(f).toTupleRDD() + toJavaPairRDD().mapValues(f).toTupleRDD() /** * 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. */ fun JavaRDD>.flatMapValues(f: (V) -> Iterator): JavaRDD> = - toPairRDD().flatMapValues(f).toTupleRDD() + toJavaPairRDD().flatMapValues(f).toTupleRDD() /** * For each key k in [this] or [other1] or [other2] or [other3], @@ -541,7 +541,7 @@ fun JavaRDD>.cogroup( other3: JavaRDD>, partitioner: Partitioner, ): JavaRDD, Iterable, Iterable, Iterable>>> = - toPairRDD().cogroup(other1.toPairRDD(), other2.toPairRDD(), other3.toPairRDD(), partitioner).toTupleRDD() + toJavaPairRDD().cogroup(other1.toJavaPairRDD(), other2.toJavaPairRDD(), other3.toJavaPairRDD(), partitioner).toTupleRDD() /** * For each key k in [this] or [other], return a resulting RDD that contains a tuple with the @@ -550,8 +550,8 @@ fun JavaRDD>.cogroup( fun JavaRDD>.cogroup( other: JavaRDD>, partitioner: Partitioner, -): JavaRDD, Iterable>>> = toPairRDD() - .cogroup(other.toPairRDD(), partitioner) +): JavaRDD, Iterable>>> = toJavaPairRDD() + .cogroup(other.toJavaPairRDD(), partitioner) .toTupleRDD() /** @@ -563,8 +563,8 @@ fun JavaRDD>.cogroup( other2: JavaRDD>, partitioner: Partitioner, ): JavaRDD, Iterable, Iterable>>> = - toPairRDD() - .cogroup(other1.toPairRDD(), other2.toPairRDD(), partitioner) + toJavaPairRDD() + .cogroup(other1.toJavaPairRDD(), other2.toJavaPairRDD(), partitioner) .toTupleRDD() /** @@ -577,8 +577,8 @@ fun JavaRDD>.cogroup( other2: JavaRDD>, other3: JavaRDD>, ): JavaRDD, Iterable, Iterable, Iterable>>> = - toPairRDD() - .cogroup(other1.toPairRDD(), other2.toPairRDD(), other3.toPairRDD()) + toJavaPairRDD() + .cogroup(other1.toJavaPairRDD(), other2.toJavaPairRDD(), other3.toJavaPairRDD()) .toTupleRDD() /** @@ -588,7 +588,7 @@ fun JavaRDD>.cogroup( fun JavaRDD>.cogroup( other: JavaRDD>, ): JavaRDD, Iterable>>> = - toPairRDD().cogroup(other.toPairRDD()).toTupleRDD() + toJavaPairRDD().cogroup(other.toJavaPairRDD()).toTupleRDD() /** * For each key k in [this] or [other1] or [other2], return a resulting RDD that contains a @@ -598,8 +598,8 @@ fun JavaRDD>.cogroup( other1: JavaRDD>, other2: JavaRDD>, ): JavaRDD, Iterable, Iterable>>> = - toPairRDD() - .cogroup(other1.toPairRDD(), other2.toPairRDD()) + toJavaPairRDD() + .cogroup(other1.toJavaPairRDD(), other2.toJavaPairRDD()) .toTupleRDD() /** @@ -610,7 +610,7 @@ fun JavaRDD>.cogroup( other: JavaRDD>, numPartitions: Int, ): JavaRDD, Iterable>>> = - toPairRDD().cogroup(other.toPairRDD(), numPartitions).toTupleRDD() + toJavaPairRDD().cogroup(other.toJavaPairRDD(), numPartitions).toTupleRDD() /** * For each key k in [this] or [other1] or [other2], return a resulting RDD that contains a @@ -621,8 +621,8 @@ fun JavaRDD>.cogroup( other2: JavaRDD>, numPartitions: Int, ): JavaRDD, Iterable, Iterable>>> = - toPairRDD() - .cogroup(other1.toPairRDD(), other2.toPairRDD(), numPartitions) + toJavaPairRDD() + .cogroup(other1.toJavaPairRDD(), other2.toJavaPairRDD(), numPartitions) .toTupleRDD() /** @@ -636,8 +636,8 @@ fun JavaRDD>.cogroup( other3: JavaRDD>, numPartitions: Int, ): JavaRDD, Iterable, Iterable, Iterable>>> = - toPairRDD() - .cogroup(other1.toPairRDD(), other2.toPairRDD(), other3.toPairRDD(), numPartitions) + toJavaPairRDD() + .cogroup(other1.toJavaPairRDD(), other2.toJavaPairRDD(), other3.toJavaPairRDD(), numPartitions) .toTupleRDD() @@ -645,14 +645,14 @@ fun JavaRDD>.cogroup( fun JavaRDD>.groupWith( other: JavaRDD>, ): JavaRDD, Iterable>>> = - toPairRDD().groupWith(other.toPairRDD()).toTupleRDD() + toJavaPairRDD().groupWith(other.toJavaPairRDD()).toTupleRDD() /** Alias for [cogroup]. */ fun JavaRDD>.groupWith( other1: JavaRDD>, other2: JavaRDD>, ): JavaRDD, Iterable, Iterable>>> = - toPairRDD().groupWith(other1.toPairRDD(), other2.toPairRDD()).toTupleRDD() + toJavaPairRDD().groupWith(other1.toJavaPairRDD(), other2.toJavaPairRDD()).toTupleRDD() /** Alias for [cogroup]. */ fun JavaRDD>.groupWith( @@ -660,7 +660,7 @@ fun JavaRDD>.groupWith( other2: JavaRDD>, other3: JavaRDD>, ): JavaRDD, Iterable, Iterable, Iterable>>> = - toPairRDD().groupWith(other1.toPairRDD(), other2.toPairRDD(), other3.toPairRDD()).toTupleRDD() + toJavaPairRDD().groupWith(other1.toJavaPairRDD(), other2.toJavaPairRDD(), other3.toJavaPairRDD()).toTupleRDD() /** * Return an RDD with the pairs from [this] whose keys are not in [other]. @@ -669,7 +669,7 @@ fun JavaRDD>.groupWith( * RDD will be less than or equal to us. */ fun JavaRDD>.subtractByKey(other: JavaRDD>): JavaRDD> = - toPairRDD().subtractByKey(other.toPairRDD()).toTupleRDD() + toJavaPairRDD().subtractByKey(other.toJavaPairRDD()).toTupleRDD() /** * Return an RDD with the pairs from [this] whose keys are not in [other]. @@ -677,8 +677,8 @@ fun JavaRDD>.subtractByKey(other: JavaRDD>): fun JavaRDD>.subtractByKey( other: JavaRDD>, numPartitions: Int, -): JavaRDD> = toPairRDD() - .subtractByKey(other.toPairRDD(), numPartitions) +): JavaRDD> = toJavaPairRDD() + .subtractByKey(other.toJavaPairRDD(), numPartitions) .toTupleRDD() /** @@ -687,15 +687,15 @@ fun JavaRDD>.subtractByKey( fun JavaRDD>.subtractByKey( other: JavaRDD>, p: Partitioner, -): JavaRDD> = toPairRDD() - .subtractByKey(other.toPairRDD(), p) +): JavaRDD> = toJavaPairRDD() + .subtractByKey(other.toJavaPairRDD(), p) .toTupleRDD() /** * Return the list of values in the RDD for key [key]. This operation is done efficiently if the * RDD has a known partitioner by only searching the partition that the key maps to. */ -fun JavaRDD>.lookup(key: K): List = toPairRDD().lookup(key) +fun JavaRDD>.lookup(key: K): List = toJavaPairRDD().lookup(key) /** Output the RDD to any Hadoop-supported file system. */ fun > JavaRDD>.saveAsHadoopFile( @@ -704,7 +704,7 @@ fun > JavaRDD>.saveAsHadoopFile( valueClass: Class<*>, outputFormatClass: Class, conf: JobConf, -): Unit = toPairRDD().saveAsHadoopFile(path, keyClass, valueClass, outputFormatClass, conf) +): Unit = toJavaPairRDD().saveAsHadoopFile(path, keyClass, valueClass, outputFormatClass, conf) /** Output the RDD to any Hadoop-supported file system. */ fun > JavaRDD>.saveAsHadoopFile( @@ -712,7 +712,7 @@ fun > JavaRDD>.saveAsHadoopFile( keyClass: Class<*>, valueClass: Class<*>, outputFormatClass: Class, -): Unit = toPairRDD().saveAsHadoopFile(path, keyClass, valueClass, outputFormatClass) +): Unit = toJavaPairRDD().saveAsHadoopFile(path, keyClass, valueClass, outputFormatClass) /** Output the RDD to any Hadoop-supported file system, compressing with the supplied codec. */ fun > JavaRDD>.saveAsHadoopFile( @@ -721,7 +721,7 @@ fun > JavaRDD>.saveAsHadoopFile( valueClass: Class<*>, outputFormatClass: Class, codec: Class, -): Unit = toPairRDD().saveAsHadoopFile(path, keyClass, valueClass, outputFormatClass, codec) +): Unit = toJavaPairRDD().saveAsHadoopFile(path, keyClass, valueClass, outputFormatClass, codec) /** Output the RDD to any Hadoop-supported file system. */ fun > JavaRDD>.saveAsNewAPIHadoopFile( @@ -730,14 +730,14 @@ fun > JavaRDD>.saveAsNewAPIHadoopFi valueClass: Class<*>, outputFormatClass: Class, conf: Configuration, -): Unit = toPairRDD().saveAsNewAPIHadoopFile(path, keyClass, valueClass, outputFormatClass, conf) +): Unit = toJavaPairRDD().saveAsNewAPIHadoopFile(path, keyClass, valueClass, outputFormatClass, conf) /** * Output the RDD to any Hadoop-supported storage system, using * a Configuration object for that storage system. */ fun JavaRDD>.saveAsNewAPIHadoopDataset(conf: Configuration): Unit = - toPairRDD().saveAsNewAPIHadoopDataset(conf) + toJavaPairRDD().saveAsNewAPIHadoopDataset(conf) /** Output the RDD to any Hadoop-supported file system. */ fun > JavaRDD>.saveAsNewAPIHadoopFile( @@ -745,7 +745,7 @@ fun > JavaRDD>.saveAsNewAPIHadoopFi keyClass: Class<*>, valueClass: Class<*>, outputFormatClass: Class, -): Unit = toPairRDD().saveAsNewAPIHadoopFile(path, keyClass, valueClass, outputFormatClass) +): Unit = toJavaPairRDD().saveAsNewAPIHadoopFile(path, keyClass, valueClass, outputFormatClass) /** * Output the RDD to any Hadoop-supported storage system, using a Hadoop JobConf object for @@ -754,7 +754,7 @@ fun > JavaRDD>.saveAsNewAPIHadoopFi * MapReduce job. */ fun JavaRDD>.saveAsHadoopDataset(conf: JobConf): Unit = - toPairRDD().saveAsHadoopDataset(conf) + toJavaPairRDD().saveAsHadoopDataset(conf) /** * Repartition the RDD according to the given partitioner and, within each resulting partition, @@ -764,7 +764,7 @@ fun JavaRDD>.saveAsHadoopDataset(conf: JobConf): Unit = * because it can push the sorting down into the shuffle machinery. */ fun JavaRDD>.repartitionAndSortWithinPartitions(partitioner: Partitioner): JavaRDD> = - toPairRDD().repartitionAndSortWithinPartitions(partitioner).toTupleRDD() + toJavaPairRDD().repartitionAndSortWithinPartitions(partitioner).toTupleRDD() /** * Repartition the RDD according to the given partitioner and, within each resulting partition, @@ -776,7 +776,7 @@ fun JavaRDD>.repartitionAndSortWithinPartitions(partitioner: fun JavaRDD>.repartitionAndSortWithinPartitions( partitioner: Partitioner, comp: Comparator, -): JavaRDD> = toPairRDD().repartitionAndSortWithinPartitions(partitioner, comp).toTupleRDD() +): JavaRDD> = toJavaPairRDD().repartitionAndSortWithinPartitions(partitioner, comp).toTupleRDD() /** * Sort the RDD by key, so that each partition contains a sorted range of the elements. Calling @@ -785,7 +785,7 @@ fun JavaRDD>.repartitionAndSortWithinPartitions( * order of the keys). */ fun JavaRDD>.sortByKey(ascending: Boolean = true): JavaRDD> = - toPairRDD().sortByKey(ascending).toTupleRDD() + toJavaPairRDD().sortByKey(ascending).toTupleRDD() /** * Sort the RDD by key, so that each partition contains a sorted range of the elements. Calling @@ -794,7 +794,7 @@ fun JavaRDD>.sortByKey(ascending: Boolean = true): JavaRDD JavaRDD>.sortByKey(ascending: Boolean, numPartitions: Int): JavaRDD> = - toPairRDD().sortByKey(ascending, numPartitions).toTupleRDD() + toJavaPairRDD().sortByKey(ascending, numPartitions).toTupleRDD() /** * Sort the RDD by key, so that each partition contains a sorted range of the elements. Calling @@ -803,7 +803,7 @@ fun JavaRDD>.sortByKey(ascending: Boolean, numPartitions: In * order of the keys). */ fun JavaRDD>.sortByKey(comp: Comparator, ascending: Boolean = true): JavaRDD> = - toPairRDD().sortByKey(comp, ascending).toTupleRDD() + toJavaPairRDD().sortByKey(comp, ascending).toTupleRDD() /** * Sort the RDD by key, so that each partition contains a sorted range of the elements. Calling @@ -815,7 +815,7 @@ fun JavaRDD>.sortByKey( comp: Comparator, ascending: Boolean, numPartitions: Int, -): JavaRDD> = toPairRDD().sortByKey(comp, ascending, numPartitions).toTupleRDD() +): JavaRDD> = toJavaPairRDD().sortByKey(comp, ascending, numPartitions).toTupleRDD() //#if sparkMinor >= 3.1 /** @@ -827,7 +827,7 @@ fun JavaRDD>.sortByKey( * @since 3.1.0 */ fun JavaRDD>.filterByRange(lower: K, upper: K): JavaRDD> = - toPairRDD().filterByRange(lower, upper).toTupleRDD() + toJavaPairRDD().filterByRange(lower, upper).toTupleRDD() /** * Return a RDD containing only the elements in the inclusive range [lower] to [upper]. @@ -841,7 +841,7 @@ fun JavaRDD>.filterByRange( comp: Comparator, lower: K, upper: K, -): JavaRDD> = toPairRDD() +): JavaRDD> = toJavaPairRDD() .filterByRange(comp, lower, upper) .toTupleRDD() //#endif @@ -849,12 +849,12 @@ fun JavaRDD>.filterByRange( /** * Return an RDD with the keys of each tuple. */ -fun JavaRDD>.keys(): JavaRDD = toPairRDD().keys() +fun JavaRDD>.keys(): JavaRDD = toJavaPairRDD().keys() /** * Return an RDD with the values of each tuple. */ -fun JavaRDD>.values(): JavaRDD = toPairRDD().values() +fun JavaRDD>.values(): JavaRDD = toJavaPairRDD().values() /** * Return approximate number of distinct values for each key in this RDD. @@ -870,7 +870,7 @@ fun JavaRDD>.values(): JavaRDD = toPairRDD().values() fun JavaRDD>.countApproxDistinctByKey( relativeSD: Double, partitioner: Partitioner, -): JavaRDD> = toPairRDD().countApproxDistinctByKey(relativeSD, partitioner).toTupleRDD() +): JavaRDD> = toJavaPairRDD().countApproxDistinctByKey(relativeSD, partitioner).toTupleRDD() /** * Return approximate number of distinct values for each key in this RDD. @@ -886,7 +886,7 @@ fun JavaRDD>.countApproxDistinctByKey( fun JavaRDD>.countApproxDistinctByKey( relativeSD: Double, numPartitions: Int, -): JavaRDD> = toPairRDD().countApproxDistinctByKey(relativeSD, numPartitions).toTupleRDD() +): JavaRDD> = toJavaPairRDD().countApproxDistinctByKey(relativeSD, numPartitions).toTupleRDD() /** * Return approximate number of distinct values for each key in this RDD. @@ -899,4 +899,5 @@ fun JavaRDD>.countApproxDistinctByKey( * It must be greater than 0.000017. */ fun JavaRDD>.countApproxDistinctByKey(relativeSD: Double): JavaRDD> = - toPairRDD().countApproxDistinctByKey(relativeSD).toTupleRDD() \ No newline at end of file + toJavaPairRDD().countApproxDistinctByKey(relativeSD).toTupleRDD() + diff --git a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt index 95cc4a09..f9a8b079 100644 --- a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt +++ b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt @@ -106,14 +106,14 @@ class KSparkSession(val spark: SparkSession) { * Utility method to create an RDD from a list. * NOTE: [T] must be [Serializable]. */ - inline fun List.toRDD(numSlices: Int = sc.defaultParallelism()): JavaRDD = + fun List.toRDD(numSlices: Int = sc.defaultParallelism()): JavaRDD = sc.toRDD(this, numSlices) /** * Utility method to create an RDD from a list. * NOTE: [T] must be [Serializable]. */ - inline fun rddOf(vararg elements: T, numSlices: Int = sc.defaultParallelism()): JavaRDD = + fun rddOf(vararg elements: T, numSlices: Int = sc.defaultParallelism()): JavaRDD = sc.toRDD(elements.toList(), numSlices) /** diff --git a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/StreamingKeyValues.kt b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/StreamingKeyValues.kt index f0066d70..19b4ea79 100644 --- a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/StreamingKeyValues.kt +++ b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/StreamingKeyValues.kt @@ -440,7 +440,7 @@ fun JavaDStream>.updateStateByKey( updateFunc(list, s.getOrNull()).toOptional() }, partitioner, - initialRDD.toPairRDD(), + initialRDD.toJavaPairRDD(), ) .toTupleDStream() @@ -465,7 +465,7 @@ fun JavaDStream>.updateStateByKey( .updateStateByKey( updateFunc, partitioner, - initialRDD.toPairRDD(), + initialRDD.toJavaPairRDD(), ) .toTupleDStream() diff --git a/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/RddTest.kt b/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/RddTest.kt index b1a3e36e..21351124 100644 --- a/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/RddTest.kt +++ b/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/RddTest.kt @@ -1,9 +1,7 @@ package org.jetbrains.kotlinx.spark.api import io.kotest.core.spec.style.ShouldSpec -import io.kotest.matchers.collections.shouldContain import io.kotest.matchers.collections.shouldContainAll -import io.kotest.matchers.should import io.kotest.matchers.shouldBe import org.jetbrains.kotlinx.spark.api.tuples.* @@ -14,8 +12,7 @@ class RddTest : ShouldSpec({ context("Key/value") { should("work with spark example") { - val data = listOf(1, 1, 2, 2, 2, 3).map(Int::toString) - val rdd = sc.parallelize(data) + val rdd = rddOf(1, 1, 2, 2, 2, 3).map(Int::toString) val pairs = rdd.map { it X 1 } val counts = pairs.reduceByKey { a, b -> a + b } @@ -24,13 +21,52 @@ class RddTest : ShouldSpec({ } } - context("Double") { + context("Double functions") { should("get max/min") { - val data = listOf(1, 1, 2, 2, 2, 3).map(Int::toDouble) - val rdd = sc.parallelize(data) + val rdd = rddOf(1, 1, 2, 2, 2, 3) + rdd.max() shouldBe 3.0 rdd.min() shouldBe 1.0 } + + context("Work with any number") { + + should("Work with Bytes") { + val data = listOf(1, 1, 2, 2, 2, 3).map(Int::toByte) + val rdd = data.toRDD() + rdd.sum() shouldBe data.sum().toDouble() + } + + should("Work with Shorts") { + val data = listOf(1, 1, 2, 2, 2, 3).map(Int::toShort) + val rdd = data.toRDD() + rdd.sum() shouldBe data.sum().toDouble() + } + + should("Work with Ints") { + val data = listOf(1, 1, 2, 2, 2, 3) + val rdd = data.toRDD() + rdd.sum() shouldBe data.sum().toDouble() + } + + should("Work with Longs") { + val data = listOf(1, 1, 2, 2, 2, 3).map(Int::toLong) + val rdd = data.toRDD() + rdd.sum() shouldBe data.sum().toDouble() + } + + should("Work with Floats") { + val data = listOf(1, 1, 2, 2, 2, 3).map(Int::toFloat) + val rdd = data.toRDD() + rdd.sum() shouldBe data.sum().toDouble() + } + + should("Work with Doubles") { + val data = listOf(1, 1, 2, 2, 2, 3).map(Int::toDouble) + val rdd = data.toRDD().toJavaDoubleRDD() + rdd.sum() shouldBe data.sum().toDouble() + } + } } } } From 1626760cd194b7593de5c479ea2d7f2838c5231a Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Mon, 18 Jul 2022 17:43:46 +0200 Subject: [PATCH 10/16] updated jupyter example --- .../spark/examples/JupyterExample.ipynb | 27 +++++++++++++------ 1 file changed, 19 insertions(+), 8 deletions(-) diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/JupyterExample.ipynb b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/JupyterExample.ipynb index 33d9e27b..5df9c58e 100644 --- a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/JupyterExample.ipynb +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/JupyterExample.ipynb @@ -3,8 +3,8 @@ { "cell_type": "markdown", "source": [ - "By default the latest version of the API and the latest supported Spark version is chosen.\n", - "To specify your own: `%use spark(spark=3.2, v=1.1.0)`" + "By default, the latest version of the API and the latest supported Spark version is chosen.\n", + "To specify your own: `%use spark(spark=3.3.0, scala=2.13, v=1.2.0)`" ], "metadata": { "collapsed": false, @@ -35,6 +35,18 @@ } } }, + { + "cell_type": "code", + "execution_count": null, + "outputs": [], + "source": [], + "metadata": { + "collapsed": false, + "pycharm": { + "name": "#%%\n" + } + } + }, { "cell_type": "markdown", "source": [ @@ -312,14 +324,13 @@ } ], "source": [ - "val rdd: JavaRDD> = sc.parallelize(\n", - " listOf(\n", - " 1 X \"aaa\",\n", - " t(2, \"bbb\"),\n", - " tupleOf(3, \"ccc\"),\n", - " )\n", + "val rdd: JavaRDD> = rddOf(\n", + " 1 X \"aaa\",\n", + " t(2, \"bbb\"),\n", + " tupleOf(3, \"ccc\"),\n", ")\n", "\n", + "\n", "rdd" ], "metadata": { From e94c2ac09a86dfe693f80317ab5cad8f4ef20c7c Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Wed, 20 Jul 2022 16:41:05 +0200 Subject: [PATCH 11/16] improved rdd group calculation example. added transformAsSequence and other iterator tests --- .../spark/examples/RddGroupCalculation.kt | 149 ++++++++---------- .../jetbrains/kotlinx/spark/api/Iterators.kt | 16 +- .../jetbrains/kotlinx/spark/api/ApiTest.kt | 56 +++++++ 3 files changed, 139 insertions(+), 82 deletions(-) diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/RddGroupCalculation.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/RddGroupCalculation.kt index 134603ed..8f64f793 100644 --- a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/RddGroupCalculation.kt +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/RddGroupCalculation.kt @@ -5,7 +5,6 @@ import org.jetbrains.kotlinx.spark.api.* import org.jetbrains.kotlinx.spark.api.tuples.X import org.jetbrains.kotlinx.spark.examples.GroupCalculation.getAllPossibleGroups import scala.Tuple2 -import java.util.* import kotlin.math.pow /** @@ -15,8 +14,9 @@ import kotlin.math.pow */ fun main() = withSpark { - val groupIndices = getAllPossibleGroups(listSize = 10, groupSize = 3) + val groupIndices = getAllPossibleGroups(listSize = 10, groupSize = 4) .sort("value") + groupIndices.showDS(numRows = groupIndices.count().toInt()) } @@ -37,7 +37,7 @@ object GroupCalculation { listSize: Int, groupSize: Int, ): Dataset { - val indices = (0 until listSize).toList().toRDD() + val indices = (0 until listSize).toList().toRDD() // Easy RDD creation! // for a groupSize of 1, no pairing up is needed, so just return the indices converted to IntArrays if (groupSize == 1) { @@ -46,61 +46,57 @@ object GroupCalculation { it.map { intArrayOf(it) } } .toDS() - } - - val keys = indices - .mapPartitions { // this converts all indices to (number in table, index) - // key is key (item in table), value is index in list - val keyValues = ArrayList>() - while (it.hasNext()) { - val listIndex = it.next() + // this converts all indices to (number in table, index) + val keys = indices.mapPartitions { + + // _1 is key (item in table), _2 is index in list + it.transformAsSequence { + flatMap { listIndex -> // for each dimension loop over the other dimensions using addTuples - for (dimension in 0 until groupSize) { + (0 until groupSize).asSequence().flatMap { dimension -> addTuples( groupSize = groupSize, - keyValues = keyValues, value = listIndex, listSize = listSize, - currentDimension = 0, - indexTuple = IntArray(groupSize), skipDimension = dimension, ) } } - keyValues.iterator() } + } - // each number in table occurs for each dimension as key. + // Since we have a JavaRDD we can aggregateByKey! + // Each number in table occurs for each dimension as key. // The values of those two will be a tuple of (key, indices as list) val allPossibleGroups = keys.aggregateByKey( - zeroValue = IntArray(groupSize) { -1 }, - seqFunc = { base: IntArray, listIndex: Int -> - // put listIndex in the first empty spot in base - base[base.indexOfFirst { it < 0 }] = listIndex - - base - }, - - // how to merge partially filled up int arrays - combFunc = { a: IntArray, b: IntArray -> - // merge a and b - var j = 0 - for (i in a.indices) { - if (a[i] < 0) { - while (b[j] < 0) { - j++ - if (j == b.size) return@aggregateByKey a - } - a[i] = b[j] + zeroValue = IntArray(groupSize) { -1 }, + seqFunc = { base: IntArray, listIndex: Int -> + // put listIndex in the first empty spot in base + base[base.indexOfFirst { it < 0 }] = listIndex + + base + }, + + // how to merge partially filled up int arrays + combFunc = { a: IntArray, b: IntArray -> + // merge a and b + var j = 0 + for (i in a.indices) { + if (a[i] < 0) { + while (b[j] < 0) { j++ + if (j == b.size) return@aggregateByKey a } + a[i] = b[j] + j++ } - a - }, - ) + } + a + }, + ) .values() // finally just take the values return allPossibleGroups.toDS() @@ -113,7 +109,7 @@ object GroupCalculation { * @param listSize The size of the list, aka the max width, height etc. of the table * @return the unique number for this [indexTuple] */ - private fun getTupleValue(indexTuple: IntArray, listSize: Int): Int = + private fun getTupleValue(indexTuple: List, listSize: Int): Int = indexTuple.indices.sumOf { indexTuple[it] * listSize.toDouble().pow(it).toInt() } @@ -140,7 +136,7 @@ object GroupCalculation { * @param indexTuple a tuple of indices in the form of an IntArray * @return true if this tuple is in the right corner and should be included */ - private fun isValidIndexTuple(indexTuple: IntArray): Boolean { + private fun isValidIndexTuple(indexTuple: List): Boolean { // x - y > 0; 2d // (x - y) > 0 && (x - z) > 0 && (y - z) > 0; 3d // (x - y) > 0 && (x - z) > 0 && (x - a) > 0 && (y - z) > 0 && (y - a) > 0 && (z - a) > 0; 4d @@ -154,9 +150,9 @@ object GroupCalculation { } /** - * Recursive method that for [skipDimension] loops over all the other dimensions and stores - * in [keyValues] all results from [getTupleValue] as key and [value] as value. - * In the end, [keyValues] will have, for each key in the table below, a value for the key's column, row etc. + * Recursive method that for [skipDimension] loops over all the other dimensions and returns all results from + * [getTupleValue] as key and [value] as value. + * In the end, the return value will have, for each key in the table below, a value for the key's column, row etc. * * * This is an example for 2D. The letters will be int indices as well (a = 0, b = 1, ..., [listSize]), but help for clarification. @@ -181,52 +177,47 @@ object GroupCalculation { * * * @param groupSize the size of index tuples to form - * @param keyValues the MutableList in which to store the key-value results * @param value the current index to work from (can be seen as a letter in the table above) * @param listSize the size of the list to make - * @param currentDimension the indicator for which dimension we're currently calculating for (and how deep in the recursion we are) - * @param indexTuple the array (or tuple) in which to store the current indices * @param skipDimension the current dimension that will have a set value [value] while looping over the other dimensions */ private fun addTuples( groupSize: Int, - keyValues: MutableList>, value: Int, listSize: Int, - currentDimension: Int, - indexTuple: IntArray, skipDimension: Int, - ) { - if (currentDimension >= groupSize) { // base case - if (isValidIndexTuple(indexTuple)) { - keyValues += getTupleValue(indexTuple, listSize) X value - } - return - } - if (currentDimension == skipDimension) { - indexTuple[currentDimension] = value - addTuples( - groupSize = groupSize, - keyValues = keyValues, - value = value, - listSize = listSize, - currentDimension = currentDimension + 1, - indexTuple = indexTuple, - skipDimension = skipDimension, - ) - } else { - for (i in 0 until listSize) { - indexTuple[currentDimension] = i - addTuples( - groupSize = groupSize, - keyValues = keyValues, - value = value, - listSize = listSize, + ): List> { + + /** + * @param currentDimension the indicator for which dimension we're currently calculating for (and how deep in the recursion we are) + * @param indexTuple the list (or tuple) in which to store the current indices + */ + fun recursiveCall( + currentDimension: Int = 0, + indexTuple: List = emptyList(), + ): List> = when { + // base case + currentDimension >= groupSize -> + if (isValidIndexTuple(indexTuple)) + listOf(getTupleValue(indexTuple, listSize) X value) + else + emptyList() + + currentDimension == skipDimension -> + recursiveCall( currentDimension = currentDimension + 1, - indexTuple = indexTuple, - skipDimension = skipDimension, + indexTuple = indexTuple + value, ) - } + + else -> + (0 until listSize).flatMap { i -> + recursiveCall( + currentDimension = currentDimension + 1, + indexTuple = indexTuple + i, + ) + } } + + return recursiveCall() } } diff --git a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Iterators.kt b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Iterators.kt index 1be79918..addb0449 100644 --- a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Iterators.kt +++ b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Iterators.kt @@ -50,6 +50,7 @@ class PartitioningIterator( } /** Maps the values of the iterator lazily using [func]. */ +@Deprecated("[Iterator.map] now uses the [Sequence.map] function") class MappingIterator( private val source: Iterator, private val func: (T) -> R, @@ -63,6 +64,7 @@ class MappingIterator( } /** Filters the values of the iterator lazily using [predicate]. */ +@Deprecated("[Iterator.filter] now uses the [Sequence.filter] function") class FilteringIterator( private val source: Iterator, private val predicate: (T) -> Boolean, @@ -81,12 +83,20 @@ class FilteringIterator( } +/** Allows to transform an Iterator using the Sequence functions. */ +fun Iterator.transformAsSequence(func: Sequence.() -> Sequence): Iterator = + func(this.asSequence()).iterator() + +/** Flattens iterator. */ +fun Iterator>.flatten(): Iterator = transformAsSequence { flatMap { it.asSequence() } } + /** Maps the values of the iterator lazily using [func]. */ -fun Iterator.map(func: (T) -> R): Iterator = MappingIterator(this, func) +fun Iterator.map(func: (T) -> R): Iterator = transformAsSequence { map(func) } /** Filters the values of the iterator lazily using [predicate]. */ -fun Iterator.filter(predicate: (T) -> Boolean): Iterator = FilteringIterator(this, predicate) +fun Iterator.filter(predicate: (T) -> Boolean): Iterator = transformAsSequence { filter(predicate) } /** Partitions the values of the iterator lazily in groups of [size]. */ -fun Iterator.partition(size: Int): Iterator> = PartitioningIterator(this, size) +fun Iterator.partition(size: Int, cutIncomplete: Boolean = false): Iterator> = + PartitioningIterator(this, size, cutIncomplete) diff --git a/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/ApiTest.kt b/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/ApiTest.kt index 20044807..9a7168e5 100644 --- a/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/ApiTest.kt +++ b/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/ApiTest.kt @@ -103,6 +103,62 @@ class ApiTest : ShouldSpec({ kotlinList.first() shouldBe "a" kotlinList.last() shouldBe "b" } + + should("Map iterators") { + val data = (1..50).toList() + val iterator = iterator { yieldAll(data) } + .map { it.toString() } + + iterator.asSequence().toList() shouldBe data.map { it.toString() } + } + + should("Filter iterators") { + val data = (1..50).toList() + val iterator = iterator { yieldAll(data) } + .filter { it % 2 == 0 } + + iterator.asSequence().toList() shouldBe data.filter { it % 2 == 0 } + } + + should("Partition iterators") { + val data = (1..50).toList() + + val iterator1 = iterator { yieldAll(data) } + .partition(8, cutIncomplete = false) + val result1 = iterator1.asSequence().toList() + result1.size shouldBe (50 / 8 + 1) + result1.map { it.size }.distinct().size shouldBe 2 // two difference sizes should exist, 8 and the rest + + val iterator2 = iterator { yieldAll(data) } + .partition(8, cutIncomplete = true) + + val result2 = iterator2.asSequence().toList() + result2.size shouldBe (50 / 8) + result2.forEach { it.size shouldBe 8 } + } + + should("Flatten iterators") { + val data = (1..50).toList() + val (data1, data2) = data.partition { it <= 25 } + val iterator = iterator { + yield(data1.iterator()) + yield(data2.iterator()) + }.flatten() + + iterator.asSequence().toList() shouldBe data + } + + should("Flatmap iterators using transformAsSequence") { + val data = (1..50).toList() + val iterator = data.iterator() + .transformAsSequence { + flatMap { + listOf(it.toDouble(), it + 0.5) + } + } + + iterator.asSequence().toList() shouldBe data.flatMap { listOf(it.toDouble(), it + 0.5) } + } } } }) From 675b3d38922363f518b05c2c1fec1a57c6b1ecf7 Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Wed, 20 Jul 2022 17:33:13 +0200 Subject: [PATCH 12/16] added mapKeys, filter by kotlin range and more tests --- .../kotlinx/spark/api/RddKeyValue.kt | 41 ++++++++++++++++++- .../jetbrains/kotlinx/spark/api/RddTest.kt | 41 +++++++++++++++++++ 2 files changed, 81 insertions(+), 1 deletion(-) diff --git a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddKeyValue.kt b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddKeyValue.kt index 3d6c5c80..b6f78b07 100644 --- a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddKeyValue.kt +++ b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddKeyValue.kt @@ -14,6 +14,7 @@ import org.apache.spark.api.java.Optional import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.PartialResult import org.apache.spark.serializer.Serializer +import org.jetbrains.kotlinx.spark.api.tuples.* import scala.Tuple2 import scala.Tuple3 import scala.Tuple4 @@ -516,6 +517,17 @@ fun JavaRDD>.fullOuterJoin( */ fun JavaRDD>.collectAsMap(): Map = toJavaPairRDD().collectAsMap() +/** + * Pass each key in the key-value pair RDD through a map function without changing the values; + * this also retains the original RDD's partitioning. + */ +fun JavaRDD>.mapKeys(f: (K) -> U): JavaRDD> = + mapPartitions({ + it.map { (_1, _2) -> + tupleOf(f(_1), _2) + } + }, true) + /** * 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. @@ -541,7 +553,8 @@ fun JavaRDD>.cogroup( other3: JavaRDD>, partitioner: Partitioner, ): JavaRDD, Iterable, Iterable, Iterable>>> = - toJavaPairRDD().cogroup(other1.toJavaPairRDD(), other2.toJavaPairRDD(), other3.toJavaPairRDD(), partitioner).toTupleRDD() + toJavaPairRDD().cogroup(other1.toJavaPairRDD(), other2.toJavaPairRDD(), other3.toJavaPairRDD(), partitioner) + .toTupleRDD() /** * For each key k in [this] or [other], return a resulting RDD that contains a tuple with the @@ -829,6 +842,17 @@ fun JavaRDD>.sortByKey( fun JavaRDD>.filterByRange(lower: K, upper: K): JavaRDD> = toJavaPairRDD().filterByRange(lower, upper).toTupleRDD() +/** + * Return a RDD containing only the elements in the range [range]. + * If the RDD has been partitioned using a [RangePartitioner], then this operation can be + * performed efficiently by only scanning the partitions that might contain matching elements. + * Otherwise, a standard [filter] is applied to all partitions. + * + * @since 3.1.0 + */ +fun , V> JavaRDD>.filterByRange(range: ClosedRange): JavaRDD> = + filterByRange(range.start, range.endInclusive) + /** * Return a RDD containing only the elements in the inclusive range [lower] to [upper]. * If the RDD has been partitioned using a [RangePartitioner], then this operation can be @@ -844,6 +868,21 @@ fun JavaRDD>.filterByRange( ): JavaRDD> = toJavaPairRDD() .filterByRange(comp, lower, upper) .toTupleRDD() + +/** + * Return a RDD containing only the elements in the inclusive range [range]. + * If the RDD has been partitioned using a [RangePartitioner], then this operation can be + * performed efficiently by only scanning the partitions that might contain matching elements. + * Otherwise, a standard [filter] is applied to all partitions. + * + * @since 3.1.0 + */ +fun , V> JavaRDD>.filterByRange( + comp: Comparator, + range: ClosedRange, +): JavaRDD> = toJavaPairRDD() + .filterByRange(comp, range.start, range.endInclusive) + .toTupleRDD() //#endif /** diff --git a/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/RddTest.kt b/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/RddTest.kt index 21351124..6c1f671c 100644 --- a/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/RddTest.kt +++ b/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/RddTest.kt @@ -3,7 +3,9 @@ package org.jetbrains.kotlinx.spark.api import io.kotest.core.spec.style.ShouldSpec import io.kotest.matchers.collections.shouldContainAll import io.kotest.matchers.shouldBe +import org.apache.spark.api.java.JavaRDD import org.jetbrains.kotlinx.spark.api.tuples.* +import scala.Tuple2 class RddTest : ShouldSpec({ context("RDD extension functions") { @@ -19,6 +21,45 @@ class RddTest : ShouldSpec({ val list = counts.collect().toList() list.shouldContainAll("1" X 2, "2" X 3, "3" X 1) } + + should("Have handy functions") { + val rdd = rddOf( + 1 X "a", + 2 X "b", + 3 X "c", + 4 X "d", + 5 X "e", + 6 X "f", + ) + + //#if sparkMinor >= 3.1 + val rangeFiltered: JavaRDD> = rdd.filterByRange(2..5) + rangeFiltered.collect().shouldContainAll( + 2 X "b", + 3 X "c", + 4 X "d", + 5 X "e", + ) + //#endif + + val result = rdd + .flatMapValues { + listOf(it + 1, it + 2, it + 3, it + 4).iterator() + } + .also { + it.countByKey().values.forEach { it shouldBe 4 } + } + .foldByKey("", String::plus) // (1,"a1a2a3a4") etc. + .mapValues { it.toSortedSet().fold("", String::plus) } // (1,"1234a") etc. + .map { it.swap() } // ("1234a",1) etc. + .mapKeys { it.take(4) } // ("1234",1) etc. + .groupByKey() + .mapValues { it.toList() } // ("1234",[1,2,3,4, 5, 6]) + .collect() + .single() + + result shouldBe t("1234", listOf(1, 2, 3, 4, 5, 6)) + } } context("Double functions") { From 39dd0e91c468a99ff635b8407b98d4d01d525873 Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Thu, 21 Jul 2022 11:55:15 +0200 Subject: [PATCH 13/16] test failing? --- .../src/test/kotlin/org/jetbrains/kotlinx/spark/api/RddTest.kt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/RddTest.kt b/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/RddTest.kt index 6c1f671c..93aa3528 100644 --- a/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/RddTest.kt +++ b/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/RddTest.kt @@ -58,7 +58,7 @@ class RddTest : ShouldSpec({ .collect() .single() - result shouldBe t("1234", listOf(1, 2, 3, 4, 5, 6)) + (result == t("1234", listOf(1, 2, 3, 4, 5, 6))) shouldBe true } } From 1e924a7c01fd16c11673d4d06870f32a838b3256 Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Thu, 21 Jul 2022 12:23:22 +0200 Subject: [PATCH 14/16] test failing? --- .github/workflows/build.yml | 2 +- .../test/kotlin/org/jetbrains/kotlinx/spark/api/RddTest.kt | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index b6951d11..27acae0d 100644 --- a/.github/workflows/build.yml +++ b/.github/workflows/build.yml @@ -59,7 +59,7 @@ jobs: -Pspark=${{ matrix.spark }} -Pscala=${{ matrix.scala }} clean - build + test --scan # qodana: diff --git a/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/RddTest.kt b/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/RddTest.kt index 93aa3528..d5be31e1 100644 --- a/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/RddTest.kt +++ b/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/RddTest.kt @@ -54,11 +54,11 @@ class RddTest : ShouldSpec({ .map { it.swap() } // ("1234a",1) etc. .mapKeys { it.take(4) } // ("1234",1) etc. .groupByKey() - .mapValues { it.toList() } // ("1234",[1,2,3,4, 5, 6]) + .mapValues { it.toList() } // ("1234",[1,2,3,4,5,6]) .collect() .single() - (result == t("1234", listOf(1, 2, 3, 4, 5, 6))) shouldBe true + result shouldBe t("1234", listOf(1, 2, 3, 4, 5, 6)) } } From dfd4d1e16e2de52dc7f74e0ed259fbb802fbe430 Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Thu, 21 Jul 2022 12:41:43 +0200 Subject: [PATCH 15/16] test failing? --- .../src/test/kotlin/org/jetbrains/kotlinx/spark/api/RddTest.kt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/RddTest.kt b/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/RddTest.kt index d5be31e1..5f9b6d94 100644 --- a/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/RddTest.kt +++ b/kotlin-spark-api/src/test/kotlin/org/jetbrains/kotlinx/spark/api/RddTest.kt @@ -54,7 +54,7 @@ class RddTest : ShouldSpec({ .map { it.swap() } // ("1234a",1) etc. .mapKeys { it.take(4) } // ("1234",1) etc. .groupByKey() - .mapValues { it.toList() } // ("1234",[1,2,3,4,5,6]) + .mapValues { it.toList().sorted() } // ("1234",[1,2,3,4,5,6]) .collect() .single() From ecdcac441e7c09a5576d9ff49cb6bf01e5dc4d4f Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Fri, 22 Jul 2022 14:27:48 +0200 Subject: [PATCH 16/16] small docs fixes --- .../org/jetbrains/kotlinx/spark/api/Rdd.kt | 9 ++++++++- .../jetbrains/kotlinx/spark/api/RddDouble.kt | 18 ++++++------------ .../jetbrains/kotlinx/spark/api/RddKeyValue.kt | 2 ++ .../org/jetbrains/kotlinx/spark/api/Seq.kt | 16 ++++------------ 4 files changed, 20 insertions(+), 25 deletions(-) diff --git a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Rdd.kt b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Rdd.kt index d7db6249..8ba90d25 100644 --- a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Rdd.kt +++ b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Rdd.kt @@ -4,12 +4,19 @@ import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.java.JavaSparkContext import java.io.Serializable - +/** + * Utility method to create an RDD from a list. + * NOTE: [T] must be [Serializable]. + */ fun JavaSparkContext.rddOf( vararg elements: T, numSlices: Int = defaultParallelism(), ): JavaRDD = parallelize(elements.toList(), numSlices) +/** + * Utility method to create an RDD from a list. + * NOTE: [T] must be [Serializable]. + */ fun JavaSparkContext.toRDD( elements: List, numSlices: Int = defaultParallelism(), diff --git a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddDouble.kt b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddDouble.kt index 199337d9..3ba3ab72 100644 --- a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddDouble.kt +++ b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddDouble.kt @@ -8,6 +8,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.util.StatCounter import scala.Tuple2 +/** Utility method to convert [JavaRDD]<[Number]> to [JavaDoubleRDD]. */ @Suppress("UNCHECKED_CAST") inline fun JavaRDD.toJavaDoubleRDD(): JavaDoubleRDD = JavaDoubleRDD.fromRDD( @@ -17,6 +18,7 @@ inline fun JavaRDD.toJavaDoubleRDD(): JavaDoubleRDD = }.rdd() as RDD ) +/** Utility method to convert [JavaDoubleRDD] to [JavaRDD]<[Double]>. */ @Suppress("UNCHECKED_CAST") fun JavaDoubleRDD.toDoubleRDD(): JavaRDD = JavaDoubleRDD.toRDD(this).toJavaRDD() as JavaRDD @@ -51,27 +53,19 @@ inline fun JavaRDD.sampleStdev(): Double = toJavaDoubleR */ inline fun JavaRDD.sampleVariance(): Double = toJavaDoubleRDD().sampleVariance() -/** - * Compute the population standard deviation of this RDD's elements. - */ +/** Compute the population standard deviation of this RDD's elements. */ inline fun JavaRDD.popStdev(): Double = toJavaDoubleRDD().popStdev() -/** - * Compute the population variance of this RDD's elements. - */ +/** Compute the population variance of this RDD's elements. */ inline fun JavaRDD.popVariance(): Double = toJavaDoubleRDD().popVariance() -/** - * Approximate operation to return the mean within a timeout. - */ +/** Approximate operation to return the mean within a timeout. */ inline fun JavaRDD.meanApprox( timeout: Long, confidence: Double = 0.95, ): PartialResult = toJavaDoubleRDD().meanApprox(timeout, confidence) -/** - * Approximate operation to return the sum within a timeout. - */ +/** Approximate operation to return the sum within a timeout. */ inline fun JavaRDD.sumApprox( timeout: Long, confidence: Double = 0.95, diff --git a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddKeyValue.kt b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddKeyValue.kt index b6f78b07..b04bf6b9 100644 --- a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddKeyValue.kt +++ b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/RddKeyValue.kt @@ -21,9 +21,11 @@ import scala.Tuple4 import kotlin.random.Random import org.apache.hadoop.mapreduce.OutputFormat as NewOutputFormat +/** Utility method to convert [JavaRDD]<[Tuple2]> to [JavaPairRDD]. */ fun JavaRDD>.toJavaPairRDD(): JavaPairRDD = JavaPairRDD.fromJavaRDD(this) +/** Utility method to convert [JavaPairRDD] to [JavaRDD]<[Tuple2]>. */ fun JavaPairRDD.toTupleRDD(): JavaRDD> = JavaPairRDD.toRDD(this).toJavaRDD() diff --git a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Seq.kt b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Seq.kt index 2cb44ca2..adc39bcc 100644 --- a/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Seq.kt +++ b/kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Seq.kt @@ -5,14 +5,10 @@ import scala.collection.immutable.Seq as Seq import scala.collection.mutable.`Seq$`.`MODULE$` as MutableSeq import scala.collection.mutable.Seq as MutableSeq -/** - * Returns a new empty immutable Seq. - */ +/** Returns a new empty immutable Seq. */ fun emptySeq(): Seq = Seq.empty() as Seq -/** - * Returns a new immutable Seq with the given elements. - */ +/** Returns a new immutable Seq with the given elements. */ fun seqOf(vararg elements: T): Seq = if (elements.isEmpty()) emptySeq() @@ -22,14 +18,10 @@ fun seqOf(vararg elements: T): Seq = `$plus$eq`(it) }.result() as Seq -/** - * Returns a new mutable Seq with the given elements. - */ +/** Returns a new mutable Seq with the given elements. */ fun emptyMutableSeq(): MutableSeq = MutableSeq.empty() as MutableSeq -/** - * Returns a new mutable Seq with the given elements. - */ +/** Returns a new mutable Seq with the given elements. */ fun mutableSeqOf(vararg elements: T): MutableSeq = if (elements.isEmpty()) emptyMutableSeq()