From b24172f9c36f576c893f0e5e78df5720c77ceced Mon Sep 17 00:00:00 2001 From: Jolan Rensen Date: Mon, 21 Feb 2022 17:11:49 +0100 Subject: [PATCH 01/44] added withSparkStreaming function and example. Let's see if something like this could form the basis of easy streaming support :) --- .../kotlinx/spark/examples/Streaming.kt | 39 ++++-------- .../kotlinx/spark/api/SparkHelper.kt | 59 ++++++++++++++++++- 2 files changed, 70 insertions(+), 28 deletions(-) diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Streaming.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Streaming.kt index 7c562bd5..bc9284d5 100644 --- a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Streaming.kt +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Streaming.kt @@ -20,42 +20,29 @@ package org.jetbrains.kotlinx.spark.examples import org.apache.spark.SparkConf -import org.apache.spark.api.java.JavaSparkContext +import org.apache.spark.sql.Dataset +import org.apache.spark.streaming.Duration import org.apache.spark.streaming.Durations import org.apache.spark.streaming.api.java.JavaStreamingContext -import org.jetbrains.kotlinx.spark.api.withSpark -import scala.Tuple2 -import java.io.Serializable +import org.jetbrains.kotlinx.spark.api.* -data class Row @JvmOverloads constructor( - var word: String = "", -) : Serializable +data class TestRow( + val word: String, +) -fun main() = withSpark { - - val context = JavaStreamingContext( - SparkConf() - .setMaster("local[*]") - .setAppName("Test"), - Durations.seconds(1), - ) - - val lines = context.socketTextStream("localhost", 9999) +fun main() = withSparkStreaming(Durations.seconds(1)) { + val lines = ssc.socketTextStream("localhost", 9999) val words = lines.flatMap { it.split(" ").iterator() } words.foreachRDD { rdd, time -> + val dataframe: Dataset = rdd.map { TestRow(it) }.toDS() - // todo convert rdd to dataset using kotlin data class? - - val rowRdd = rdd.map { Row(it) } - - val dataframe = spark.createDataFrame(rowRdd, Row::class.java) - + dataframe + .groupByKey { it.word } + .count() + .show() } - - context.start() - context.awaitTermination() } \ No newline at end of file diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkHelper.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkHelper.kt index d9b4823a..db093cfc 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkHelper.kt +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkHelper.kt @@ -27,6 +27,8 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.Dataset import org.apache.spark.sql.SparkSession.Builder import org.apache.spark.sql.UDFRegistration +import org.apache.spark.streaming.Duration +import org.apache.spark.streaming.api.java.JavaStreamingContext import org.jetbrains.kotlinx.spark.api.SparkLogLevel.ERROR /** @@ -105,10 +107,57 @@ inline fun withSpark(sparkConf: SparkConf, logLevel: SparkLogLevel = ERROR, func ) } + +/** + * Wrapper for spark streaming creation. `spark: SparkSession` and `ssc: JavaStreamingContext` are provided, started, + * awaited, and stopped automatically. + * + * @param batchDuration The time interval at which streaming data will be divided into batches + * @param props spark options, value types are runtime-checked for type-correctness + * @param master Sets the Spark master URL to connect to, such as "local" to run locally, "local[4]" to + * run locally with 4 cores, or "spark://master:7077" to run on a Spark standalone cluster. By default, it + * tries to get the system value "spark.master", otherwise it uses "local[*]" + * @param appName Sets a name for the application, which will be shown in the Spark web UI. + * If no application name is set, a randomly generated name will be used. + * @param logLevel Control our logLevel. This overrides any user-defined log settings. + * @param func function which will be executed in context of [KSparkStreamingSession] (it means that `this` inside block will point to [KSparkStreamingSession]) + * todo: provide alternatives with path instead of batchDuration etc + */ +@JvmOverloads +inline fun withSparkStreaming( + batchDuration: Duration, + props: Map = emptyMap(), + master: String = SparkConf().get("spark.master", "local[*]"), + appName: String = "Kotlin Spark Sample", + logLevel: SparkLogLevel = SparkLogLevel.ERROR, + func: KSparkStreamingSession.() -> Unit, +) { + val conf = SparkConf() + .setMaster(master) + .setAppName(appName) + .apply { + props.forEach { + set(it.key, it.toString()) + } + } + + val ssc = JavaStreamingContext(conf, batchDuration) + val spark = SparkSession.builder().config(conf).getOrCreate() + + KSparkStreamingSession(spark, ssc).apply { + spark.sparkContext.setLogLevel(logLevel) + func() + ssc.start() + ssc.awaitTermination() + sc.stop() + spark.stop() + } +} + /** - * This wrapper over [SparkSession] which provides several additional methods to create [org.apache.spark.sql.Dataset] + * This wrapper over [SparkSession] provides several additional methods to create [org.apache.spark.sql.Dataset] */ -class KSparkSession(val spark: SparkSession) { +open class KSparkSession(val spark: SparkSession) { val sc: JavaSparkContext = JavaSparkContext(spark.sparkContext) @@ -119,3 +168,9 @@ class KSparkSession(val spark: SparkSession) { inline fun JavaRDDLike.toDS() = toDS(spark) val udf: UDFRegistration get() = spark.udf() } + +/** + * This wrapper over [SparkSession] and [JavaStreamingContext] provides several additional methods to create [org.apache.spark.sql.Dataset] + */ +class KSparkStreamingSession(spark: SparkSession, val ssc: JavaStreamingContext) : KSparkSession(spark) + From a378070c2368c770edf40bb575802abf5e783079 Mon Sep 17 00:00:00 2001 From: Jolan Rensen Date: Mon, 21 Feb 2022 17:47:53 +0100 Subject: [PATCH 02/44] makes withSparkStreaming reuse the normal withSpark --- .../kotlinx/spark/api/SparkHelper.kt | 37 +++++++------------ 1 file changed, 14 insertions(+), 23 deletions(-) diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkHelper.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkHelper.kt index db093cfc..2e750c81 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkHelper.kt +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkHelper.kt @@ -30,6 +30,7 @@ import org.apache.spark.sql.UDFRegistration import org.apache.spark.streaming.Duration import org.apache.spark.streaming.api.java.JavaStreamingContext import org.jetbrains.kotlinx.spark.api.SparkLogLevel.ERROR +import kotlin.math.log /** * Wrapper for spark creation which allows setting different spark params. @@ -132,35 +133,25 @@ inline fun withSparkStreaming( logLevel: SparkLogLevel = SparkLogLevel.ERROR, func: KSparkStreamingSession.() -> Unit, ) { - val conf = SparkConf() - .setMaster(master) - .setAppName(appName) - .apply { - props.forEach { - set(it.key, it.toString()) - } + withSpark( + props = props, + master = master, + appName = appName, + logLevel = logLevel, + ) { + val ssc = JavaStreamingContext(sc, batchDuration) + KSparkStreamingSession(session = this, ssc = ssc).apply { + func() + ssc.start() + ssc.awaitTermination() } - - val ssc = JavaStreamingContext(conf, batchDuration) - val spark = SparkSession.builder().config(conf).getOrCreate() - - KSparkStreamingSession(spark, ssc).apply { - spark.sparkContext.setLogLevel(logLevel) - func() - ssc.start() - ssc.awaitTermination() - sc.stop() - spark.stop() } } /** * This wrapper over [SparkSession] provides several additional methods to create [org.apache.spark.sql.Dataset] */ -open class KSparkSession(val spark: SparkSession) { - - val sc: JavaSparkContext = JavaSparkContext(spark.sparkContext) - +open class KSparkSession(val spark: SparkSession, val sc: JavaSparkContext = JavaSparkContext(spark.sparkContext)) { inline fun List.toDS() = toDS(spark) inline fun Array.toDS() = spark.dsOf(*this) inline fun dsOf(vararg arg: T) = spark.dsOf(*arg) @@ -172,5 +163,5 @@ open class KSparkSession(val spark: SparkSession) { /** * This wrapper over [SparkSession] and [JavaStreamingContext] provides several additional methods to create [org.apache.spark.sql.Dataset] */ -class KSparkStreamingSession(spark: SparkSession, val ssc: JavaStreamingContext) : KSparkSession(spark) +class KSparkStreamingSession(session: KSparkSession, val ssc: JavaStreamingContext) : KSparkSession(session.spark, session.sc) From 47e1ccb0df52a8d00059485c56b1d3780e612ff3 Mon Sep 17 00:00:00 2001 From: Jolan Rensen Date: Thu, 24 Feb 2022 15:16:06 +0100 Subject: [PATCH 03/44] removed sc.stop() --- .../kotlin/org/jetbrains/kotlinx/spark/api/SparkHelper.kt | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkHelper.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkHelper.kt index 2e750c81..039b14b3 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkHelper.kt +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkHelper.kt @@ -78,6 +78,7 @@ inline fun withSpark( * @param logLevel Control our logLevel. This overrides any user-defined log settings. * @param func function which will be executed in context of [KSparkSession] (it means that `this` inside block will point to [KSparkSession]) */ +@Suppress("UsePropertyAccessSyntax") @JvmOverloads inline fun withSpark(builder: Builder, logLevel: SparkLogLevel = ERROR, func: KSparkSession.() -> Unit) { builder @@ -86,7 +87,6 @@ inline fun withSpark(builder: Builder, logLevel: SparkLogLevel = ERROR, func: KS KSparkSession(this).apply { sparkContext.setLogLevel(logLevel) func() - sc.stop() spark.stop() } } @@ -163,5 +163,6 @@ open class KSparkSession(val spark: SparkSession, val sc: JavaSparkContext = Jav /** * This wrapper over [SparkSession] and [JavaStreamingContext] provides several additional methods to create [org.apache.spark.sql.Dataset] */ -class KSparkStreamingSession(session: KSparkSession, val ssc: JavaStreamingContext) : KSparkSession(session.spark, session.sc) +class KSparkStreamingSession(session: KSparkSession, val ssc: JavaStreamingContext) : + KSparkSession(session.spark, session.sc) From 6c1b0d9ec3447e754fb5eebd3fc542a1b00eb9e7 Mon Sep 17 00:00:00 2001 From: Jolan Rensen Date: Mon, 21 Feb 2022 17:11:49 +0100 Subject: [PATCH 04/44] added withSparkStreaming function and example. Let's see if something like this could form the basis of easy streaming support :) --- .../kotlinx/spark/examples/Streaming.kt | 48 +++++++++++++++ .../kotlinx/spark/api/SparkSession.kt | 58 ++++++++++++++++++- 2 files changed, 105 insertions(+), 1 deletion(-) create mode 100644 examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Streaming.kt diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Streaming.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Streaming.kt new file mode 100644 index 00000000..bc9284d5 --- /dev/null +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Streaming.kt @@ -0,0 +1,48 @@ +/*- + * =LICENSE= + * Kotlin Spark API: Examples for Spark 3.2+ (Scala 2.12) + * ---------- + * Copyright (C) 2019 - 2022 JetBrains + * ---------- + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * =LICENSEEND= + */ +package org.jetbrains.kotlinx.spark.examples + +import org.apache.spark.SparkConf +import org.apache.spark.sql.Dataset +import org.apache.spark.streaming.Duration +import org.apache.spark.streaming.Durations +import org.apache.spark.streaming.api.java.JavaStreamingContext +import org.jetbrains.kotlinx.spark.api.* + +data class TestRow( + val word: String, +) + +fun main() = withSparkStreaming(Durations.seconds(1)) { + + val lines = ssc.socketTextStream("localhost", 9999) + val words = lines.flatMap { it.split(" ").iterator() } + + words.foreachRDD { rdd, time -> + val dataframe: Dataset = rdd.map { TestRow(it) }.toDS() + + dataframe + .groupByKey { it.word } + .count() + .show() + + } + +} \ No newline at end of file diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt index 118abf48..b65fc6b1 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt @@ -33,6 +33,8 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.Dataset import org.apache.spark.sql.SparkSession.Builder import org.apache.spark.sql.UDFRegistration +import org.apache.spark.streaming.Duration +import org.apache.spark.streaming.api.java.JavaStreamingContext import org.jetbrains.kotlinx.spark.api.SparkLogLevel.ERROR import org.jetbrains.kotlinx.spark.extensions.KSparkExtensions @@ -41,7 +43,7 @@ import org.jetbrains.kotlinx.spark.extensions.KSparkExtensions * * @param spark The current [SparkSession] to wrap */ -class KSparkSession(val spark: SparkSession) { +open class KSparkSession(val spark: SparkSession) { /** Lazy instance of [JavaSparkContext] wrapper around [sparkContext]. */ val sc: JavaSparkContext by lazy { JavaSparkContext(spark.sparkContext) } @@ -76,6 +78,13 @@ class KSparkSession(val spark: SparkSession) { val udf: UDFRegistration get() = spark.udf() } +/** + * This wrapper over [SparkSession] and [JavaStreamingContext] provides several additional methods to create [org.apache.spark.sql.Dataset] + */ +class KSparkStreamingSession(spark: SparkSession, val ssc: JavaStreamingContext) : KSparkSession(spark) + + + /** * The entry point to programming Spark with the Dataset and DataFrame API. * @@ -175,6 +184,53 @@ inline fun withSpark(sparkConf: SparkConf, logLevel: SparkLogLevel = ERROR, func ) } + +/** + * Wrapper for spark streaming creation. `spark: SparkSession` and `ssc: JavaStreamingContext` are provided, started, + * awaited, and stopped automatically. + * + * @param batchDuration The time interval at which streaming data will be divided into batches + * @param props spark options, value types are runtime-checked for type-correctness + * @param master Sets the Spark master URL to connect to, such as "local" to run locally, "local[4]" to + * run locally with 4 cores, or "spark://master:7077" to run on a Spark standalone cluster. By default, it + * tries to get the system value "spark.master", otherwise it uses "local[*]" + * @param appName Sets a name for the application, which will be shown in the Spark web UI. + * If no application name is set, a randomly generated name will be used. + * @param logLevel Control our logLevel. This overrides any user-defined log settings. + * @param func function which will be executed in context of [KSparkStreamingSession] (it means that `this` inside block will point to [KSparkStreamingSession]) + * todo: provide alternatives with path instead of batchDuration etc + */ +@JvmOverloads +inline fun withSparkStreaming( + batchDuration: Duration, + props: Map = emptyMap(), + master: String = SparkConf().get("spark.master", "local[*]"), + appName: String = "Kotlin Spark Sample", + logLevel: SparkLogLevel = SparkLogLevel.ERROR, + func: KSparkStreamingSession.() -> Unit, +) { + val conf = SparkConf() + .setMaster(master) + .setAppName(appName) + .apply { + props.forEach { + set(it.key, it.toString()) + } + } + + val ssc = JavaStreamingContext(conf, batchDuration) + val spark = SparkSession.builder().config(conf).getOrCreate() + + KSparkStreamingSession(spark, ssc).apply { + spark.sparkContext.setLogLevel(logLevel) + func() + ssc.start() + ssc.awaitTermination() + sc.stop() + spark.stop() + } +} + /** * Broadcast a read-only variable to the cluster, returning a * [org.apache.spark.broadcast.Broadcast] object for reading it in distributed functions. From 95a95630913decc5d4be4fedfdaa08d09ed0f67b Mon Sep 17 00:00:00 2001 From: Jolan Rensen Date: Mon, 21 Feb 2022 17:47:53 +0100 Subject: [PATCH 05/44] makes withSparkStreaming reuse the normal withSpark --- .../kotlinx/spark/api/SparkSession.kt | 69 +++++++------------ 1 file changed, 24 insertions(+), 45 deletions(-) diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt index b65fc6b1..bb0c4299 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt @@ -209,59 +209,38 @@ inline fun withSparkStreaming( logLevel: SparkLogLevel = SparkLogLevel.ERROR, func: KSparkStreamingSession.() -> Unit, ) { - val conf = SparkConf() - .setMaster(master) - .setAppName(appName) - .apply { - props.forEach { - set(it.key, it.toString()) - } + withSpark( + props = props, + master = master, + appName = appName, + logLevel = logLevel, + ) { + val ssc = JavaStreamingContext(sc, batchDuration) + KSparkStreamingSession(session = this, ssc = ssc).apply { + func() + ssc.start() + ssc.awaitTermination() } - - val ssc = JavaStreamingContext(conf, batchDuration) - val spark = SparkSession.builder().config(conf).getOrCreate() - - KSparkStreamingSession(spark, ssc).apply { - spark.sparkContext.setLogLevel(logLevel) - func() - ssc.start() - ssc.awaitTermination() - sc.stop() - spark.stop() } } /** - * Broadcast a read-only variable to the cluster, returning a - * [org.apache.spark.broadcast.Broadcast] object for reading it in distributed functions. - * The variable will be sent to each cluster only once. - * - * @param value value to broadcast to the Spark nodes - * @return `Broadcast` object, a read-only variable cached on each machine + * This wrapper over [SparkSession] provides several additional methods to create [org.apache.spark.sql.Dataset] */ -inline fun SparkSession.broadcast(value: T): Broadcast = try { - sparkContext.broadcast(value, encoder().clsTag()) -} catch (e: ClassNotFoundException) { - JavaSparkContext(sparkContext).broadcast(value) +open class KSparkSession(val spark: SparkSession) { + + val sc: JavaSparkContext = JavaSparkContext(spark.sparkContext) + + inline fun List.toDS() = toDS(spark) + inline fun Array.toDS() = spark.dsOf(*this) + inline fun dsOf(vararg arg: T) = spark.dsOf(*arg) + inline fun RDD.toDS() = toDS(spark) + inline fun JavaRDDLike.toDS() = toDS(spark) + val udf: UDFRegistration get() = spark.udf() } /** - * Broadcast a read-only variable to the cluster, returning a - * [org.apache.spark.broadcast.Broadcast] object for reading it in distributed functions. - * The variable will be sent to each cluster only once. - * - * @param value value to broadcast to the Spark nodes - * @return `Broadcast` object, a read-only variable cached on each machine - * @see broadcast + * This wrapper over [SparkSession] and [JavaStreamingContext] provides several additional methods to create [org.apache.spark.sql.Dataset] */ -@Deprecated( - "You can now use `spark.broadcast()` instead.", - ReplaceWith("spark.broadcast(value)"), - DeprecationLevel.WARNING -) -inline fun SparkContext.broadcast(value: T): Broadcast = try { - broadcast(value, encoder().clsTag()) -} catch (e: ClassNotFoundException) { - JavaSparkContext(this).broadcast(value) -} +class KSparkStreamingSession(spark: SparkSession, val ssc: JavaStreamingContext) : KSparkSession(spark) From 9d1450bbe2bcfc74363416aac0cb5c920b16d442 Mon Sep 17 00:00:00 2001 From: Jolan Rensen Date: Thu, 24 Feb 2022 15:16:06 +0100 Subject: [PATCH 06/44] removed sc.stop() --- .../kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt index bb0c4299..9b33ae49 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt @@ -155,6 +155,7 @@ inline fun withSpark( * @param logLevel Control our logLevel. This overrides any user-defined log settings. * @param func function which will be executed in context of [KSparkSession] (it means that `this` inside block will point to [KSparkSession]) */ +@Suppress("UsePropertyAccessSyntax") @JvmOverloads inline fun withSpark(builder: Builder, logLevel: SparkLogLevel = ERROR, func: KSparkSession.() -> Unit) { builder @@ -229,7 +230,7 @@ inline fun withSparkStreaming( */ open class KSparkSession(val spark: SparkSession) { - val sc: JavaSparkContext = JavaSparkContext(spark.sparkContext) + val sc: JavaSparkContext by lazy { JavaSparkContext(spark.sparkContext) } inline fun List.toDS() = toDS(spark) inline fun Array.toDS() = spark.dsOf(*this) From f2ce00096c7f66018010b97dc5905b84ee5f2f74 Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Mon, 7 Mar 2022 16:32:49 +0100 Subject: [PATCH 07/44] fixed merge --- .../kotlinx/spark/api/SparkSession.kt | 43 ++++++++++++------- .../kotlinx/spark/api/DatasetFunctionTest.kt | 19 ++++++++ 2 files changed, 47 insertions(+), 15 deletions(-) diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt index 9b33ae49..d81898c3 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt @@ -217,7 +217,7 @@ inline fun withSparkStreaming( logLevel = logLevel, ) { val ssc = JavaStreamingContext(sc, batchDuration) - KSparkStreamingSession(session = this, ssc = ssc).apply { + KSparkStreamingSession(spark = spark, ssc = ssc).apply { func() ssc.start() ssc.awaitTermination() @@ -226,22 +226,35 @@ inline fun withSparkStreaming( } /** - * This wrapper over [SparkSession] provides several additional methods to create [org.apache.spark.sql.Dataset] + * Broadcast a read-only variable to the cluster, returning a + * [org.apache.spark.broadcast.Broadcast] object for reading it in distributed functions. + * The variable will be sent to each cluster only once. + * + * @param value value to broadcast to the Spark nodes + * @return `Broadcast` object, a read-only variable cached on each machine */ -open class KSparkSession(val spark: SparkSession) { - - val sc: JavaSparkContext by lazy { JavaSparkContext(spark.sparkContext) } - - inline fun List.toDS() = toDS(spark) - inline fun Array.toDS() = spark.dsOf(*this) - inline fun dsOf(vararg arg: T) = spark.dsOf(*arg) - inline fun RDD.toDS() = toDS(spark) - inline fun JavaRDDLike.toDS() = toDS(spark) - val udf: UDFRegistration get() = spark.udf() +inline fun SparkSession.broadcast(value: T): Broadcast = try { + sparkContext.broadcast(value, encoder().clsTag()) +} catch (e: ClassNotFoundException) { + JavaSparkContext(sparkContext).broadcast(value) } /** - * This wrapper over [SparkSession] and [JavaStreamingContext] provides several additional methods to create [org.apache.spark.sql.Dataset] + * Broadcast a read-only variable to the cluster, returning a + * [org.apache.spark.broadcast.Broadcast] object for reading it in distributed functions. + * The variable will be sent to each cluster only once. + * + * @param value value to broadcast to the Spark nodes + * @return `Broadcast` object, a read-only variable cached on each machine + * @see broadcast */ -class KSparkStreamingSession(spark: SparkSession, val ssc: JavaStreamingContext) : KSparkSession(spark) - +@Deprecated( + "You can now use `spark.broadcast()` instead.", + ReplaceWith("spark.broadcast(value)"), + DeprecationLevel.WARNING +) +inline fun SparkContext.broadcast(value: T): Broadcast = try { + broadcast(value, encoder().clsTag()) +} catch (e: ClassNotFoundException) { + JavaSparkContext(this).broadcast(value) +} \ No newline at end of file diff --git a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/DatasetFunctionTest.kt b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/DatasetFunctionTest.kt index f352ff68..b9667e8a 100644 --- a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/DatasetFunctionTest.kt +++ b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/DatasetFunctionTest.kt @@ -1,3 +1,22 @@ +/*- + * =LICENSE= + * Kotlin Spark API: API for Spark 3.2+ (Scala 2.12) + * ---------- + * Copyright (C) 2019 - 2022 JetBrains + * ---------- + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * =LICENSEEND= + */ package org.jetbrains.kotlinx.spark.api import ch.tutteli.atrium.api.fluent.en_GB.* From cdf7296283eb8fb14cb147515e84e84b166c4284 Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Mon, 7 Mar 2022 17:53:58 +0100 Subject: [PATCH 08/44] working on tests --- .../kotlinx/spark/examples/Streaming.kt | 3 ++ .../kotlinx/spark/api/SparkSession.kt | 9 +++- .../kotlinx/spark/api/StreamingTest.kt | 42 +++++++++++++++++++ 3 files changed, 52 insertions(+), 2 deletions(-) create mode 100644 kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Streaming.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Streaming.kt index bc9284d5..d66ef3e3 100644 --- a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Streaming.kt +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Streaming.kt @@ -35,6 +35,9 @@ fun main() = withSparkStreaming(Durations.seconds(1)) { val lines = ssc.socketTextStream("localhost", 9999) val words = lines.flatMap { it.split(" ").iterator() } + lines. + + words.foreachRDD { rdd, time -> val dataframe: Dataset = rdd.map { TestRow(it) }.toDS() diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt index d81898c3..6cbdf29e 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt @@ -81,7 +81,11 @@ open class KSparkSession(val spark: SparkSession) { /** * This wrapper over [SparkSession] and [JavaStreamingContext] provides several additional methods to create [org.apache.spark.sql.Dataset] */ -class KSparkStreamingSession(spark: SparkSession, val ssc: JavaStreamingContext) : KSparkSession(spark) +class KSparkStreamingSession(spark: SparkSession, val ssc: JavaStreamingContext) : KSparkSession(spark) { + + /** Can be overwritten to be run after the streaming session has started and before it's terminated. */ + var runAfterStart: KSparkStreamingSession.() -> Unit = {} +} @@ -198,7 +202,7 @@ inline fun withSpark(sparkConf: SparkConf, logLevel: SparkLogLevel = ERROR, func * @param appName Sets a name for the application, which will be shown in the Spark web UI. * If no application name is set, a randomly generated name will be used. * @param logLevel Control our logLevel. This overrides any user-defined log settings. - * @param func function which will be executed in context of [KSparkStreamingSession] (it means that `this` inside block will point to [KSparkStreamingSession]) + * @param beforeStart function which will be executed in context of [KSparkStreamingSession] (it means that `this` inside block will point to [KSparkStreamingSession]) * todo: provide alternatives with path instead of batchDuration etc */ @JvmOverloads @@ -220,6 +224,7 @@ inline fun withSparkStreaming( KSparkStreamingSession(spark = spark, ssc = ssc).apply { func() ssc.start() + runAfterStart() ssc.awaitTermination() } } diff --git a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt new file mode 100644 index 00000000..50bbce76 --- /dev/null +++ b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt @@ -0,0 +1,42 @@ +package org.jetbrains.kotlinx.spark.api + +import io.kotest.core.spec.style.ShouldSpec +import io.kotest.matchers.shouldBe +import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.streaming.OutputMode +import org.apache.spark.streaming.Duration + +class StreamingTest : ShouldSpec({ + context("streaming") { + should("stream") { + + withSpark/*Streaming(Duration(1))*/ { + // WIP this doesn't use ssc at all? + + val events = MemoryStream(100, spark.sqlContext(), null, encoder()) + val sessions = events.toDS() + sessions.isStreaming shouldBe true + + val transformedSessions = sessions.map { (it * 2).toString() } + + val streamingQuery = transformedSessions + .writeStream() + .format("memory") + .queryName("test") + .outputMode(OutputMode.Append()) + .start() + + val currentOffset = events.addData(listOf(1, 2, 3).asScalaIterable()) + streamingQuery.processAllAvailable() + events.commit(currentOffset) + + spark.table("test") + .show(false) + + + } + + + } + } +}) \ No newline at end of file From 330536c6c9b471a18bbe2671e8f9d78309a12a62 Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Tue, 8 Mar 2022 11:24:28 +0100 Subject: [PATCH 09/44] added timeout and working streaming test --- .../kotlinx/spark/examples/Streaming.kt | 4 +- .../kotlinx/spark/api/SparkSession.kt | 7 ++- .../kotlinx/spark/api/StreamingTest.kt | 44 +++++++++---------- 3 files changed, 28 insertions(+), 27 deletions(-) diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Streaming.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Streaming.kt index d66ef3e3..de77c96f 100644 --- a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Streaming.kt +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Streaming.kt @@ -30,13 +30,11 @@ data class TestRow( val word: String, ) -fun main() = withSparkStreaming(Durations.seconds(1)) { +fun main() = withSparkStreaming(Durations.seconds(1), timeout = 10_000) { val lines = ssc.socketTextStream("localhost", 9999) val words = lines.flatMap { it.split(" ").iterator() } - lines. - words.foreachRDD { rdd, time -> val dataframe: Dataset = rdd.map { TestRow(it) }.toDS() diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt index 6cbdf29e..96f753c7 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt @@ -202,7 +202,8 @@ inline fun withSpark(sparkConf: SparkConf, logLevel: SparkLogLevel = ERROR, func * @param appName Sets a name for the application, which will be shown in the Spark web UI. * If no application name is set, a randomly generated name will be used. * @param logLevel Control our logLevel. This overrides any user-defined log settings. - * @param beforeStart function which will be executed in context of [KSparkStreamingSession] (it means that `this` inside block will point to [KSparkStreamingSession]) + * @param timeout The time in milliseconds to wait for the stream to terminate without input. -1 by default, this means no timeout. + * @param func function which will be executed in context of [KSparkStreamingSession] (it means that `this` inside block will point to [KSparkStreamingSession]) * todo: provide alternatives with path instead of batchDuration etc */ @JvmOverloads @@ -212,6 +213,7 @@ inline fun withSparkStreaming( master: String = SparkConf().get("spark.master", "local[*]"), appName: String = "Kotlin Spark Sample", logLevel: SparkLogLevel = SparkLogLevel.ERROR, + timeout: Long = -1L, func: KSparkStreamingSession.() -> Unit, ) { withSpark( @@ -225,7 +227,8 @@ inline fun withSparkStreaming( func() ssc.start() runAfterStart() - ssc.awaitTermination() + ssc.awaitTerminationOrTimeout(timeout) + ssc.stop() } } } diff --git a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt index 50bbce76..672832b5 100644 --- a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt +++ b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt @@ -1,41 +1,41 @@ package org.jetbrains.kotlinx.spark.api import io.kotest.core.spec.style.ShouldSpec +import io.kotest.matchers.collections.shouldBeIn import io.kotest.matchers.shouldBe -import org.apache.spark.sql.execution.streaming.MemoryStream -import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.streaming.Duration +import java.io.Serializable +import org.jetbrains.kotlinx.spark.api.* +import java.util.LinkedList + class StreamingTest : ShouldSpec({ context("streaming") { should("stream") { - withSpark/*Streaming(Duration(1))*/ { - // WIP this doesn't use ssc at all? - - val events = MemoryStream(100, spark.sqlContext(), null, encoder()) - val sessions = events.toDS() - sessions.isStreaming shouldBe true - - val transformedSessions = sessions.map { (it * 2).toString() } + val input = listOf("aaa", "bbb", "aaa", "ccc") - val streamingQuery = transformedSessions - .writeStream() - .format("memory") - .queryName("test") - .outputMode(OutputMode.Append()) - .start() + val results = object : Serializable { + @Volatile + var counter = 0 + } - val currentOffset = events.addData(listOf(1, 2, 3).asScalaIterable()) - streamingQuery.processAllAvailable() - events.commit(currentOffset) + withSparkStreaming(Duration(10), timeout = 1000) { + val resultsBroadcast = spark.broadcast(results) - spark.table("test") - .show(false) + val rdd = sc.parallelize(input) + val queue = LinkedList(listOf(rdd)) + val inputStream = ssc.queueStream(queue) + inputStream.foreachRDD { rdd, _ -> + rdd.foreach { + it shouldBeIn input + resultsBroadcast.value.counter++ + } + } } - + results.counter shouldBe input.size } } From d2e792a3144f383f904228400298a06d77e69485 Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Tue, 8 Mar 2022 11:37:30 +0100 Subject: [PATCH 10/44] added timeout and working streaming test --- .../kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt index 672832b5..e0139b9c 100644 --- a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt +++ b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt @@ -29,7 +29,7 @@ class StreamingTest : ShouldSpec({ val inputStream = ssc.queueStream(queue) inputStream.foreachRDD { rdd, _ -> - rdd.foreach { + rdd.toDS().forEach { it shouldBeIn input resultsBroadcast.value.counter++ } From 4222a0380ee537fbd6610dc8fdc14df3afcf6eb7 Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Thu, 17 Mar 2022 19:18:44 +0100 Subject: [PATCH 11/44] still exploring, just pushing to keep my work safe :) --- .../GeneratePairStreamingFunctions.kt | 87 +++ .../JavaRecoverableNetworkWordCount.kt | 215 +++++++ .../examples/KotlinDirectKafkaWordCount.kt | 113 ++++ kotlin-spark-api/3.2/pom_2.12.xml | 16 + .../kotlinx/spark/api/Conversions.kt | 14 + .../kotlinx/spark/api/SparkSession.kt | 69 +- .../jetbrains/kotlinx/spark/api/Streaming.kt | 603 ++++++++++++++++++ .../kotlinx/spark/api/StreamingTest.kt | 21 +- 8 files changed, 1124 insertions(+), 14 deletions(-) create mode 100644 examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/GeneratePairStreamingFunctions.kt create mode 100644 examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/JavaRecoverableNetworkWordCount.kt create mode 100644 examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/KotlinDirectKafkaWordCount.kt create mode 100644 kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Streaming.kt diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/GeneratePairStreamingFunctions.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/GeneratePairStreamingFunctions.kt new file mode 100644 index 00000000..740774e2 --- /dev/null +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/GeneratePairStreamingFunctions.kt @@ -0,0 +1,87 @@ +/*- + * =LICENSE= + * Kotlin Spark API: API for Spark 3.2+ (Scala 2.12) + * ---------- + * Copyright (C) 2019 - 2022 JetBrains + * ---------- + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * =LICENSEEND= + */ +package org.jetbrains.kotlinx.spark.examples + +import org.apache.spark.streaming.dstream.PairDStreamFunctions +import org.intellij.lang.annotations.Language +import kotlin.reflect.KFunction +import kotlin.reflect.full.functions + + +object GeneratePairStreamingFunctions { + +// fun JavaDStream>.reduceByKey(func: (V, V) -> V): JavaDStream> = +// mapToPair { it.toTuple() } +// .reduceByKey(func) +// .map { it.toArity() } + + @JvmStatic + fun main(args: Array) { + + val klass = PairDStreamFunctions::class + + val functions = klass.functions + + for (function: KFunction<*> in functions) with(function) { + + val types = (typeParameters.map { it.name }.toSet() + "K" + "V").joinToString() + + val parameterString = parameters.drop(1).joinToString { + "${it.name}: ${it.type}" + } + val parameterStringNoType = parameters.drop(1).joinToString { it.name!! } + + @Language("kt") + val new = """ + fun <$types> JavaDStream>.$name($parameterString) + + """.trimIndent() + +// +// val new = +// if (returnType.toString().contains("org.apache.spark.streaming.api.java.JavaPairDStream")) { +// val newReturnType = returnType.toString() +// .replaceFirst("JavaPairDStream<", "JavaDStream", ">") +// +// """ +// fun <$types> JavaDStream>.$name($parameterString): $newReturnType = +// mapToPair { it.toTuple() } +// .$name($parameterStringNoType) +// .map { it.toArity() } +// +// """.trimIndent() +// } else { +// """ +// fun <$types> JavaDStream>.$name($parameterString): $returnType = +// mapToPair { it.toTuple() } +// .$name($parameterStringNoType) +// +// """.trimIndent() +// } +// .replace("!", "") +// .replace("(Mutable)", "") +// +// if ("\$" !in new) println(new) + } + + + } +} diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/JavaRecoverableNetworkWordCount.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/JavaRecoverableNetworkWordCount.kt new file mode 100644 index 00000000..ef2fc398 --- /dev/null +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/JavaRecoverableNetworkWordCount.kt @@ -0,0 +1,215 @@ +/*- + * =LICENSE= + * Kotlin Spark API: Examples for Spark 3.2+ (Scala 2.12) + * ---------- + * Copyright (C) 2019 - 2022 JetBrains + * ---------- + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * =LICENSEEND= + */ +package org.jetbrains.kotlinx.spark.examples + +import com.google.common.io.Files +import org.apache.spark.api.java.JavaPairRDD +import org.apache.spark.api.java.JavaSparkContext +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.streaming.Durations +import org.apache.spark.streaming.Time +import org.apache.spark.util.LongAccumulator +import org.jetbrains.kotlinx.spark.api.* +import scala.Tuple2 +import java.io.File +import java.nio.charset.Charset +import java.util.regex.Pattern +import kotlin.system.exitProcess + + +/** + * Use this singleton to get or register a Broadcast variable. + */ +internal object JavaWordExcludeList { + + @Volatile + private var instance: Broadcast>? = null + + fun getInstance(sc: JavaSparkContext): Broadcast> { + if (instance == null) synchronized(JavaWordExcludeList::class.java) { + if (instance == null) { + val wordExcludeList = listOf("a", "b", "c") + instance = sc.broadcast(wordExcludeList) + } + } + return instance!! + } +} + +/** + * Use this singleton to get or register an Accumulator. + */ +internal object JavaDroppedWordsCounter { + + @Volatile + private var instance: LongAccumulator? = null + + fun getInstance(sc: JavaSparkContext): LongAccumulator { + if (instance == null) synchronized(JavaDroppedWordsCounter::class.java) { + if (instance == null) + instance = sc.sc().longAccumulator("DroppedWordsCounter") + } + return instance!! + } +} + +/** + * Counts words in text encoded with UTF8 received from the network every second. This example also + * shows how to use lazily instantiated singleton instances for Accumulator and Broadcast so that + * they can be registered on driver failures. + * + * Usage: JavaRecoverableNetworkWordCount + * and describe the TCP server that Spark Streaming would connect to receive + * data. directory to HDFS-compatible file system which checkpoint data + * file to which the word counts will be appended + * + * and must be absolute paths + * + * To run this on your local machine, you need to first run a Netcat server + * + * `$ nc -lk 9999` + * + * and run the example as + * + * `$ ./bin/run-example org.apache.spark.examples.streaming.JavaRecoverableNetworkWordCount \ + * localhost 9999 ~/checkpoint/ ~/out` + * + * If the directory ~/checkpoint/ does not exist (e.g. running for the first time), it will create + * a new StreamingContext (will print "Creating new context" to the console). Otherwise, if + * checkpoint data exists in ~/checkpoint/, then it will create StreamingContext from + * the checkpoint data. + * + * Refer to the online documentation for more details. + */ +object JavaRecoverableNetworkWordCount { + + private val SPACE = Pattern.compile(" ") + + private const val DEFAULT_IP = "localhost" + private const val DEFAULT_PORT = "9999" + private const val DEFAULT_CHECKPOINT_DIRECTORY = "~/checkpoint/" + private const val DEFAULT_OUTPUT_PATH = "~/out" + + @Throws(Exception::class) + @JvmStatic + fun main(args: Array) { + if (args.size != 4 && args.isNotEmpty()) { + System.err.println("You arguments were " + listOf(*args)) + System.err.println( + """Usage: JavaRecoverableNetworkWordCount + . and describe the TCP server that Spark + Streaming would connect to receive data. directory to + HDFS-compatible file system which checkpoint data file to which + the word counts will be appended + + In local mode, should be 'local[n]' with n > 1 + Both and must be absolute paths""".trimIndent() + ) + exitProcess(1) + } + val ip = args.getOrElse(0) { DEFAULT_IP } + val port = args.getOrElse(1) { DEFAULT_PORT }.toInt() + val checkpointDirectory = args.getOrElse(2) { DEFAULT_CHECKPOINT_DIRECTORY } + val outputPath = args.getOrElse(3) { DEFAULT_OUTPUT_PATH } + + // (used to detect the new context) + // Create the context with a 1 second batch size or load from checkpointDirectory + withSparkStreaming( +// checkpointPath = checkpointDirectory, TODO + batchDuration = Durations.seconds(1), + appName = "JavaRecoverableNetworkWordCount", + ) { + createContext( + ip = ip, + port = port, + outputPath = outputPath, + ) + } + } + + @Suppress("UnstableApiUsage") + private fun KSparkStreamingSession.createContext( + ip: String, + port: Int, + outputPath: String, + ) { + // If you do not see this printed, that means the StreamingContext has been loaded + // from the new checkpoint + println("Creating new context") + val outputFile = File(outputPath) + if (outputFile.exists()) outputFile.delete() + + // Create a socket stream on target ip:port and count the + // words in input stream of \n delimited text (e.g. generated by 'nc') + val lines = ssc.socketTextStream(ip, port) + + val words = lines.flatMap { it.split(SPACE).iterator() } + + val wordCounts = words + .mapToPair { c(it, 1).toTuple() } + .reduceByKey { a: Int, b: Int -> a + b } + +// val wordCounts = words +// .mapToPair { Tuple2(it, 1) } +// .reduceByKey { a: Int, b: Int -> a + b } + +// val wordCounts = words +// .map { it to 1 } +// .reduceByKey { a: Int, b: Int -> a + b } +// +// val wordCounts = words +// .map { c(it, 1) } +// .reduceByKey { a: Int, b: Int -> a + b } + + + wordCounts.foreachRDD { rdd, time: Time -> + + // Get or register the excludeList Broadcast + val excludeList = JavaWordExcludeList.getInstance(JavaSparkContext(rdd.context())) + + // Get or register the droppedWordsCounter Accumulator + val droppedWordsCounter = JavaDroppedWordsCounter.getInstance(JavaSparkContext(rdd.context())) + + // Use excludeList to drop words and use droppedWordsCounter to count them + val counts = rdd.filter { wordCount -> + if (excludeList.value().contains(wordCount._1)) { + droppedWordsCounter.add(wordCount._2.toLong()) + false + } else { + true + } + }.collect().toString() + val output = "Counts at time $time $counts" + println(output) + println("Dropped ${droppedWordsCounter.value()} word(s) totally") + println("Appending to " + outputFile.absolutePath) + Files.append( + """ + $output + + """.trimIndent(), + outputFile, + Charset.defaultCharset(), + ) + } + } + + +} diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/KotlinDirectKafkaWordCount.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/KotlinDirectKafkaWordCount.kt new file mode 100644 index 00000000..eea40720 --- /dev/null +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/KotlinDirectKafkaWordCount.kt @@ -0,0 +1,113 @@ +/*- + * =LICENSE= + * Kotlin Spark API: Examples for Spark 3.2+ (Scala 2.12) + * ---------- + * Copyright (C) 2019 - 2022 JetBrains + * ---------- + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * =LICENSEEND= + */ +package org.jetbrains.kotlinx.spark.examples + +import org.apache.kafka.clients.consumer.ConsumerConfig.* +import org.apache.kafka.clients.consumer.ConsumerRecord +import org.apache.kafka.common.serialization.StringDeserializer +import org.apache.spark.streaming.Durations +import org.apache.spark.streaming.api.java.JavaDStream +import org.apache.spark.streaming.api.java.JavaInputDStream +import org.apache.spark.streaming.api.java.JavaPairDStream +import org.apache.spark.streaming.kafka010.ConsumerStrategies +import org.apache.spark.streaming.kafka010.KafkaUtils +import org.apache.spark.streaming.kafka010.LocationStrategies +import org.jetbrains.kotlinx.spark.api.c +import org.jetbrains.kotlinx.spark.api.toTuple +import org.jetbrains.kotlinx.spark.api.withSparkStreaming +import java.io.Serializable +import java.util.regex.Pattern +import kotlin.system.exitProcess + + +/** + * Src: https://github.com/apache/spark/blob/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaDirectKafkaWordCount.java + * + * Consumes messages from one or more topics in Kafka and does wordcount. + * Usage: JavaDirectKafkaWordCount + * is a list of one or more Kafka brokers + * is a consumer group name to consume from topics + * is a list of one or more kafka topics to consume from + * + * Example: + * + * First make sure you have a Kafka producer running. For instance, when running locally: + * $ kafka-console-producer.sh --topic quickstart-events --bootstrap-server localhost:9092 + * + * Then start the program normally or like this: + * $ bin/run-example streaming.JavaDirectKafkaWordCount broker1-host:port,broker2-host:port \ + * consumer-group topic1,topic2 + */ +object KotlinDirectKafkaWordCount { + private val SPACE = Pattern.compile(" ") + + private const val DEFAULT_BROKER = "localhost:9092" + private const val DEFAULT_GROUP_ID = "consumer-group" + private const val DEFAULT_TOPIC = "quickstart-events" + + @JvmStatic + fun main(args: Array) { + if (args.size < 3 && args.isNotEmpty()) { + System.err.println( + """Usage: JavaDirectKafkaWordCount + is a list of one or more Kafka brokers + is a consumer group name to consume from topics + is a list of one or more kafka topics to consume from + """.trimIndent() + ) + exitProcess(1) + } + + val brokers: String = args.getOrElse(0) { DEFAULT_BROKER } + val groupId: String = args.getOrElse(1) { DEFAULT_GROUP_ID } + val topics: String = args.getOrElse(2) { DEFAULT_TOPIC } + + // Create context with a 2 seconds batch interval + withSparkStreaming(batchDuration = Durations.seconds(2), appName = "JavaDirectKafkaWordCount") { + + val topicsSet: Set = topics.split(',').toSet() + + val kafkaParams: Map = mapOf( + BOOTSTRAP_SERVERS_CONFIG to brokers, + GROUP_ID_CONFIG to groupId, + KEY_DESERIALIZER_CLASS_CONFIG to StringDeserializer::class.java, + VALUE_DESERIALIZER_CLASS_CONFIG to StringDeserializer::class.java, + ) + + // Create direct kafka stream with brokers and topics + val messages: JavaInputDStream> = KafkaUtils.createDirectStream( + ssc, + LocationStrategies.PreferConsistent(), + ConsumerStrategies.Subscribe(topicsSet, kafkaParams), + ) + + // Get the lines, split them into words, count the words and print + val lines: JavaDStream = messages.map { it.value() } + val words: JavaDStream = lines.flatMap { it.split(SPACE).iterator() } + + val wordCounts: JavaPairDStream = words + .mapToPair { c(it, 1).toTuple() } + .reduceByKey { a: Int, b: Int -> a + b } + + wordCounts.print() + + } + } +} diff --git a/kotlin-spark-api/3.2/pom_2.12.xml b/kotlin-spark-api/3.2/pom_2.12.xml index 826547d2..9a33982a 100644 --- a/kotlin-spark-api/3.2/pom_2.12.xml +++ b/kotlin-spark-api/3.2/pom_2.12.xml @@ -28,6 +28,22 @@ core-3.2_${scala.compat.version} + + org.apache.spark + spark-sql-kafka-0-10_${scala.compat.version} + ${spark3.version} + + + org.apache.spark + spark-streaming-kafka-0-10_${scala.compat.version} + ${spark3.version} + + + org.apache.kafka + kafka-streams-test-utils + 3.1.0 + test + diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt index 8b67a1bc..2863ad08 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt @@ -27,6 +27,7 @@ package org.jetbrains.kotlinx.spark.api +import org.apache.spark.api.java.Optional import scala.* import scala.collection.JavaConverters import java.util.* @@ -43,6 +44,19 @@ import scala.collection.mutable.Map as ScalaMutableMap import scala.collection.mutable.Seq as ScalaMutableSeq import scala.collection.mutable.Set as ScalaMutableSet + +/** Converts Scala [Option] to Kotlin nullable. */ +fun Option.toNullable(): T? = getOrElse { null } + +/** Converts nullable value to Scala [Option]. */ +fun T?.toOption(): Option = Option.apply(this) + +/** Converts [Optional] to Kotlin nullable. */ +fun Optional.toNullable(): T? = orElse(null) + +/** Converts nullable value to [Optional]. */ +fun T?.toOptional(): Optional = Optional.ofNullable(this) + /** * @see JavaConverters.asScalaIterator for more information. */ diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt index 96f753c7..d4481f1d 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt @@ -34,6 +34,7 @@ import org.apache.spark.sql.Dataset import org.apache.spark.sql.SparkSession.Builder import org.apache.spark.sql.UDFRegistration import org.apache.spark.streaming.Duration +import org.apache.spark.streaming.Durations import org.apache.spark.streaming.api.java.JavaStreamingContext import org.jetbrains.kotlinx.spark.api.SparkLogLevel.ERROR import org.jetbrains.kotlinx.spark.extensions.KSparkExtensions @@ -193,8 +194,15 @@ inline fun withSpark(sparkConf: SparkConf, logLevel: SparkLogLevel = ERROR, func /** * Wrapper for spark streaming creation. `spark: SparkSession` and `ssc: JavaStreamingContext` are provided, started, * awaited, and stopped automatically. + * The use of a checkpoint directory is optional. + * If checkpoint data exists in the provided `checkpointPath`, then StreamingContext will be + * recreated from the checkpoint data. If the data does not exist, then the provided factory + * will be used to create a JavaStreamingContext. * - * @param batchDuration The time interval at which streaming data will be divided into batches + * @param batchDuration The time interval at which streaming data will be divided into batches. Defaults to 1 second. + * @param checkpointPath If checkpoint data exists in the provided `checkpointPath`, then StreamingContext will be + * recreated from the checkpoint data. If the data does not exist (or `null` is provided), then the streaming context will be built using + * the other provided parameters. * @param props spark options, value types are runtime-checked for type-correctness * @param master Sets the Spark master URL to connect to, such as "local" to run locally, "local[4]" to * run locally with 4 cores, or "spark://master:7077" to run on a Spark standalone cluster. By default, it @@ -208,7 +216,8 @@ inline fun withSpark(sparkConf: SparkConf, logLevel: SparkLogLevel = ERROR, func */ @JvmOverloads inline fun withSparkStreaming( - batchDuration: Duration, + batchDuration: Duration = Durations.seconds(1L), + checkpointPath: String? = null, props: Map = emptyMap(), master: String = SparkConf().get("spark.master", "local[*]"), appName: String = "Kotlin Spark Sample", @@ -216,19 +225,53 @@ inline fun withSparkStreaming( timeout: Long = -1L, func: KSparkStreamingSession.() -> Unit, ) { - withSpark( - props = props, - master = master, - appName = appName, - logLevel = logLevel, - ) { - val ssc = JavaStreamingContext(sc, batchDuration) - KSparkStreamingSession(spark = spark, ssc = ssc).apply { - func() - ssc.start() - runAfterStart() + if (checkpointPath != null) { + TODO() +// var kSparkStreamingSession: KSparkStreamingSession? = null +// val ssc = JavaStreamingContext.getOrCreate(checkpointPath) { +// val jssc = JavaStreamingContext( +// SparkConf() +// .setAppName(appName) +// .setMaster(master) +// .setAll(props.map { (key, value) -> +// c(key, value.toString()).toTuple() +// }.asScalaIterable()), +// batchDuration, +// ) +// jssc.sparkContext().sc().setLogLevel(logLevel) +// jssc.checkpoint(checkpointPath) +// kSparkStreamingSession = KSparkStreamingSession( +// spark = SparkSession +// .builder() +// .sparkContext(jssc.sparkContext().sc()) +// .getOrCreate(), +// ssc = jssc, +// ).apply { func() } +// +// jssc +// } +// ssc.start() +// kSparkStreamingSession?.apply { runAfterStart() } +// ssc.awaitTerminationOrTimeout(timeout) +// ssc.stop() + } else { + + withSpark( + props = props, + master = master, + appName = appName, + logLevel = logLevel, + ) { + val ssc = JavaStreamingContext(sc, batchDuration) + KSparkStreamingSession(spark = spark, ssc = ssc).apply { + func() + ssc.start() + runAfterStart() + } + ssc.awaitTerminationOrTimeout(timeout) ssc.stop() + } } } diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Streaming.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Streaming.kt new file mode 100644 index 00000000..7fc71530 --- /dev/null +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Streaming.kt @@ -0,0 +1,603 @@ +/*- + * =LICENSE= + * Kotlin Spark API: API for Spark 3.2+ (Scala 2.12) + * ---------- + * Copyright (C) 2019 - 2022 JetBrains + * ---------- + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * =LICENSEEND= + */ +package org.jetbrains.kotlinx.spark.api + +import org.apache.spark.Partitioner +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.api.java.Optional +import org.apache.spark.streaming.Duration +import org.apache.spark.streaming.StateSpec +import org.apache.spark.streaming.api.java.JavaDStream +import org.apache.spark.streaming.api.java.JavaDStreamLike +import org.apache.spark.streaming.api.java.JavaMapWithStateDStream +import org.apache.spark.streaming.api.java.JavaPairDStream +import scala.Tuple2 +import scala.Tuple3 + +//fun JavaDStreamLike, *, *>.reduceByKey(func: (V, V) -> V): JavaDStream> = +// mapToPair(Arity2::toTuple) +// .reduceByKey(func) +// .map(Tuple2::toArity) + + +@JvmName("tuple2ToPairDStream") +fun JavaDStreamLike, *, *>.toPairDStream(): JavaPairDStream = + mapToPair { it } + +@JvmName("arity2ToPairDStream") +fun JavaDStreamLike, *, *>.toPairDStream(): JavaPairDStream = + mapToPair(Arity2::toTuple) + +@JvmName("pairToPairDStream") +fun JavaDStreamLike, *, *>.toPairDStream(): JavaPairDStream = + mapToPair(Pair::toTuple) + +/** + * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to + * generate the RDDs with Spark's default number of partitions. + */ +fun JavaDStreamLike, *, *>.groupByKey(): JavaDStream>> = + mapToPair { it.toTuple() } + .groupByKey() + .map { it.toArity() } + +/** + * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to + * generate the RDDs with `numPartitions` partitions. + */ +fun JavaDStreamLike, *, *>.groupByKey(numPartitions: Int): JavaDStream>> = + mapToPair { it.toTuple() } + .groupByKey(numPartitions) + .map { it.toArity() } + +/** + * Return a new DStream by applying `groupByKey` on each RDD. The supplied + * org.apache.spark.Partitioner is used to control the partitioning of each RDD. + */ +fun JavaDStreamLike, *, *>.groupByKey(partitioner: Partitioner): JavaDStream>> = + mapToPair { it.toTuple() } + .groupByKey(partitioner) + .map { it.toArity() } + +/** + * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are + * merged using the associative and commutative reduce function. Hash partitioning is used to + * generate the RDDs with Spark's default number of partitions. + */ +fun JavaDStreamLike, *, *>.reduceByKey(reduceFunc: (V, V) -> V): JavaDStream> = + mapToPair { it.toTuple() } + .reduceByKey(reduceFunc) + .map { it.toArity() } + +/** + * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are + * merged using the supplied reduce function. Hash partitioning is used to generate the RDDs + * with `numPartitions` partitions. + */ +fun JavaDStreamLike, *, *>.reduceByKey( + reduceFunc: (V, V) -> V, + numPartitions: Int, +): JavaDStream> = + mapToPair { it.toTuple() } + .reduceByKey(reduceFunc, numPartitions) + .map { it.toArity() } + +/** + * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are + * merged using the supplied reduce function. org.apache.spark.Partitioner is used to control + * the partitioning of each RDD. + */ +fun JavaDStreamLike, *, *>.reduceByKey( + reduceFunc: (V, V) -> V, + partitioner: Partitioner, +): JavaDStream> = + mapToPair { it.toTuple() } + .reduceByKey(reduceFunc, partitioner) + .map { it.toArity() } + +/** + * Combine elements of each key in DStream's RDDs using custom functions. This is similar to the + * combineByKey for RDDs. Please refer to combineByKey in + * org.apache.spark.rdd.PairRDDFunctions in the Spark core documentation for more information. + */ +fun JavaDStreamLike, *, *>.combineByKey( + createCombiner: (V) -> C, + mergeValue: (C, V) -> C, + mergeCombiner: (C, C) -> C, + partitioner: Partitioner, + mapSideCombine: Boolean = true, +): JavaDStream> = + mapToPair { it.toTuple() } + .combineByKey(createCombiner, mergeValue, mergeCombiner, partitioner, mapSideCombine) + .map { it.toArity() } + +/** + * Return a new DStream by applying `groupByKey` over a sliding window on `this` DStream. + * Similar to `DStream.groupByKey()`, but applies it over a sliding window. + * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. + * @param windowDuration width of the window; must be a multiple of this DStream's + * batching interval + * @param slideDuration sliding interval of the window (i.e., the interval after which + * the new DStream will generate RDDs); must be a multiple of this + * DStream's batching interval + * @param numPartitions number of partitions of each RDD in the new DStream; if not specified + * then Spark's default number of partitions will be used + */ +fun JavaDStreamLike, *, *>.groupByKeyAndWindow( + windowDuration: Duration, + slideDuration: Duration = dstream().slideDuration(), + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +): JavaDStream>> = + mapToPair { it.toTuple() } + .groupByKeyAndWindow(windowDuration, slideDuration, numPartitions) + .map { it.toArity() } + +/** + * Create a new DStream by applying `groupByKey` over a sliding window on `this` DStream. + * Similar to `DStream.groupByKey()`, but applies it over a sliding window. + * @param windowDuration width of the window; must be a multiple of this DStream's + * batching interval + * @param slideDuration sliding interval of the window (i.e., the interval after which + * the new DStream will generate RDDs); must be a multiple of this + * DStream's batching interval + * @param partitioner partitioner for controlling the partitioning of each RDD in the new + * DStream. + */ +fun JavaDStreamLike, *, *>.groupByKeyAndWindow( + windowDuration: Duration, + slideDuration: Duration = dstream().slideDuration(), + partitioner: Partitioner, +): JavaDStream>> = + mapToPair { it.toTuple() } + .groupByKeyAndWindow(windowDuration, slideDuration, partitioner) + .map { it.toArity() } + +/** + * Return a new DStream by applying `reduceByKey` over a sliding window. This is similar to + * `DStream.reduceByKey()` but applies it over a sliding window. Hash partitioning is used to + * generate the RDDs with `numPartitions` partitions. + * @param reduceFunc associative and commutative reduce function + * @param windowDuration width of the window; must be a multiple of this DStream's + * batching interval + * @param slideDuration sliding interval of the window (i.e., the interval after which + * the new DStream will generate RDDs); must be a multiple of this + * DStream's batching interval + * @param numPartitions number of partitions of each RDD in the new DStream. + */ +fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( + reduceFunc: (V, V) -> V, + windowDuration: Duration, + slideDuration: Duration = dstream().slideDuration(), + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +): JavaDStream> = + mapToPair { it.toTuple() } + .reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, numPartitions) + .map { it.toArity() } + +/** + * Return a new DStream by applying `reduceByKey` over a sliding window. Similar to + * `DStream.reduceByKey()`, but applies it over a sliding window. + * @param reduceFunc associative and commutative reduce function + * @param windowDuration width of the window; must be a multiple of this DStream's + * batching interval + * @param slideDuration sliding interval of the window (i.e., the interval after which + * the new DStream will generate RDDs); must be a multiple of this + * DStream's batching interval + * @param partitioner partitioner for controlling the partitioning of each RDD + * in the new DStream. + */ +fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( + reduceFunc: (V, V) -> V, + windowDuration: Duration, + slideDuration: Duration = dstream().slideDuration(), + partitioner: Partitioner, +): JavaDStream> = + mapToPair { it.toTuple() } + .reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, partitioner) + .map { it.toArity() } + +/** + * Return a new DStream by applying incremental `reduceByKey` over a sliding window. + * The reduced value of over a new window is calculated using the old window's reduced value : + * 1. reduce the new values that entered the window (e.g., adding new counts) + * + * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) + * + * This is more efficient than reduceByKeyAndWindow without "inverse reduce" function. + * However, it is applicable to only "invertible reduce functions". + * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. + * @param reduceFunc associative and commutative reduce function + * @param invReduceFunc inverse reduce function; such that for all y, invertible x: + * `invReduceFunc(reduceFunc(x, y), x) = y` + * @param windowDuration width of the window; must be a multiple of this DStream's + * batching interval + * @param slideDuration sliding interval of the window (i.e., the interval after which + * the new DStream will generate RDDs); must be a multiple of this + * DStream's batching interval + * @param filterFunc Optional function to filter expired key-value pairs; + * only pairs that satisfy the function are retained + */ +fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( + reduceFunc: (V, V) -> V, + invReduceFunc: (V, V) -> V, + windowDuration: Duration, + slideDuration: Duration = dstream().slideDuration(), + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), + filterFunc: ((Arity2) -> Boolean)? = null, +): JavaDStream> = + mapToPair { it.toTuple() } + .reduceByKeyAndWindow( + reduceFunc, + invReduceFunc, + windowDuration, + slideDuration, + numPartitions, + filterFunc?.let { + { tuple -> + filterFunc(tuple.toArity()) + } + } + ) + .map { it.toArity() } + +/** + * Return a new DStream by applying incremental `reduceByKey` over a sliding window. + * The reduced value of over a new window is calculated using the old window's reduced value : + * 1. reduce the new values that entered the window (e.g., adding new counts) + * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) + * This is more efficient than reduceByKeyAndWindow without "inverse reduce" function. + * However, it is applicable to only "invertible reduce functions". + * @param reduceFunc associative and commutative reduce function + * @param invReduceFunc inverse reduce function + * @param windowDuration width of the window; must be a multiple of this DStream's + * batching interval + * @param slideDuration sliding interval of the window (i.e., the interval after which + * the new DStream will generate RDDs); must be a multiple of this + * DStream's batching interval + * @param partitioner partitioner for controlling the partitioning of each RDD in the new + * DStream. + * @param filterFunc Optional function to filter expired key-value pairs; + * only pairs that satisfy the function are retained + */ +fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( + reduceFunc: (V, V) -> V, + invReduceFunc: (V, V) -> V, + windowDuration: Duration, + slideDuration: Duration = dstream().slideDuration(), + partitioner: Partitioner, + filterFunc: ((Arity2) -> Boolean)? = null, +): JavaDStream> = + mapToPair { it.toTuple() } + .reduceByKeyAndWindow( + reduceFunc, + invReduceFunc, + windowDuration, + slideDuration, + partitioner, + filterFunc?.let { + { tuple -> + filterFunc(tuple.toArity()) + } + } + ) + .map { it.toArity() } + +/** + * Return a [MapWithStateDStream] by applying a function to every key-value element of + * `this` stream, while maintaining some state data for each unique key. The mapping function + * and other specification (e.g. partitioners, timeouts, initial state data, etc.) of this + * transformation can be specified using `StateSpec` class. The state data is accessible in + * as a parameter of type `State` in the mapping function. + * + * Example of using `mapWithState`: + * {{{ + * // A mapping function that maintains an integer state and return a String + * def mappingFunction(key: String, value: Option[Int], state: State[Int]): Option[String] = { + * // Use state.exists(), state.get(), state.update() and state.remove() + * // to manage state, and return the necessary string + * } + * + * val spec = StateSpec.function(mappingFunction).numPartitions(10) + * + * val mapWithStateDStream = keyValueDStream.mapWithState[StateType, MappedType](spec) + * }}} + * + * @param spec Specification of this transformation + * @tparam StateType Class type of the state data + * @tparam MappedType Class type of the mapped data + */ +fun JavaDStreamLike, *, *>.mapWithState( + spec: StateSpec, +): JavaMapWithStateDStream = + mapToPair { it.toTuple() } + .mapWithState(spec) + +/** + * Return a new "state" DStream where the state for each key is updated by applying + * the given function on the previous state of the key and the new values of each key. + * In every batch the updateFunc will be called for each state even if there are no new values. + * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. + * @param updateFunc State update function. If `this` function returns None, then + * corresponding state key-value pair will be eliminated. + * @tparam S State type + */ +fun JavaDStreamLike, *, *>.updateStateByKey( + updateFunc: (List, S?) -> S?, + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +): JavaDStream> = + mapToPair { it.toTuple() } + .updateStateByKey( + { list: List, s: Optional -> + updateFunc(list, s.toNullable()).toOptional() + }, + numPartitions, + ) + .map { it.toArity() } + +/** + * Return a new "state" DStream where the state for each key is updated by applying + * the given function on the previous state of the key and the new values of each key. + * In every batch the updateFunc will be called for each state even if there are no new values. + * [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD. + * @param updateFunc State update function. Note, that this function may generate a different + * tuple with a different key than the input key. Therefore keys may be removed + * or added in this way. It is up to the developer to decide whether to + * remember the partitioner despite the key being changed. + * @param partitioner Partitioner for controlling the partitioning of each RDD in the new + * DStream + * @tparam S State type + */ +fun JavaDStreamLike, *, *>.updateStateByKey( + updateFunc: (List, S?) -> S?, + partitioner: Partitioner, +): JavaDStream> = + mapToPair { it.toTuple() } + .updateStateByKey( + { list: List, s: Optional -> + updateFunc(list, s.toNullable()).toOptional() + }, + partitioner, + ) + .map { it.toArity() } + +/** + * Return a new "state" DStream where the state for each key is updated by applying + * the given function on the previous state of the key and the new values of the key. + * org.apache.spark.Partitioner is used to control the partitioning of each RDD. + * @param updateFunc State update function. If `this` function returns None, then + * corresponding state key-value pair will be eliminated. + * @param partitioner Partitioner for controlling the partitioning of each RDD in the new + * DStream. + * @param initialRDD initial state value of each key. + * @tparam S State type + */ +fun JavaDStreamLike, *, *>.updateStateByKey( + updateFunc: (List, S?) -> S?, + partitioner: Partitioner, + initialRDD: JavaRDD>, +): JavaDStream> = + mapToPair { it.toTuple() } + .updateStateByKey( + { list: List, s: Optional -> + updateFunc(list, s.toNullable()).toOptional() + }, + partitioner, + initialRDD.mapToPair { it.toTuple() }, + ) + .map { it.toArity() } + +/** + * Return a new DStream by applying a map function to the value of each key-value pairs in + * 'this' DStream without changing the key. + */ +fun JavaDStreamLike, *, *>.mapValues( + mapValuesFunc: (V) -> U, +): JavaDStream> = + mapToPair { it.toTuple() } + .mapValues(mapValuesFunc) + .map { it.toArity() } + +/** + * Return a new DStream by applying a flatmap function to the value of each key-value pairs in + * 'this' DStream without changing the key. + */ +fun JavaDStreamLike, *, *>.flatMapValues( + flatMapValuesFunc: (V) -> Iterator, +): JavaDStream> = + mapToPair { it.toTuple() } + .flatMapValues(flatMapValuesFunc) + .map { it.toArity() } + +/** + * Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream. + * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. + */ +fun JavaDStreamLike, *, *>.cogroup( + other: JavaDStreamLike, *, *>, + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +): JavaDStream, Iterable>>> = + mapToPair { it.toTuple() } + .cogroup( + other.mapToPair { it.toTuple() }, + numPartitions, + ) + .map { + c(it._1, it._2.toArity()) + } + +/** + * Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream. + * The supplied org.apache.spark.Partitioner is used to partition the generated RDDs. + */ +fun JavaDStreamLike, *, *>.cogroup( + other: JavaDStreamLike, *, *>, + partitioner: Partitioner, +): JavaDStream, Iterable>>> = + mapToPair { it.toTuple() } + .cogroup( + other.mapToPair { it.toTuple() }, + partitioner, + ) + .map { + c(it._1, it._2.toArity()) + } + +/** + * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream. + * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. + */ +fun JavaDStreamLike, *, *>.join( + other: JavaDStreamLike, *, *>, + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +): JavaDStream>> = + mapToPair { it.toTuple() } + .join( + other.mapToPair { it.toTuple() }, + numPartitions, + ) + .map { + c(it._1, it._2.toArity()) + } + +/** + * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream. + * The supplied org.apache.spark.Partitioner is used to control the partitioning of each RDD. + */ +fun JavaDStreamLike, *, *>.join( + other: JavaDStreamLike, *, *>, + partitioner: Partitioner, +): JavaDStream>> = + mapToPair { it.toTuple() } + .join( + other.mapToPair { it.toTuple() }, + partitioner, + ) + .map { + c(it._1, it._2.toArity()) + } + +/** + * Return a new DStream by applying 'left outer join' between RDDs of `this` DStream and + * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` + * partitions. + */ +fun JavaDStreamLike, *, *>.leftOuterJoin( + other: JavaDStreamLike, *, *>, + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +): JavaDStream>> = + mapToPair { it.toTuple() } + .leftOuterJoin( + other.mapToPair { it.toTuple() }, + numPartitions, + ) + .map { + c(it._1, c(it._2._1, it._2._2.toNullable())) + } + +/** + * Return a new DStream by applying 'left outer join' between RDDs of `this` DStream and + * `other` DStream. The supplied org.apache.spark.Partitioner is used to control + * the partitioning of each RDD. + */ +fun JavaDStreamLike, *, *>.leftOuterJoin( + other: JavaDStreamLike, *, *>, + partitioner: Partitioner, +): JavaDStream>> = + mapToPair { it.toTuple() } + .leftOuterJoin( + other.mapToPair { it.toTuple() }, + partitioner, + ) + .map { + c(it._1, c(it._2._1, it._2._2.toNullable())) + } + +/** + * Return a new DStream by applying 'right outer join' between RDDs of `this` DStream and + * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` + * partitions. + */ +fun JavaDStreamLike, *, *>.rightOuterJoin( + other: JavaDStreamLike, *, *>, + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +): JavaDStream>> = + mapToPair { it.toTuple() } + .rightOuterJoin( + other.mapToPair { it.toTuple() }, + numPartitions, + ) + .map { + c(it._1, c(it._2._1.toNullable(), it._2._2)) + } + +/** + * Return a new DStream by applying 'right outer join' between RDDs of `this` DStream and + * `other` DStream. The supplied org.apache.spark.Partitioner is used to control + * the partitioning of each RDD. + */ +fun JavaDStreamLike, *, *>.rightOuterJoin( + other: JavaDStreamLike, *, *>, + partitioner: Partitioner, +): JavaDStream>> = + mapToPair { it.toTuple() } + .rightOuterJoin( + other.mapToPair { it.toTuple() }, + partitioner, + ) + .map { + c(it._1, c(it._2._1.toNullable(), it._2._2)) + } + +/** + * Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and + * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` + * partitions. + */ +fun JavaDStreamLike, *, *>.fullOuterJoin( + other: JavaDStreamLike, *, *>, + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +): JavaDStream>> = + mapToPair { it.toTuple() } + .fullOuterJoin( + other.mapToPair { it.toTuple() }, + numPartitions, + ) + .map { + c(it._1, c(it._2._1.toNullable(), it._2._2.toNullable())) + } + +/** + * Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and + * `other` DStream. The supplied org.apache.spark.Partitioner is used to control + * the partitioning of each RDD. + */ +fun JavaDStreamLike, *, *>.fullOuterJoin( + other: JavaDStreamLike, *, *>, + partitioner: Partitioner, +): JavaDStream>> = + mapToPair { it.toTuple() } + .fullOuterJoin( + other.mapToPair { it.toTuple() }, + partitioner, + ) + .map { + c(it._1, c(it._2._1.toNullable(), it._2._2.toNullable())) + } + diff --git a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt index e0139b9c..d93c7d23 100644 --- a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt +++ b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt @@ -1,3 +1,22 @@ +/*- + * =LICENSE= + * Kotlin Spark API: API for Spark 3.2+ (Scala 2.12) + * ---------- + * Copyright (C) 2019 - 2022 JetBrains + * ---------- + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * =LICENSEEND= + */ package org.jetbrains.kotlinx.spark.api import io.kotest.core.spec.style.ShouldSpec @@ -39,4 +58,4 @@ class StreamingTest : ShouldSpec({ } } -}) \ No newline at end of file +}) From 5769e925f705edd74c659d798c2c865d8d44eb54 Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Thu, 17 Mar 2022 19:41:32 +0100 Subject: [PATCH 12/44] still exploring, just pushing to keep my work safe :) --- .../jetbrains/kotlinx/spark/api/Streaming.kt | 20 +++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Streaming.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Streaming.kt index 7fc71530..b551b081 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Streaming.kt +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Streaming.kt @@ -19,6 +19,7 @@ */ package org.jetbrains.kotlinx.spark.api +import com.sun.org.apache.xml.internal.serialize.OutputFormat import org.apache.spark.Partitioner import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.java.Optional @@ -601,3 +602,22 @@ fun JavaDStreamLike, *, *>.fullOuterJoin( c(it._1, c(it._2._1.toNullable(), it._2._2.toNullable())) } +/** + * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is + * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix". + */ +fun JavaDStreamLike, *, *>.saveAsHadoopFiles( + prefix: String, suffix: String, +): Unit = + mapToPair { it.toTuple() } + .saveAsHadoopFiles(prefix, suffix) + +/** + * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is + * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix". + */ +fun JavaDStreamLike, *, *>.saveAsNewAPIHadoopFiles( + prefix: String, suffix: String, +): Unit = + mapToPair { it.toTuple() } + .saveAsNewAPIHadoopFiles(prefix, suffix) \ No newline at end of file From 14278ffb5c1eb6691d475642e9efd036fe225f16 Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Thu, 17 Mar 2022 20:07:14 +0100 Subject: [PATCH 13/44] pairs are recognized too --- .../JavaRecoverableNetworkWordCount.kt | 17 +- .../jetbrains/kotlinx/spark/api/Streaming.kt | 657 +++++++++++++++++- 2 files changed, 631 insertions(+), 43 deletions(-) diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/JavaRecoverableNetworkWordCount.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/JavaRecoverableNetworkWordCount.kt index ef2fc398..f89e09f3 100644 --- a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/JavaRecoverableNetworkWordCount.kt +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/JavaRecoverableNetworkWordCount.kt @@ -163,21 +163,12 @@ object JavaRecoverableNetworkWordCount { val words = lines.flatMap { it.split(SPACE).iterator() } val wordCounts = words - .mapToPair { c(it, 1).toTuple() } + .map { c(it, 1) } .reduceByKey { a: Int, b: Int -> a + b } -// val wordCounts = words -// .mapToPair { Tuple2(it, 1) } -// .reduceByKey { a: Int, b: Int -> a + b } - -// val wordCounts = words -// .map { it to 1 } -// .reduceByKey { a: Int, b: Int -> a + b } -// -// val wordCounts = words -// .map { c(it, 1) } -// .reduceByKey { a: Int, b: Int -> a + b } - + val wordCounts2 = words + .map { it to 1 } + .reduceByKey { a: Int, b: Int -> a + b } wordCounts.foreachRDD { rdd, time: Time -> diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Streaming.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Streaming.kt index b551b081..f0152f83 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Streaming.kt +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Streaming.kt @@ -50,20 +50,14 @@ fun JavaDStreamLike, *, *>.toPairDStream(): JavaPairDStream< fun JavaDStreamLike, *, *>.toPairDStream(): JavaPairDStream = mapToPair(Pair::toTuple) -/** - * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to - * generate the RDDs with Spark's default number of partitions. - */ -fun JavaDStreamLike, *, *>.groupByKey(): JavaDStream>> = - mapToPair { it.toTuple() } - .groupByKey() - .map { it.toArity() } - /** * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to * generate the RDDs with `numPartitions` partitions. */ -fun JavaDStreamLike, *, *>.groupByKey(numPartitions: Int): JavaDStream>> = +@JvmName("groupByKeyArity2") +fun JavaDStreamLike, *, *>.groupByKey( + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +): JavaDStream>> = mapToPair { it.toTuple() } .groupByKey(numPartitions) .map { it.toArity() } @@ -72,29 +66,21 @@ fun JavaDStreamLike, *, *>.groupByKey(numPartitions: Int): J * Return a new DStream by applying `groupByKey` on each RDD. The supplied * org.apache.spark.Partitioner is used to control the partitioning of each RDD. */ +@JvmName("groupByKeyArity2") fun JavaDStreamLike, *, *>.groupByKey(partitioner: Partitioner): JavaDStream>> = mapToPair { it.toTuple() } .groupByKey(partitioner) .map { it.toArity() } -/** - * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are - * merged using the associative and commutative reduce function. Hash partitioning is used to - * generate the RDDs with Spark's default number of partitions. - */ -fun JavaDStreamLike, *, *>.reduceByKey(reduceFunc: (V, V) -> V): JavaDStream> = - mapToPair { it.toTuple() } - .reduceByKey(reduceFunc) - .map { it.toArity() } - /** * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are * merged using the supplied reduce function. Hash partitioning is used to generate the RDDs * with `numPartitions` partitions. */ +@JvmName("reduceByKeyArity2") fun JavaDStreamLike, *, *>.reduceByKey( + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), reduceFunc: (V, V) -> V, - numPartitions: Int, ): JavaDStream> = mapToPair { it.toTuple() } .reduceByKey(reduceFunc, numPartitions) @@ -105,9 +91,10 @@ fun JavaDStreamLike, *, *>.reduceByKey( * merged using the supplied reduce function. org.apache.spark.Partitioner is used to control * the partitioning of each RDD. */ +@JvmName("reduceByKeyArity2") fun JavaDStreamLike, *, *>.reduceByKey( - reduceFunc: (V, V) -> V, partitioner: Partitioner, + reduceFunc: (V, V) -> V, ): JavaDStream> = mapToPair { it.toTuple() } .reduceByKey(reduceFunc, partitioner) @@ -118,6 +105,7 @@ fun JavaDStreamLike, *, *>.reduceByKey( * combineByKey for RDDs. Please refer to combineByKey in * org.apache.spark.rdd.PairRDDFunctions in the Spark core documentation for more information. */ +@JvmName("combineByKeyArity2") fun JavaDStreamLike, *, *>.combineByKey( createCombiner: (V) -> C, mergeValue: (C, V) -> C, @@ -141,6 +129,7 @@ fun JavaDStreamLike, *, *>.combineByKey( * @param numPartitions number of partitions of each RDD in the new DStream; if not specified * then Spark's default number of partitions will be used */ +@JvmName("groupByKeyAndWindowArity2") fun JavaDStreamLike, *, *>.groupByKeyAndWindow( windowDuration: Duration, slideDuration: Duration = dstream().slideDuration(), @@ -161,6 +150,7 @@ fun JavaDStreamLike, *, *>.groupByKeyAndWindow( * @param partitioner partitioner for controlling the partitioning of each RDD in the new * DStream. */ +@JvmName("groupByKeyAndWindowArity2") fun JavaDStreamLike, *, *>.groupByKeyAndWindow( windowDuration: Duration, slideDuration: Duration = dstream().slideDuration(), @@ -182,11 +172,12 @@ fun JavaDStreamLike, *, *>.groupByKeyAndWindow( * DStream's batching interval * @param numPartitions number of partitions of each RDD in the new DStream. */ +@JvmName("reduceByKeyAndWindowArity2") fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( - reduceFunc: (V, V) -> V, windowDuration: Duration, slideDuration: Duration = dstream().slideDuration(), numPartitions: Int = dstream().ssc().sc().defaultParallelism(), + reduceFunc: (V, V) -> V, ): JavaDStream> = mapToPair { it.toTuple() } .reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, numPartitions) @@ -204,11 +195,12 @@ fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( * @param partitioner partitioner for controlling the partitioning of each RDD * in the new DStream. */ +@JvmName("reduceByKeyAndWindowArity2") fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( - reduceFunc: (V, V) -> V, windowDuration: Duration, slideDuration: Duration = dstream().slideDuration(), partitioner: Partitioner, + reduceFunc: (V, V) -> V, ): JavaDStream> = mapToPair { it.toTuple() } .reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, partitioner) @@ -235,13 +227,14 @@ fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( * @param filterFunc Optional function to filter expired key-value pairs; * only pairs that satisfy the function are retained */ +@JvmName("reduceByKeyAndWindowArity2") fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( - reduceFunc: (V, V) -> V, invReduceFunc: (V, V) -> V, windowDuration: Duration, slideDuration: Duration = dstream().slideDuration(), numPartitions: Int = dstream().ssc().sc().defaultParallelism(), filterFunc: ((Arity2) -> Boolean)? = null, + reduceFunc: (V, V) -> V, ): JavaDStream> = mapToPair { it.toTuple() } .reduceByKeyAndWindow( @@ -277,13 +270,14 @@ fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( * @param filterFunc Optional function to filter expired key-value pairs; * only pairs that satisfy the function are retained */ +@JvmName("reduceByKeyAndWindowArity2") fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( - reduceFunc: (V, V) -> V, invReduceFunc: (V, V) -> V, windowDuration: Duration, slideDuration: Duration = dstream().slideDuration(), partitioner: Partitioner, filterFunc: ((Arity2) -> Boolean)? = null, + reduceFunc: (V, V) -> V, ): JavaDStream> = mapToPair { it.toTuple() } .reduceByKeyAndWindow( @@ -324,6 +318,7 @@ fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( * @tparam StateType Class type of the state data * @tparam MappedType Class type of the mapped data */ +@JvmName("mapWithStateArity2") fun JavaDStreamLike, *, *>.mapWithState( spec: StateSpec, ): JavaMapWithStateDStream = @@ -339,9 +334,10 @@ fun JavaDStreamLike, *, *>.mapWithSta * corresponding state key-value pair will be eliminated. * @tparam S State type */ +@JvmName("updateStateByKeyArity2") fun JavaDStreamLike, *, *>.updateStateByKey( - updateFunc: (List, S?) -> S?, numPartitions: Int = dstream().ssc().sc().defaultParallelism(), + updateFunc: (List, S?) -> S?, ): JavaDStream> = mapToPair { it.toTuple() } .updateStateByKey( @@ -365,9 +361,10 @@ fun JavaDStreamLike, *, *>.updateStateByKey( * DStream * @tparam S State type */ +@JvmName("updateStateByKeyArity2") fun JavaDStreamLike, *, *>.updateStateByKey( - updateFunc: (List, S?) -> S?, partitioner: Partitioner, + updateFunc: (List, S?) -> S?, ): JavaDStream> = mapToPair { it.toTuple() } .updateStateByKey( @@ -389,10 +386,11 @@ fun JavaDStreamLike, *, *>.updateStateByKey( * @param initialRDD initial state value of each key. * @tparam S State type */ +@JvmName("updateStateByKeyArity2") fun JavaDStreamLike, *, *>.updateStateByKey( - updateFunc: (List, S?) -> S?, partitioner: Partitioner, initialRDD: JavaRDD>, + updateFunc: (List, S?) -> S?, ): JavaDStream> = mapToPair { it.toTuple() } .updateStateByKey( @@ -408,6 +406,7 @@ fun JavaDStreamLike, *, *>.updateStateByKey( * Return a new DStream by applying a map function to the value of each key-value pairs in * 'this' DStream without changing the key. */ +@JvmName("mapValuesArity2") fun JavaDStreamLike, *, *>.mapValues( mapValuesFunc: (V) -> U, ): JavaDStream> = @@ -419,6 +418,7 @@ fun JavaDStreamLike, *, *>.mapValues( * Return a new DStream by applying a flatmap function to the value of each key-value pairs in * 'this' DStream without changing the key. */ +@JvmName("flatMapValuesArity2") fun JavaDStreamLike, *, *>.flatMapValues( flatMapValuesFunc: (V) -> Iterator, ): JavaDStream> = @@ -430,6 +430,7 @@ fun JavaDStreamLike, *, *>.flatMapValues( * Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream. * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. */ +@JvmName("cogroupArity2") fun JavaDStreamLike, *, *>.cogroup( other: JavaDStreamLike, *, *>, numPartitions: Int = dstream().ssc().sc().defaultParallelism(), @@ -447,6 +448,7 @@ fun JavaDStreamLike, *, *>.cogroup( * Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream. * The supplied org.apache.spark.Partitioner is used to partition the generated RDDs. */ +@JvmName("cogroupArity2") fun JavaDStreamLike, *, *>.cogroup( other: JavaDStreamLike, *, *>, partitioner: Partitioner, @@ -464,6 +466,7 @@ fun JavaDStreamLike, *, *>.cogroup( * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream. * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. */ +@JvmName("joinArity2") fun JavaDStreamLike, *, *>.join( other: JavaDStreamLike, *, *>, numPartitions: Int = dstream().ssc().sc().defaultParallelism(), @@ -481,6 +484,7 @@ fun JavaDStreamLike, *, *>.join( * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream. * The supplied org.apache.spark.Partitioner is used to control the partitioning of each RDD. */ +@JvmName("joinArity2") fun JavaDStreamLike, *, *>.join( other: JavaDStreamLike, *, *>, partitioner: Partitioner, @@ -499,6 +503,7 @@ fun JavaDStreamLike, *, *>.join( * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` * partitions. */ +@JvmName("leftOuterJoinArity2") fun JavaDStreamLike, *, *>.leftOuterJoin( other: JavaDStreamLike, *, *>, numPartitions: Int = dstream().ssc().sc().defaultParallelism(), @@ -517,6 +522,7 @@ fun JavaDStreamLike, *, *>.leftOuterJoin( * `other` DStream. The supplied org.apache.spark.Partitioner is used to control * the partitioning of each RDD. */ +@JvmName("leftOuterJoinArity2") fun JavaDStreamLike, *, *>.leftOuterJoin( other: JavaDStreamLike, *, *>, partitioner: Partitioner, @@ -535,6 +541,7 @@ fun JavaDStreamLike, *, *>.leftOuterJoin( * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` * partitions. */ +@JvmName("rightOuterJoinArity2") fun JavaDStreamLike, *, *>.rightOuterJoin( other: JavaDStreamLike, *, *>, numPartitions: Int = dstream().ssc().sc().defaultParallelism(), @@ -553,6 +560,7 @@ fun JavaDStreamLike, *, *>.rightOuterJoin( * `other` DStream. The supplied org.apache.spark.Partitioner is used to control * the partitioning of each RDD. */ +@JvmName("rightOuterJoinArity2") fun JavaDStreamLike, *, *>.rightOuterJoin( other: JavaDStreamLike, *, *>, partitioner: Partitioner, @@ -571,6 +579,7 @@ fun JavaDStreamLike, *, *>.rightOuterJoin( * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` * partitions. */ +@JvmName("fullOuterJoinArity2") fun JavaDStreamLike, *, *>.fullOuterJoin( other: JavaDStreamLike, *, *>, numPartitions: Int = dstream().ssc().sc().defaultParallelism(), @@ -589,6 +598,7 @@ fun JavaDStreamLike, *, *>.fullOuterJoin( * `other` DStream. The supplied org.apache.spark.Partitioner is used to control * the partitioning of each RDD. */ +@JvmName("fullOuterJoinArity2") fun JavaDStreamLike, *, *>.fullOuterJoin( other: JavaDStreamLike, *, *>, partitioner: Partitioner, @@ -606,6 +616,7 @@ fun JavaDStreamLike, *, *>.fullOuterJoin( * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix". */ +@JvmName("saveAsHadoopFilesArity2") fun JavaDStreamLike, *, *>.saveAsHadoopFiles( prefix: String, suffix: String, ): Unit = @@ -616,8 +627,594 @@ fun JavaDStreamLike, *, *>.saveAsHadoopFiles( * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix". */ +@JvmName("saveAsNewAPIHadoopFilesArity2") fun JavaDStreamLike, *, *>.saveAsNewAPIHadoopFiles( prefix: String, suffix: String, ): Unit = mapToPair { it.toTuple() } - .saveAsNewAPIHadoopFiles(prefix, suffix) \ No newline at end of file + .saveAsNewAPIHadoopFiles(prefix, suffix) + +/** + * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to + * generate the RDDs with `numPartitions` partitions. + */ +@JvmName("groupByKeyPair") +fun JavaDStreamLike, *, *>.groupByKey( + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +): JavaDStream>> = + mapToPair { it.toTuple() } + .groupByKey(numPartitions) + .map { it.toPair() } + +/** + * Return a new DStream by applying `groupByKey` on each RDD. The supplied + * org.apache.spark.Partitioner is used to control the partitioning of each RDD. + */ +@JvmName("groupByKeyPair") +fun JavaDStreamLike, *, *>.groupByKey(partitioner: Partitioner): JavaDStream>> = + mapToPair { it.toTuple() } + .groupByKey(partitioner) + .map { it.toPair() } + +/** + * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are + * merged using the supplied reduce function. Hash partitioning is used to generate the RDDs + * with `numPartitions` partitions. + */ +@JvmName("reduceByKeyPair") +fun JavaDStreamLike, *, *>.reduceByKey( + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), + reduceFunc: (V, V) -> V, +): JavaDStream> = + mapToPair { it.toTuple() } + .reduceByKey(reduceFunc, numPartitions) + .map { it.toPair() } + +/** + * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are + * merged using the supplied reduce function. org.apache.spark.Partitioner is used to control + * the partitioning of each RDD. + */ +@JvmName("reduceByKeyPair") +fun JavaDStreamLike, *, *>.reduceByKey( + partitioner: Partitioner, + reduceFunc: (V, V) -> V, +): JavaDStream> = + mapToPair { it.toTuple() } + .reduceByKey(reduceFunc, partitioner) + .map { it.toPair() } + +/** + * Combine elements of each key in DStream's RDDs using custom functions. This is similar to the + * combineByKey for RDDs. Please refer to combineByKey in + * org.apache.spark.rdd.PairRDDFunctions in the Spark core documentation for more information. + */ +@JvmName("combineByKeyPair") +fun JavaDStreamLike, *, *>.combineByKey( + createCombiner: (V) -> C, + mergeValue: (C, V) -> C, + mergeCombiner: (C, C) -> C, + partitioner: Partitioner, + mapSideCombine: Boolean = true, +): JavaDStream> = + mapToPair { it.toTuple() } + .combineByKey(createCombiner, mergeValue, mergeCombiner, partitioner, mapSideCombine) + .map { it.toPair() } + +/** + * Return a new DStream by applying `groupByKey` over a sliding window on `this` DStream. + * Similar to `DStream.groupByKey()`, but applies it over a sliding window. + * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. + * @param windowDuration width of the window; must be a multiple of this DStream's + * batching interval + * @param slideDuration sliding interval of the window (i.e., the interval after which + * the new DStream will generate RDDs); must be a multiple of this + * DStream's batching interval + * @param numPartitions number of partitions of each RDD in the new DStream; if not specified + * then Spark's default number of partitions will be used + */ +@JvmName("groupByKeyAndWindowPair") +fun JavaDStreamLike, *, *>.groupByKeyAndWindow( + windowDuration: Duration, + slideDuration: Duration = dstream().slideDuration(), + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +): JavaDStream>> = + mapToPair { it.toTuple() } + .groupByKeyAndWindow(windowDuration, slideDuration, numPartitions) + .map { it.toPair() } + +/** + * Create a new DStream by applying `groupByKey` over a sliding window on `this` DStream. + * Similar to `DStream.groupByKey()`, but applies it over a sliding window. + * @param windowDuration width of the window; must be a multiple of this DStream's + * batching interval + * @param slideDuration sliding interval of the window (i.e., the interval after which + * the new DStream will generate RDDs); must be a multiple of this + * DStream's batching interval + * @param partitioner partitioner for controlling the partitioning of each RDD in the new + * DStream. + */ +@JvmName("groupByKeyAndWindowPair") +fun JavaDStreamLike, *, *>.groupByKeyAndWindow( + windowDuration: Duration, + slideDuration: Duration = dstream().slideDuration(), + partitioner: Partitioner, +): JavaDStream>> = + mapToPair { it.toTuple() } + .groupByKeyAndWindow(windowDuration, slideDuration, partitioner) + .map { it.toPair() } + +/** + * Return a new DStream by applying `reduceByKey` over a sliding window. This is similar to + * `DStream.reduceByKey()` but applies it over a sliding window. Hash partitioning is used to + * generate the RDDs with `numPartitions` partitions. + * @param reduceFunc associative and commutative reduce function + * @param windowDuration width of the window; must be a multiple of this DStream's + * batching interval + * @param slideDuration sliding interval of the window (i.e., the interval after which + * the new DStream will generate RDDs); must be a multiple of this + * DStream's batching interval + * @param numPartitions number of partitions of each RDD in the new DStream. + */ +@JvmName("reduceByKeyAndWindowPair") +fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( + windowDuration: Duration, + slideDuration: Duration = dstream().slideDuration(), + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), + reduceFunc: (V, V) -> V, +): JavaDStream> = + mapToPair { it.toTuple() } + .reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, numPartitions) + .map { it.toPair() } + +/** + * Return a new DStream by applying `reduceByKey` over a sliding window. Similar to + * `DStream.reduceByKey()`, but applies it over a sliding window. + * @param reduceFunc associative and commutative reduce function + * @param windowDuration width of the window; must be a multiple of this DStream's + * batching interval + * @param slideDuration sliding interval of the window (i.e., the interval after which + * the new DStream will generate RDDs); must be a multiple of this + * DStream's batching interval + * @param partitioner partitioner for controlling the partitioning of each RDD + * in the new DStream. + */ +@JvmName("reduceByKeyAndWindowPair") +fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( + windowDuration: Duration, + slideDuration: Duration = dstream().slideDuration(), + partitioner: Partitioner, + reduceFunc: (V, V) -> V, +): JavaDStream> = + mapToPair { it.toTuple() } + .reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, partitioner) + .map { it.toPair() } + +/** + * Return a new DStream by applying incremental `reduceByKey` over a sliding window. + * The reduced value of over a new window is calculated using the old window's reduced value : + * 1. reduce the new values that entered the window (e.g., adding new counts) + * + * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) + * + * This is more efficient than reduceByKeyAndWindow without "inverse reduce" function. + * However, it is applicable to only "invertible reduce functions". + * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. + * @param reduceFunc associative and commutative reduce function + * @param invReduceFunc inverse reduce function; such that for all y, invertible x: + * `invReduceFunc(reduceFunc(x, y), x) = y` + * @param windowDuration width of the window; must be a multiple of this DStream's + * batching interval + * @param slideDuration sliding interval of the window (i.e., the interval after which + * the new DStream will generate RDDs); must be a multiple of this + * DStream's batching interval + * @param filterFunc Optional function to filter expired key-value pairs; + * only pairs that satisfy the function are retained + */ +@JvmName("reduceByKeyAndWindowPair") +fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( + invReduceFunc: (V, V) -> V, + windowDuration: Duration, + slideDuration: Duration = dstream().slideDuration(), + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), + filterFunc: ((Pair) -> Boolean)? = null, + reduceFunc: (V, V) -> V, +): JavaDStream> = + mapToPair { it.toTuple() } + .reduceByKeyAndWindow( + reduceFunc, + invReduceFunc, + windowDuration, + slideDuration, + numPartitions, + filterFunc?.let { + { tuple -> + filterFunc(tuple.toPair()) + } + } + ) + .map { it.toPair() } + +/** + * Return a new DStream by applying incremental `reduceByKey` over a sliding window. + * The reduced value of over a new window is calculated using the old window's reduced value : + * 1. reduce the new values that entered the window (e.g., adding new counts) + * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) + * This is more efficient than reduceByKeyAndWindow without "inverse reduce" function. + * However, it is applicable to only "invertible reduce functions". + * @param reduceFunc associative and commutative reduce function + * @param invReduceFunc inverse reduce function + * @param windowDuration width of the window; must be a multiple of this DStream's + * batching interval + * @param slideDuration sliding interval of the window (i.e., the interval after which + * the new DStream will generate RDDs); must be a multiple of this + * DStream's batching interval + * @param partitioner partitioner for controlling the partitioning of each RDD in the new + * DStream. + * @param filterFunc Optional function to filter expired key-value pairs; + * only pairs that satisfy the function are retained + */ +@JvmName("reduceByKeyAndWindowPair") +fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( + invReduceFunc: (V, V) -> V, + windowDuration: Duration, + slideDuration: Duration = dstream().slideDuration(), + partitioner: Partitioner, + filterFunc: ((Pair) -> Boolean)? = null, + reduceFunc: (V, V) -> V, +): JavaDStream> = + mapToPair { it.toTuple() } + .reduceByKeyAndWindow( + reduceFunc, + invReduceFunc, + windowDuration, + slideDuration, + partitioner, + filterFunc?.let { + { tuple -> + filterFunc(tuple.toPair()) + } + } + ) + .map { it.toPair() } + +/** + * Return a [MapWithStateDStream] by applying a function to every key-value element of + * `this` stream, while maintaining some state data for each unique key. The mapping function + * and other specification (e.g. partitioners, timeouts, initial state data, etc.) of this + * transformation can be specified using `StateSpec` class. The state data is accessible in + * as a parameter of type `State` in the mapping function. + * + * Example of using `mapWithState`: + * {{{ + * // A mapping function that maintains an integer state and return a String + * def mappingFunction(key: String, value: Option[Int], state: State[Int]): Option[String] = { + * // Use state.exists(), state.get(), state.update() and state.remove() + * // to manage state, and return the necessary string + * } + * + * val spec = StateSpec.function(mappingFunction).numPartitions(10) + * + * val mapWithStateDStream = keyValueDStream.mapWithState[StateType, MappedType](spec) + * }}} + * + * @param spec Specification of this transformation + * @tparam StateType Class type of the state data + * @tparam MappedType Class type of the mapped data + */ +@JvmName("mapWithStatePair") +fun JavaDStreamLike, *, *>.mapWithState( + spec: StateSpec, +): JavaMapWithStateDStream = + mapToPair { it.toTuple() } + .mapWithState(spec) + +/** + * Return a new "state" DStream where the state for each key is updated by applying + * the given function on the previous state of the key and the new values of each key. + * In every batch the updateFunc will be called for each state even if there are no new values. + * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. + * @param updateFunc State update function. If `this` function returns None, then + * corresponding state key-value pair will be eliminated. + * @tparam S State type + */ +@JvmName("updateStateByKeyPair") +fun JavaDStreamLike, *, *>.updateStateByKey( + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), + updateFunc: (List, S?) -> S?, +): JavaDStream> = + mapToPair { it.toTuple() } + .updateStateByKey( + { list: List, s: Optional -> + updateFunc(list, s.toNullable()).toOptional() + }, + numPartitions, + ) + .map { it.toPair() } + +/** + * Return a new "state" DStream where the state for each key is updated by applying + * the given function on the previous state of the key and the new values of each key. + * In every batch the updateFunc will be called for each state even if there are no new values. + * [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD. + * @param updateFunc State update function. Note, that this function may generate a different + * tuple with a different key than the input key. Therefore keys may be removed + * or added in this way. It is up to the developer to decide whether to + * remember the partitioner despite the key being changed. + * @param partitioner Partitioner for controlling the partitioning of each RDD in the new + * DStream + * @tparam S State type + */ +@JvmName("updateStateByKeyPair") +fun JavaDStreamLike, *, *>.updateStateByKey( + partitioner: Partitioner, + updateFunc: (List, S?) -> S?, +): JavaDStream> = + mapToPair { it.toTuple() } + .updateStateByKey( + { list: List, s: Optional -> + updateFunc(list, s.toNullable()).toOptional() + }, + partitioner, + ) + .map { it.toPair() } + +/** + * Return a new "state" DStream where the state for each key is updated by applying + * the given function on the previous state of the key and the new values of the key. + * org.apache.spark.Partitioner is used to control the partitioning of each RDD. + * @param updateFunc State update function. If `this` function returns None, then + * corresponding state key-value pair will be eliminated. + * @param partitioner Partitioner for controlling the partitioning of each RDD in the new + * DStream. + * @param initialRDD initial state value of each key. + * @tparam S State type + */ +@JvmName("updateStateByKeyPair") +fun JavaDStreamLike, *, *>.updateStateByKey( + partitioner: Partitioner, + initialRDD: JavaRDD>, + updateFunc: (List, S?) -> S?, +): JavaDStream> = + mapToPair { it.toTuple() } + .updateStateByKey( + { list: List, s: Optional -> + updateFunc(list, s.toNullable()).toOptional() + }, + partitioner, + initialRDD.mapToPair { it.toTuple() }, + ) + .map { it.toPair() } + +/** + * Return a new DStream by applying a map function to the value of each key-value pairs in + * 'this' DStream without changing the key. + */ +@JvmName("mapValuesPair") +fun JavaDStreamLike, *, *>.mapValues( + mapValuesFunc: (V) -> U, +): JavaDStream> = + mapToPair { it.toTuple() } + .mapValues(mapValuesFunc) + .map { it.toPair() } + +/** + * Return a new DStream by applying a flatmap function to the value of each key-value pairs in + * 'this' DStream without changing the key. + */ +@JvmName("flatMapValuesPair") +fun JavaDStreamLike, *, *>.flatMapValues( + flatMapValuesFunc: (V) -> Iterator, +): JavaDStream> = + mapToPair { it.toTuple() } + .flatMapValues(flatMapValuesFunc) + .map { it.toPair() } + +/** + * Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream. + * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. + */ +@JvmName("cogroupPair") +fun JavaDStreamLike, *, *>.cogroup( + other: JavaDStreamLike, *, *>, + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +): JavaDStream, Iterable>>> = + mapToPair { it.toTuple() } + .cogroup( + other.mapToPair { it.toTuple() }, + numPartitions, + ) + .map { + Pair(it._1, it._2.toPair()) + } + +/** + * Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream. + * The supplied org.apache.spark.Partitioner is used to partition the generated RDDs. + */ +@JvmName("cogroupPair") +fun JavaDStreamLike, *, *>.cogroup( + other: JavaDStreamLike, *, *>, + partitioner: Partitioner, +): JavaDStream, Iterable>>> = + mapToPair { it.toTuple() } + .cogroup( + other.mapToPair { it.toTuple() }, + partitioner, + ) + .map { + Pair(it._1, it._2.toPair()) + } + +/** + * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream. + * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. + */ +@JvmName("joinPair") +fun JavaDStreamLike, *, *>.join( + other: JavaDStreamLike, *, *>, + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +): JavaDStream>> = + mapToPair { it.toTuple() } + .join( + other.mapToPair { it.toTuple() }, + numPartitions, + ) + .map { + Pair(it._1, it._2.toPair()) + } + +/** + * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream. + * The supplied org.apache.spark.Partitioner is used to control the partitioning of each RDD. + */ +@JvmName("joinPair") +fun JavaDStreamLike, *, *>.join( + other: JavaDStreamLike, *, *>, + partitioner: Partitioner, +): JavaDStream>> = + mapToPair { it.toTuple() } + .join( + other.mapToPair { it.toTuple() }, + partitioner, + ) + .map { + Pair(it._1, it._2.toPair()) + } + +/** + * Return a new DStream by applying 'left outer join' between RDDs of `this` DStream and + * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` + * partitions. + */ +@JvmName("leftOuterJoinPair") +fun JavaDStreamLike, *, *>.leftOuterJoin( + other: JavaDStreamLike, *, *>, + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +): JavaDStream>> = + mapToPair { it.toTuple() } + .leftOuterJoin( + other.mapToPair { it.toTuple() }, + numPartitions, + ) + .map { + Pair(it._1, Pair(it._2._1, it._2._2.toNullable())) + } + +/** + * Return a new DStream by applying 'left outer join' between RDDs of `this` DStream and + * `other` DStream. The supplied org.apache.spark.Partitioner is used to control + * the partitioning of each RDD. + */ +@JvmName("leftOuterJoinPair") +fun JavaDStreamLike, *, *>.leftOuterJoin( + other: JavaDStreamLike, *, *>, + partitioner: Partitioner, +): JavaDStream>> = + mapToPair { it.toTuple() } + .leftOuterJoin( + other.mapToPair { it.toTuple() }, + partitioner, + ) + .map { + Pair(it._1, Pair(it._2._1, it._2._2.toNullable())) + } + +/** + * Return a new DStream by applying 'right outer join' between RDDs of `this` DStream and + * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` + * partitions. + */ +@JvmName("rightOuterJoinPair") +fun JavaDStreamLike, *, *>.rightOuterJoin( + other: JavaDStreamLike, *, *>, + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +): JavaDStream>> = + mapToPair { it.toTuple() } + .rightOuterJoin( + other.mapToPair { it.toTuple() }, + numPartitions, + ) + .map { + Pair(it._1, Pair(it._2._1.toNullable(), it._2._2)) + } + +/** + * Return a new DStream by applying 'right outer join' between RDDs of `this` DStream and + * `other` DStream. The supplied org.apache.spark.Partitioner is used to control + * the partitioning of each RDD. + */ +@JvmName("rightOuterJoinPair") +fun JavaDStreamLike, *, *>.rightOuterJoin( + other: JavaDStreamLike, *, *>, + partitioner: Partitioner, +): JavaDStream>> = + mapToPair { it.toTuple() } + .rightOuterJoin( + other.mapToPair { it.toTuple() }, + partitioner, + ) + .map { + Pair(it._1, Pair(it._2._1.toNullable(), it._2._2)) + } + +/** + * Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and + * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` + * partitions. + */ +@JvmName("fullOuterJoinPair") +fun JavaDStreamLike, *, *>.fullOuterJoin( + other: JavaDStreamLike, *, *>, + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +): JavaDStream>> = + mapToPair { it.toTuple() } + .fullOuterJoin( + other.mapToPair { it.toTuple() }, + numPartitions, + ) + .map { + Pair(it._1, Pair(it._2._1.toNullable(), it._2._2.toNullable())) + } + +/** + * Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and + * `other` DStream. The supplied org.apache.spark.Partitioner is used to control + * the partitioning of each RDD. + */ +@JvmName("fullOuterJoinPair") +fun JavaDStreamLike, *, *>.fullOuterJoin( + other: JavaDStreamLike, *, *>, + partitioner: Partitioner, +): JavaDStream>> = + mapToPair { it.toTuple() } + .fullOuterJoin( + other.mapToPair { it.toTuple() }, + partitioner, + ) + .map { + Pair(it._1, Pair(it._2._1.toNullable(), it._2._2.toNullable())) + } + +/** + * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is + * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix". + */ +@JvmName("saveAsHadoopFilesPair") +fun JavaDStreamLike, *, *>.saveAsHadoopFiles( + prefix: String, suffix: String, +): Unit = + mapToPair { it.toTuple() } + .saveAsHadoopFiles(prefix, suffix) + +/** + * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is + * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix". + */ +@JvmName("saveAsNewAPIHadoopFilesPair") +fun JavaDStreamLike, *, *>.saveAsNewAPIHadoopFiles( + prefix: String, suffix: String, +): Unit = + mapToPair { it.toTuple() } + .saveAsNewAPIHadoopFiles(prefix, suffix) + From eec03cbb80b80b5e2a4b2ead8c64c221f54d9134 Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Thu, 17 Mar 2022 20:11:12 +0100 Subject: [PATCH 14/44] and tuples cause why not --- .../JavaRecoverableNetworkWordCount.kt | 4 + .../jetbrains/kotlinx/spark/api/Streaming.kt | 591 ++++++++++++++++++ 2 files changed, 595 insertions(+) diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/JavaRecoverableNetworkWordCount.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/JavaRecoverableNetworkWordCount.kt index f89e09f3..4f06dd75 100644 --- a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/JavaRecoverableNetworkWordCount.kt +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/JavaRecoverableNetworkWordCount.kt @@ -170,6 +170,10 @@ object JavaRecoverableNetworkWordCount { .map { it to 1 } .reduceByKey { a: Int, b: Int -> a + b } + val wordCounts3 = words + .map { Tuple2(it, 1) } + .reduceByKey { a: Int, b: Int -> a + b } + wordCounts.foreachRDD { rdd, time: Time -> // Get or register the excludeList Broadcast diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Streaming.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Streaming.kt index f0152f83..bdad23e1 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Streaming.kt +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Streaming.kt @@ -634,6 +634,10 @@ fun JavaDStreamLike, *, *>.saveAsNewAPIHadoopFiles( mapToPair { it.toTuple() } .saveAsNewAPIHadoopFiles(prefix, suffix) + + + + /** * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to * generate the RDDs with `numPartitions` partitions. @@ -1218,3 +1222,590 @@ fun JavaDStreamLike, *, *>.saveAsNewAPIHadoopFiles( mapToPair { it.toTuple() } .saveAsNewAPIHadoopFiles(prefix, suffix) + + + + +/** + * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to + * generate the RDDs with `numPartitions` partitions. + */ +@JvmName("groupByKeyTuple2") +fun JavaDStreamLike, *, *>.groupByKey( + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +): JavaDStream>> = + mapToPair { it } + .groupByKey(numPartitions) + .map { it } + +/** + * Return a new DStream by applying `groupByKey` on each RDD. The supplied + * org.apache.spark.Partitioner is used to control the partitioning of each RDD. + */ +@JvmName("groupByKeyTuple2") +fun JavaDStreamLike, *, *>.groupByKey(partitioner: Partitioner): JavaDStream>> = + mapToPair { it } + .groupByKey(partitioner) + .map { it } + +/** + * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are + * merged using the supplied reduce function. Hash partitioning is used to generate the RDDs + * with `numPartitions` partitions. + */ +@JvmName("reduceByKeyTuple2") +fun JavaDStreamLike, *, *>.reduceByKey( + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), + reduceFunc: (V, V) -> V, +): JavaDStream> = + mapToPair { it } + .reduceByKey(reduceFunc, numPartitions) + .map { it } + +/** + * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are + * merged using the supplied reduce function. org.apache.spark.Partitioner is used to control + * the partitioning of each RDD. + */ +@JvmName("reduceByKeyTuple2") +fun JavaDStreamLike, *, *>.reduceByKey( + partitioner: Partitioner, + reduceFunc: (V, V) -> V, +): JavaDStream> = + mapToPair { it } + .reduceByKey(reduceFunc, partitioner) + .map { it } + +/** + * Combine elements of each key in DStream's RDDs using custom functions. This is similar to the + * combineByKey for RDDs. Please refer to combineByKey in + * org.apache.spark.rdd.PairRDDFunctions in the Spark core documentation for more information. + */ +@JvmName("combineByKeyTuple2") +fun JavaDStreamLike, *, *>.combineByKey( + createCombiner: (V) -> C, + mergeValue: (C, V) -> C, + mergeCombiner: (C, C) -> C, + partitioner: Partitioner, + mapSideCombine: Boolean = true, +): JavaDStream> = + mapToPair { it } + .combineByKey(createCombiner, mergeValue, mergeCombiner, partitioner, mapSideCombine) + .map { it } + +/** + * Return a new DStream by applying `groupByKey` over a sliding window on `this` DStream. + * Similar to `DStream.groupByKey()`, but applies it over a sliding window. + * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. + * @param windowDuration width of the window; must be a multiple of this DStream's + * batching interval + * @param slideDuration sliding interval of the window (i.e., the interval after which + * the new DStream will generate RDDs); must be a multiple of this + * DStream's batching interval + * @param numPartitions number of partitions of each RDD in the new DStream; if not specified + * then Spark's default number of partitions will be used + */ +@JvmName("groupByKeyAndWindowTuple2") +fun JavaDStreamLike, *, *>.groupByKeyAndWindow( + windowDuration: Duration, + slideDuration: Duration = dstream().slideDuration(), + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +): JavaDStream>> = + mapToPair { it } + .groupByKeyAndWindow(windowDuration, slideDuration, numPartitions) + .map { it } + +/** + * Create a new DStream by applying `groupByKey` over a sliding window on `this` DStream. + * Similar to `DStream.groupByKey()`, but applies it over a sliding window. + * @param windowDuration width of the window; must be a multiple of this DStream's + * batching interval + * @param slideDuration sliding interval of the window (i.e., the interval after which + * the new DStream will generate RDDs); must be a multiple of this + * DStream's batching interval + * @param partitioner partitioner for controlling the partitioning of each RDD in the new + * DStream. + */ +@JvmName("groupByKeyAndWindowTuple2") +fun JavaDStreamLike, *, *>.groupByKeyAndWindow( + windowDuration: Duration, + slideDuration: Duration = dstream().slideDuration(), + partitioner: Partitioner, +): JavaDStream>> = + mapToPair { it } + .groupByKeyAndWindow(windowDuration, slideDuration, partitioner) + .map { it } + +/** + * Return a new DStream by applying `reduceByKey` over a sliding window. This is similar to + * `DStream.reduceByKey()` but applies it over a sliding window. Hash partitioning is used to + * generate the RDDs with `numPartitions` partitions. + * @param reduceFunc associative and commutative reduce function + * @param windowDuration width of the window; must be a multiple of this DStream's + * batching interval + * @param slideDuration sliding interval of the window (i.e., the interval after which + * the new DStream will generate RDDs); must be a multiple of this + * DStream's batching interval + * @param numPartitions number of partitions of each RDD in the new DStream. + */ +@JvmName("reduceByKeyAndWindowTuple2") +fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( + windowDuration: Duration, + slideDuration: Duration = dstream().slideDuration(), + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), + reduceFunc: (V, V) -> V, +): JavaDStream> = + mapToPair { it } + .reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, numPartitions) + .map { it } + +/** + * Return a new DStream by applying `reduceByKey` over a sliding window. Similar to + * `DStream.reduceByKey()`, but applies it over a sliding window. + * @param reduceFunc associative and commutative reduce function + * @param windowDuration width of the window; must be a multiple of this DStream's + * batching interval + * @param slideDuration sliding interval of the window (i.e., the interval after which + * the new DStream will generate RDDs); must be a multiple of this + * DStream's batching interval + * @param partitioner partitioner for controlling the partitioning of each RDD + * in the new DStream. + */ +@JvmName("reduceByKeyAndWindowTuple2") +fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( + windowDuration: Duration, + slideDuration: Duration = dstream().slideDuration(), + partitioner: Partitioner, + reduceFunc: (V, V) -> V, +): JavaDStream> = + mapToPair { it } + .reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, partitioner) + .map { it } + +/** + * Return a new DStream by applying incremental `reduceByKey` over a sliding window. + * The reduced value of over a new window is calculated using the old window's reduced value : + * 1. reduce the new values that entered the window (e.g., adding new counts) + * + * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) + * + * This is more efficient than reduceByKeyAndWindow without "inverse reduce" function. + * However, it is applicable to only "invertible reduce functions". + * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. + * @param reduceFunc associative and commutative reduce function + * @param invReduceFunc inverse reduce function; such that for all y, invertible x: + * `invReduceFunc(reduceFunc(x, y), x) = y` + * @param windowDuration width of the window; must be a multiple of this DStream's + * batching interval + * @param slideDuration sliding interval of the window (i.e., the interval after which + * the new DStream will generate RDDs); must be a multiple of this + * DStream's batching interval + * @param filterFunc Optional function to filter expired key-value pairs; + * only pairs that satisfy the function are retained + */ +@JvmName("reduceByKeyAndWindowTuple2") +fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( + invReduceFunc: (V, V) -> V, + windowDuration: Duration, + slideDuration: Duration = dstream().slideDuration(), + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), + filterFunc: ((Tuple2) -> Boolean)? = null, + reduceFunc: (V, V) -> V, +): JavaDStream> = + mapToPair { it } + .reduceByKeyAndWindow( + reduceFunc, + invReduceFunc, + windowDuration, + slideDuration, + numPartitions, + filterFunc?.let { + { tuple -> + filterFunc(tuple) + } + } + ) + .map { it } + +/** + * Return a new DStream by applying incremental `reduceByKey` over a sliding window. + * The reduced value of over a new window is calculated using the old window's reduced value : + * 1. reduce the new values that entered the window (e.g., adding new counts) + * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) + * This is more efficient than reduceByKeyAndWindow without "inverse reduce" function. + * However, it is applicable to only "invertible reduce functions". + * @param reduceFunc associative and commutative reduce function + * @param invReduceFunc inverse reduce function + * @param windowDuration width of the window; must be a multiple of this DStream's + * batching interval + * @param slideDuration sliding interval of the window (i.e., the interval after which + * the new DStream will generate RDDs); must be a multiple of this + * DStream's batching interval + * @param partitioner partitioner for controlling the partitioning of each RDD in the new + * DStream. + * @param filterFunc Optional function to filter expired key-value pairs; + * only pairs that satisfy the function are retained + */ +@JvmName("reduceByKeyAndWindowTuple2") +fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( + invReduceFunc: (V, V) -> V, + windowDuration: Duration, + slideDuration: Duration = dstream().slideDuration(), + partitioner: Partitioner, + filterFunc: ((Tuple2) -> Boolean)? = null, + reduceFunc: (V, V) -> V, +): JavaDStream> = + mapToPair { it } + .reduceByKeyAndWindow( + reduceFunc, + invReduceFunc, + windowDuration, + slideDuration, + partitioner, + filterFunc?.let { + { tuple -> + filterFunc(tuple) + } + } + ) + .map { it } + +/** + * Return a [MapWithStateDStream] by applying a function to every key-value element of + * `this` stream, while maintaining some state data for each unique key. The mapping function + * and other specification (e.g. partitioners, timeouts, initial state data, etc.) of this + * transformation can be specified using `StateSpec` class. The state data is accessible in + * as a parameter of type `State` in the mapping function. + * + * Example of using `mapWithState`: + * {{{ + * // A mapping function that maintains an integer state and return a String + * def mappingFunction(key: String, value: Option[Int], state: State[Int]): Option[String] = { + * // Use state.exists(), state.get(), state.update() and state.remove() + * // to manage state, and return the necessary string + * } + * + * val spec = StateSpec.function(mappingFunction).numPartitions(10) + * + * val mapWithStateDStream = keyValueDStream.mapWithState[StateType, MappedType](spec) + * }}} + * + * @param spec Specification of this transformation + * @tparam StateType Class type of the state data + * @tparam MappedType Class type of the mapped data + */ +@JvmName("mapWithStateTuple2") +fun JavaDStreamLike, *, *>.mapWithState( + spec: StateSpec, +): JavaMapWithStateDStream = + mapToPair { it } + .mapWithState(spec) + +/** + * Return a new "state" DStream where the state for each key is updated by applying + * the given function on the previous state of the key and the new values of each key. + * In every batch the updateFunc will be called for each state even if there are no new values. + * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. + * @param updateFunc State update function. If `this` function returns None, then + * corresponding state key-value pair will be eliminated. + * @tparam S State type + */ +@JvmName("updateStateByKeyTuple2") +fun JavaDStreamLike, *, *>.updateStateByKey( + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), + updateFunc: (List, S?) -> S?, +): JavaDStream> = + mapToPair { it } + .updateStateByKey( + { list: List, s: Optional -> + updateFunc(list, s.toNullable()).toOptional() + }, + numPartitions, + ) + .map { it } + +/** + * Return a new "state" DStream where the state for each key is updated by applying + * the given function on the previous state of the key and the new values of each key. + * In every batch the updateFunc will be called for each state even if there are no new values. + * [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD. + * @param updateFunc State update function. Note, that this function may generate a different + * tuple with a different key than the input key. Therefore keys may be removed + * or added in this way. It is up to the developer to decide whether to + * remember the partitioner despite the key being changed. + * @param partitioner Partitioner for controlling the partitioning of each RDD in the new + * DStream + * @tparam S State type + */ +@JvmName("updateStateByKeyTuple2") +fun JavaDStreamLike, *, *>.updateStateByKey( + partitioner: Partitioner, + updateFunc: (List, S?) -> S?, +): JavaDStream> = + mapToPair { it } + .updateStateByKey( + { list: List, s: Optional -> + updateFunc(list, s.toNullable()).toOptional() + }, + partitioner, + ) + .map { it } + +/** + * Return a new "state" DStream where the state for each key is updated by applying + * the given function on the previous state of the key and the new values of the key. + * org.apache.spark.Partitioner is used to control the partitioning of each RDD. + * @param updateFunc State update function. If `this` function returns None, then + * corresponding state key-value pair will be eliminated. + * @param partitioner Partitioner for controlling the partitioning of each RDD in the new + * DStream. + * @param initialRDD initial state value of each key. + * @tparam S State type + */ +@JvmName("updateStateByKeyTuple2") +fun JavaDStreamLike, *, *>.updateStateByKey( + partitioner: Partitioner, + initialRDD: JavaRDD>, + updateFunc: (List, S?) -> S?, +): JavaDStream> = + mapToPair { it } + .updateStateByKey( + { list: List, s: Optional -> + updateFunc(list, s.toNullable()).toOptional() + }, + partitioner, + initialRDD.mapToPair { it }, + ) + .map { it } + +/** + * Return a new DStream by applying a map function to the value of each key-value pairs in + * 'this' DStream without changing the key. + */ +@JvmName("mapValuesTuple2") +fun JavaDStreamLike, *, *>.mapValues( + mapValuesFunc: (V) -> U, +): JavaDStream> = + mapToPair { it } + .mapValues(mapValuesFunc) + .map { it } + +/** + * Return a new DStream by applying a flatmap function to the value of each key-value pairs in + * 'this' DStream without changing the key. + */ +@JvmName("flatMapValuesTuple2") +fun JavaDStreamLike, *, *>.flatMapValues( + flatMapValuesFunc: (V) -> Iterator, +): JavaDStream> = + mapToPair { it } + .flatMapValues(flatMapValuesFunc) + .map { it } + +/** + * Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream. + * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. + */ +@JvmName("cogroupTuple2") +fun JavaDStreamLike, *, *>.cogroup( + other: JavaDStreamLike, *, *>, + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +): JavaDStream, Iterable>>> = + mapToPair { it } + .cogroup( + other.mapToPair { it }, + numPartitions, + ) + .map { + Tuple2(it._1, it._2) + } + +/** + * Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream. + * The supplied org.apache.spark.Partitioner is used to partition the generated RDDs. + */ +@JvmName("cogroupTuple2") +fun JavaDStreamLike, *, *>.cogroup( + other: JavaDStreamLike, *, *>, + partitioner: Partitioner, +): JavaDStream, Iterable>>> = + mapToPair { it } + .cogroup( + other.mapToPair { it }, + partitioner, + ) + .map { + Tuple2(it._1, it._2) + } + +/** + * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream. + * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. + */ +@JvmName("joinTuple2") +fun JavaDStreamLike, *, *>.join( + other: JavaDStreamLike, *, *>, + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +): JavaDStream>> = + mapToPair { it } + .join( + other.mapToPair { it }, + numPartitions, + ) + .map { + Tuple2(it._1, it._2) + } + +/** + * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream. + * The supplied org.apache.spark.Partitioner is used to control the partitioning of each RDD. + */ +@JvmName("joinTuple2") +fun JavaDStreamLike, *, *>.join( + other: JavaDStreamLike, *, *>, + partitioner: Partitioner, +): JavaDStream>> = + mapToPair { it } + .join( + other.mapToPair { it }, + partitioner, + ) + .map { + Tuple2(it._1, it._2) + } + +/** + * Return a new DStream by applying 'left outer join' between RDDs of `this` DStream and + * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` + * partitions. + */ +@JvmName("leftOuterJoinTuple2") +fun JavaDStreamLike, *, *>.leftOuterJoin( + other: JavaDStreamLike, *, *>, + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +): JavaDStream>> = + mapToPair { it } + .leftOuterJoin( + other.mapToPair { it }, + numPartitions, + ) + .map { + Tuple2(it._1, Tuple2(it._2._1, it._2._2.toNullable())) + } + +/** + * Return a new DStream by applying 'left outer join' between RDDs of `this` DStream and + * `other` DStream. The supplied org.apache.spark.Partitioner is used to control + * the partitioning of each RDD. + */ +@JvmName("leftOuterJoinTuple2") +fun JavaDStreamLike, *, *>.leftOuterJoin( + other: JavaDStreamLike, *, *>, + partitioner: Partitioner, +): JavaDStream>> = + mapToPair { it } + .leftOuterJoin( + other.mapToPair { it }, + partitioner, + ) + .map { + Tuple2(it._1, Tuple2(it._2._1, it._2._2.toNullable())) + } + +/** + * Return a new DStream by applying 'right outer join' between RDDs of `this` DStream and + * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` + * partitions. + */ +@JvmName("rightOuterJoinTuple2") +fun JavaDStreamLike, *, *>.rightOuterJoin( + other: JavaDStreamLike, *, *>, + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +): JavaDStream>> = + mapToPair { it } + .rightOuterJoin( + other.mapToPair { it }, + numPartitions, + ) + .map { + Tuple2(it._1, Tuple2(it._2._1.toNullable(), it._2._2)) + } + +/** + * Return a new DStream by applying 'right outer join' between RDDs of `this` DStream and + * `other` DStream. The supplied org.apache.spark.Partitioner is used to control + * the partitioning of each RDD. + */ +@JvmName("rightOuterJoinTuple2") +fun JavaDStreamLike, *, *>.rightOuterJoin( + other: JavaDStreamLike, *, *>, + partitioner: Partitioner, +): JavaDStream>> = + mapToPair { it } + .rightOuterJoin( + other.mapToPair { it }, + partitioner, + ) + .map { + Tuple2(it._1, Tuple2(it._2._1.toNullable(), it._2._2)) + } + +/** + * Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and + * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` + * partitions. + */ +@JvmName("fullOuterJoinTuple2") +fun JavaDStreamLike, *, *>.fullOuterJoin( + other: JavaDStreamLike, *, *>, + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +): JavaDStream>> = + mapToPair { it } + .fullOuterJoin( + other.mapToPair { it }, + numPartitions, + ) + .map { + Tuple2(it._1, Tuple2(it._2._1.toNullable(), it._2._2.toNullable())) + } + +/** + * Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and + * `other` DStream. The supplied org.apache.spark.Partitioner is used to control + * the partitioning of each RDD. + */ +@JvmName("fullOuterJoinTuple2") +fun JavaDStreamLike, *, *>.fullOuterJoin( + other: JavaDStreamLike, *, *>, + partitioner: Partitioner, +): JavaDStream>> = + mapToPair { it } + .fullOuterJoin( + other.mapToPair { it }, + partitioner, + ) + .map { + Tuple2(it._1, Tuple2(it._2._1.toNullable(), it._2._2.toNullable())) + } + +/** + * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is + * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix". + */ +@JvmName("saveAsHadoopFilesTuple2") +fun JavaDStreamLike, *, *>.saveAsHadoopFiles( + prefix: String, suffix: String, +): Unit = + mapToPair { it } + .saveAsHadoopFiles(prefix, suffix) + +/** + * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is + * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix". + */ +@JvmName("saveAsNewAPIHadoopFilesTuple2") +fun JavaDStreamLike, *, *>.saveAsNewAPIHadoopFiles( + prefix: String, suffix: String, +): Unit = + mapToPair { it } + .saveAsNewAPIHadoopFiles(prefix, suffix) From bc9fd3ea62b87105fd4a8fa96760d6e1632848ad Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Mon, 21 Mar 2022 18:25:11 +0100 Subject: [PATCH 15/44] added Option(al) converters, testing fakeClassTag(). Working with streaming pair-functions, decided to go tuple-first instead of arities --- .../GeneratePairStreamingFunctions.kt | 87 - .../JavaRecoverableNetworkWordCount.kt | 91 +- .../kotlinx/spark/api/Conversions.kt | 608 +++- .../jetbrains/kotlinx/spark/api/Streaming.kt | 2658 ++++++++--------- 4 files changed, 1952 insertions(+), 1492 deletions(-) delete mode 100644 examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/GeneratePairStreamingFunctions.kt diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/GeneratePairStreamingFunctions.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/GeneratePairStreamingFunctions.kt deleted file mode 100644 index 740774e2..00000000 --- a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/GeneratePairStreamingFunctions.kt +++ /dev/null @@ -1,87 +0,0 @@ -/*- - * =LICENSE= - * Kotlin Spark API: API for Spark 3.2+ (Scala 2.12) - * ---------- - * Copyright (C) 2019 - 2022 JetBrains - * ---------- - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - * =LICENSEEND= - */ -package org.jetbrains.kotlinx.spark.examples - -import org.apache.spark.streaming.dstream.PairDStreamFunctions -import org.intellij.lang.annotations.Language -import kotlin.reflect.KFunction -import kotlin.reflect.full.functions - - -object GeneratePairStreamingFunctions { - -// fun JavaDStream>.reduceByKey(func: (V, V) -> V): JavaDStream> = -// mapToPair { it.toTuple() } -// .reduceByKey(func) -// .map { it.toArity() } - - @JvmStatic - fun main(args: Array) { - - val klass = PairDStreamFunctions::class - - val functions = klass.functions - - for (function: KFunction<*> in functions) with(function) { - - val types = (typeParameters.map { it.name }.toSet() + "K" + "V").joinToString() - - val parameterString = parameters.drop(1).joinToString { - "${it.name}: ${it.type}" - } - val parameterStringNoType = parameters.drop(1).joinToString { it.name!! } - - @Language("kt") - val new = """ - fun <$types> JavaDStream>.$name($parameterString) - - """.trimIndent() - -// -// val new = -// if (returnType.toString().contains("org.apache.spark.streaming.api.java.JavaPairDStream")) { -// val newReturnType = returnType.toString() -// .replaceFirst("JavaPairDStream<", "JavaDStream", ">") -// -// """ -// fun <$types> JavaDStream>.$name($parameterString): $newReturnType = -// mapToPair { it.toTuple() } -// .$name($parameterStringNoType) -// .map { it.toArity() } -// -// """.trimIndent() -// } else { -// """ -// fun <$types> JavaDStream>.$name($parameterString): $returnType = -// mapToPair { it.toTuple() } -// .$name($parameterStringNoType) -// -// """.trimIndent() -// } -// .replace("!", "") -// .replace("(Mutable)", "") -// -// if ("\$" !in new) println(new) - } - - - } -} diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/JavaRecoverableNetworkWordCount.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/JavaRecoverableNetworkWordCount.kt index 4f06dd75..9b64149e 100644 --- a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/JavaRecoverableNetworkWordCount.kt +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/JavaRecoverableNetworkWordCount.kt @@ -17,21 +17,33 @@ * limitations under the License. * =LICENSEEND= */ +@file:OptIn(ExperimentalTime::class) + package org.jetbrains.kotlinx.spark.examples import com.google.common.io.Files import org.apache.spark.api.java.JavaPairRDD import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.broadcast.Broadcast +import org.apache.spark.streaming.Duration import org.apache.spark.streaming.Durations import org.apache.spark.streaming.Time +import org.apache.spark.streaming.api.java.JavaDStream +import org.apache.spark.streaming.api.java.JavaPairDStream +import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.dstream.PairDStreamFunctions import org.apache.spark.util.LongAccumulator import org.jetbrains.kotlinx.spark.api.* +import scala.Tuple1 import scala.Tuple2 +import scala.reflect.ClassTag import java.io.File import java.nio.charset.Charset import java.util.regex.Pattern +import kotlin.experimental.ExperimentalTypeInference import kotlin.system.exitProcess +import kotlin.time.ExperimentalTime +import kotlin.time.measureTimedValue /** @@ -144,6 +156,8 @@ object JavaRecoverableNetworkWordCount { } } + + @OptIn(ExperimentalTypeInference::class) @Suppress("UnstableApiUsage") private fun KSparkStreamingSession.createContext( ip: String, @@ -162,36 +176,75 @@ object JavaRecoverableNetworkWordCount { val words = lines.flatMap { it.split(SPACE).iterator() } - val wordCounts = words - .map { c(it, 1) } - .reduceByKey { a: Int, b: Int -> a + b } +// val wordCounts = words +// .map { c(it, 1) } +// .reduceByKey { a, b -> a + b } +// .reduceByKey { a, b -> a + b } +// .reduceByKey { a, b -> a + b } +// .reduceByKey { a, b -> a + b } +// .reduceByKey { a, b -> a + b } +// .reduceByKey { a, b -> a + b } +// .reduceByKey { a, b -> a + b } +// .reduceByKey { a, b -> a + b } + + val wordCounts4 = words + .mapToPair { Tuple2(it, 1) } + .reduceByKey { a, b -> a + b } + .reduceByKey { a, b -> a + b } + .reduceByKey { a, b -> a + b } + .reduceByKey { a, b -> a + b } + .reduceByKey { a, b -> a + b } + .reduceByKey { a, b -> a + b } + .reduceByKey { a, b -> a + b } + .reduceByKey { a, b -> a + b } + - val wordCounts2 = words - .map { it to 1 } - .reduceByKey { a: Int, b: Int -> a + b } +// val wordCounts2 = words +// .map { it to 1 } +// .reduceByKey { a, b -> a + b } val wordCounts3 = words .map { Tuple2(it, 1) } - .reduceByKey { a: Int, b: Int -> a + b } + .reduceByKey { a, b -> a + b } + .reduceByKey { a, b -> a + b } + .reduceByKey { a, b -> a + b } + .reduceByKey { a, b -> a + b } + .reduceByKey { a, b -> a + b } + .reduceByKey { a, b -> a + b } + .reduceByKey { a, b -> a + b } + .reduceByKey { a, b -> a + b } - wordCounts.foreachRDD { rdd, time: Time -> +// val wordCounts5 = words +// .dstream() +// .map({ Tuple2(it, 1) }, fakeClassTag()) +// .let { DStream.toPairDStreamFunctions(it, fakeClassTag(), fakeClassTag(), null) } +// .reduceByKey { a, b -> a + b } +// .let { JavaDStream(it, fakeClassTag()) } + + wordCounts3.foreachRDD { rdd, time: Time -> + val sc = JavaSparkContext(rdd.context()) // Get or register the excludeList Broadcast - val excludeList = JavaWordExcludeList.getInstance(JavaSparkContext(rdd.context())) + val excludeList = JavaWordExcludeList.getInstance(sc) // Get or register the droppedWordsCounter Accumulator - val droppedWordsCounter = JavaDroppedWordsCounter.getInstance(JavaSparkContext(rdd.context())) + val droppedWordsCounter = JavaDroppedWordsCounter.getInstance(sc) // Use excludeList to drop words and use droppedWordsCounter to count them - val counts = rdd.filter { wordCount -> - if (excludeList.value().contains(wordCount._1)) { - droppedWordsCounter.add(wordCount._2.toLong()) - false - } else { - true - } - }.collect().toString() - val output = "Counts at time $time $counts" + val (counts, duration) = measureTimedValue { + rdd.filter { wordCount -> + if (excludeList.value().contains(wordCount._1)) { + droppedWordsCounter.add(wordCount._2.toLong()) + false + } else { + true + } + }.collect() + } + + println("Debug: ${rdd.toDebugString()}") + + val output = "Counts at time $time $counts\n$duration" println(output) println("Dropped ${droppedWordsCounter.value()} word(s) totally") println("Appending to " + outputFile.absolutePath) diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt index 2863ad08..f644dfbf 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt @@ -30,6 +30,7 @@ package org.jetbrains.kotlinx.spark.api import org.apache.spark.api.java.Optional import scala.* import scala.collection.JavaConverters +import scala.reflect.ClassTag import java.util.* import java.util.Enumeration import java.util.concurrent.ConcurrentMap @@ -46,16 +47,30 @@ import scala.collection.mutable.Set as ScalaMutableSet /** Converts Scala [Option] to Kotlin nullable. */ -fun Option.toNullable(): T? = getOrElse { null } +fun Option.getOrNull(): T? = getOrElse(null) /** Converts nullable value to Scala [Option]. */ -fun T?.toOption(): Option = Option.apply(this) +fun T?.asOption(): Option = Option.apply(this) /** Converts [Optional] to Kotlin nullable. */ -fun Optional.toNullable(): T? = orElse(null) +fun Optional.getOrNull(): T? = orNull() /** Converts nullable value to [Optional]. */ -fun T?.toOptional(): Optional = Optional.ofNullable(this) +fun T?.asOptional(): Optional = Optional.ofNullable(this) + +/** + * TODO test + * 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. + */ +fun fakeClassTag(): ClassTag = ClassTag.AnyRef() as ClassTag /** * @see JavaConverters.asScalaIterator for more information. @@ -241,97 +256,356 @@ 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. **/ -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. **/ -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. **/ -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. **/ -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. **/ -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. **/ -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. **/ -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. **/ -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. **/ -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. **/ -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. **/ -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. **/ -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. **/ -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. **/ -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. **/ -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. **/ -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. **/ -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. **/ -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. @@ -351,94 +625,344 @@ 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. **/ -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. **/ -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. **/ -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. **/ -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. **/ -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. **/ -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. **/ -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. **/ -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. **/ -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. **/ -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. **/ -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. **/ -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. **/ -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. **/ -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. **/ -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. **/ -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. **/ -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. **/ -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/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Streaming.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Streaming.kt index bdad23e1..74aaa520 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Streaming.kt +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Streaming.kt @@ -19,8 +19,8 @@ */ package org.jetbrains.kotlinx.spark.api -import com.sun.org.apache.xml.internal.serialize.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.Optional import org.apache.spark.streaming.Duration @@ -29,1201 +29,1192 @@ import org.apache.spark.streaming.api.java.JavaDStream import org.apache.spark.streaming.api.java.JavaDStreamLike import org.apache.spark.streaming.api.java.JavaMapWithStateDStream import org.apache.spark.streaming.api.java.JavaPairDStream +import org.apache.spark.streaming.dstream.DStream import scala.Tuple2 -import scala.Tuple3 -//fun JavaDStreamLike, *, *>.reduceByKey(func: (V, V) -> V): JavaDStream> = -// mapToPair(Arity2::toTuple) -// .reduceByKey(func) -// .map(Tuple2::toArity) +@JvmName("tuple2ToPairDStream") +fun JavaDStream>.toPairDStream(): JavaPairDStream = + JavaPairDStream.fromJavaDStream(this) -@JvmName("tuple2ToPairDStream") -fun JavaDStreamLike, *, *>.toPairDStream(): JavaPairDStream = - mapToPair { it } - -@JvmName("arity2ToPairDStream") -fun JavaDStreamLike, *, *>.toPairDStream(): JavaPairDStream = - mapToPair(Arity2::toTuple) - -@JvmName("pairToPairDStream") -fun JavaDStreamLike, *, *>.toPairDStream(): JavaPairDStream = - mapToPair(Pair::toTuple) - -/** - * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to - * generate the RDDs with `numPartitions` partitions. - */ -@JvmName("groupByKeyArity2") -fun JavaDStreamLike, *, *>.groupByKey( - numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -): JavaDStream>> = - mapToPair { it.toTuple() } - .groupByKey(numPartitions) - .map { it.toArity() } - -/** - * Return a new DStream by applying `groupByKey` on each RDD. The supplied - * org.apache.spark.Partitioner is used to control the partitioning of each RDD. - */ -@JvmName("groupByKeyArity2") -fun JavaDStreamLike, *, *>.groupByKey(partitioner: Partitioner): JavaDStream>> = - mapToPair { it.toTuple() } - .groupByKey(partitioner) - .map { it.toArity() } - -/** - * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are - * merged using the supplied reduce function. Hash partitioning is used to generate the RDDs - * with `numPartitions` partitions. - */ -@JvmName("reduceByKeyArity2") -fun JavaDStreamLike, *, *>.reduceByKey( - numPartitions: Int = dstream().ssc().sc().defaultParallelism(), - reduceFunc: (V, V) -> V, -): JavaDStream> = - mapToPair { it.toTuple() } - .reduceByKey(reduceFunc, numPartitions) - .map { it.toArity() } - -/** - * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are - * merged using the supplied reduce function. org.apache.spark.Partitioner is used to control - * the partitioning of each RDD. - */ -@JvmName("reduceByKeyArity2") -fun JavaDStreamLike, *, *>.reduceByKey( - partitioner: Partitioner, - reduceFunc: (V, V) -> V, -): JavaDStream> = - mapToPair { it.toTuple() } - .reduceByKey(reduceFunc, partitioner) - .map { it.toArity() } - -/** - * Combine elements of each key in DStream's RDDs using custom functions. This is similar to the - * combineByKey for RDDs. Please refer to combineByKey in - * org.apache.spark.rdd.PairRDDFunctions in the Spark core documentation for more information. - */ -@JvmName("combineByKeyArity2") -fun JavaDStreamLike, *, *>.combineByKey( - createCombiner: (V) -> C, - mergeValue: (C, V) -> C, - mergeCombiner: (C, C) -> C, - partitioner: Partitioner, - mapSideCombine: Boolean = true, -): JavaDStream> = - mapToPair { it.toTuple() } - .combineByKey(createCombiner, mergeValue, mergeCombiner, partitioner, mapSideCombine) - .map { it.toArity() } - -/** - * Return a new DStream by applying `groupByKey` over a sliding window on `this` DStream. - * Similar to `DStream.groupByKey()`, but applies it over a sliding window. - * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. - * @param windowDuration width of the window; must be a multiple of this DStream's - * batching interval - * @param slideDuration sliding interval of the window (i.e., the interval after which - * the new DStream will generate RDDs); must be a multiple of this - * DStream's batching interval - * @param numPartitions number of partitions of each RDD in the new DStream; if not specified - * then Spark's default number of partitions will be used - */ -@JvmName("groupByKeyAndWindowArity2") -fun JavaDStreamLike, *, *>.groupByKeyAndWindow( - windowDuration: Duration, - slideDuration: Duration = dstream().slideDuration(), - numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -): JavaDStream>> = - mapToPair { it.toTuple() } - .groupByKeyAndWindow(windowDuration, slideDuration, numPartitions) - .map { it.toArity() } - -/** - * Create a new DStream by applying `groupByKey` over a sliding window on `this` DStream. - * Similar to `DStream.groupByKey()`, but applies it over a sliding window. - * @param windowDuration width of the window; must be a multiple of this DStream's - * batching interval - * @param slideDuration sliding interval of the window (i.e., the interval after which - * the new DStream will generate RDDs); must be a multiple of this - * DStream's batching interval - * @param partitioner partitioner for controlling the partitioning of each RDD in the new - * DStream. - */ -@JvmName("groupByKeyAndWindowArity2") -fun JavaDStreamLike, *, *>.groupByKeyAndWindow( - windowDuration: Duration, - slideDuration: Duration = dstream().slideDuration(), - partitioner: Partitioner, -): JavaDStream>> = - mapToPair { it.toTuple() } - .groupByKeyAndWindow(windowDuration, slideDuration, partitioner) - .map { it.toArity() } - -/** - * Return a new DStream by applying `reduceByKey` over a sliding window. This is similar to - * `DStream.reduceByKey()` but applies it over a sliding window. Hash partitioning is used to - * generate the RDDs with `numPartitions` partitions. - * @param reduceFunc associative and commutative reduce function - * @param windowDuration width of the window; must be a multiple of this DStream's - * batching interval - * @param slideDuration sliding interval of the window (i.e., the interval after which - * the new DStream will generate RDDs); must be a multiple of this - * DStream's batching interval - * @param numPartitions number of partitions of each RDD in the new DStream. - */ -@JvmName("reduceByKeyAndWindowArity2") -fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( - windowDuration: Duration, - slideDuration: Duration = dstream().slideDuration(), - numPartitions: Int = dstream().ssc().sc().defaultParallelism(), - reduceFunc: (V, V) -> V, -): JavaDStream> = - mapToPair { it.toTuple() } - .reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, numPartitions) - .map { it.toArity() } - -/** - * Return a new DStream by applying `reduceByKey` over a sliding window. Similar to - * `DStream.reduceByKey()`, but applies it over a sliding window. - * @param reduceFunc associative and commutative reduce function - * @param windowDuration width of the window; must be a multiple of this DStream's - * batching interval - * @param slideDuration sliding interval of the window (i.e., the interval after which - * the new DStream will generate RDDs); must be a multiple of this - * DStream's batching interval - * @param partitioner partitioner for controlling the partitioning of each RDD - * in the new DStream. - */ -@JvmName("reduceByKeyAndWindowArity2") -fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( - windowDuration: Duration, - slideDuration: Duration = dstream().slideDuration(), - partitioner: Partitioner, - reduceFunc: (V, V) -> V, -): JavaDStream> = - mapToPair { it.toTuple() } - .reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, partitioner) - .map { it.toArity() } - -/** - * Return a new DStream by applying incremental `reduceByKey` over a sliding window. - * The reduced value of over a new window is calculated using the old window's reduced value : - * 1. reduce the new values that entered the window (e.g., adding new counts) - * - * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) - * - * This is more efficient than reduceByKeyAndWindow without "inverse reduce" function. - * However, it is applicable to only "invertible reduce functions". - * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. - * @param reduceFunc associative and commutative reduce function - * @param invReduceFunc inverse reduce function; such that for all y, invertible x: - * `invReduceFunc(reduceFunc(x, y), x) = y` - * @param windowDuration width of the window; must be a multiple of this DStream's - * batching interval - * @param slideDuration sliding interval of the window (i.e., the interval after which - * the new DStream will generate RDDs); must be a multiple of this - * DStream's batching interval - * @param filterFunc Optional function to filter expired key-value pairs; - * only pairs that satisfy the function are retained - */ -@JvmName("reduceByKeyAndWindowArity2") -fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( - invReduceFunc: (V, V) -> V, - windowDuration: Duration, - slideDuration: Duration = dstream().slideDuration(), - numPartitions: Int = dstream().ssc().sc().defaultParallelism(), - filterFunc: ((Arity2) -> Boolean)? = null, - reduceFunc: (V, V) -> V, -): JavaDStream> = - mapToPair { it.toTuple() } - .reduceByKeyAndWindow( - reduceFunc, - invReduceFunc, - windowDuration, - slideDuration, - numPartitions, - filterFunc?.let { - { tuple -> - filterFunc(tuple.toArity()) - } - } - ) - .map { it.toArity() } - -/** - * Return a new DStream by applying incremental `reduceByKey` over a sliding window. - * The reduced value of over a new window is calculated using the old window's reduced value : - * 1. reduce the new values that entered the window (e.g., adding new counts) - * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) - * This is more efficient than reduceByKeyAndWindow without "inverse reduce" function. - * However, it is applicable to only "invertible reduce functions". - * @param reduceFunc associative and commutative reduce function - * @param invReduceFunc inverse reduce function - * @param windowDuration width of the window; must be a multiple of this DStream's - * batching interval - * @param slideDuration sliding interval of the window (i.e., the interval after which - * the new DStream will generate RDDs); must be a multiple of this - * DStream's batching interval - * @param partitioner partitioner for controlling the partitioning of each RDD in the new - * DStream. - * @param filterFunc Optional function to filter expired key-value pairs; - * only pairs that satisfy the function are retained - */ -@JvmName("reduceByKeyAndWindowArity2") -fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( - invReduceFunc: (V, V) -> V, - windowDuration: Duration, - slideDuration: Duration = dstream().slideDuration(), - partitioner: Partitioner, - filterFunc: ((Arity2) -> Boolean)? = null, - reduceFunc: (V, V) -> V, -): JavaDStream> = - mapToPair { it.toTuple() } - .reduceByKeyAndWindow( - reduceFunc, - invReduceFunc, - windowDuration, - slideDuration, - partitioner, - filterFunc?.let { - { tuple -> - filterFunc(tuple.toArity()) - } - } - ) - .map { it.toArity() } - -/** - * Return a [MapWithStateDStream] by applying a function to every key-value element of - * `this` stream, while maintaining some state data for each unique key. The mapping function - * and other specification (e.g. partitioners, timeouts, initial state data, etc.) of this - * transformation can be specified using `StateSpec` class. The state data is accessible in - * as a parameter of type `State` in the mapping function. - * - * Example of using `mapWithState`: - * {{{ - * // A mapping function that maintains an integer state and return a String - * def mappingFunction(key: String, value: Option[Int], state: State[Int]): Option[String] = { - * // Use state.exists(), state.get(), state.update() and state.remove() - * // to manage state, and return the necessary string - * } - * - * val spec = StateSpec.function(mappingFunction).numPartitions(10) - * - * val mapWithStateDStream = keyValueDStream.mapWithState[StateType, MappedType](spec) - * }}} - * - * @param spec Specification of this transformation - * @tparam StateType Class type of the state data - * @tparam MappedType Class type of the mapped data - */ -@JvmName("mapWithStateArity2") -fun JavaDStreamLike, *, *>.mapWithState( - spec: StateSpec, -): JavaMapWithStateDStream = - mapToPair { it.toTuple() } - .mapWithState(spec) - -/** - * Return a new "state" DStream where the state for each key is updated by applying - * the given function on the previous state of the key and the new values of each key. - * In every batch the updateFunc will be called for each state even if there are no new values. - * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. - * @param updateFunc State update function. If `this` function returns None, then - * corresponding state key-value pair will be eliminated. - * @tparam S State type - */ -@JvmName("updateStateByKeyArity2") -fun JavaDStreamLike, *, *>.updateStateByKey( - numPartitions: Int = dstream().ssc().sc().defaultParallelism(), - updateFunc: (List, S?) -> S?, -): JavaDStream> = - mapToPair { it.toTuple() } - .updateStateByKey( - { list: List, s: Optional -> - updateFunc(list, s.toNullable()).toOptional() - }, - numPartitions, - ) - .map { it.toArity() } - -/** - * Return a new "state" DStream where the state for each key is updated by applying - * the given function on the previous state of the key and the new values of each key. - * In every batch the updateFunc will be called for each state even if there are no new values. - * [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD. - * @param updateFunc State update function. Note, that this function may generate a different - * tuple with a different key than the input key. Therefore keys may be removed - * or added in this way. It is up to the developer to decide whether to - * remember the partitioner despite the key being changed. - * @param partitioner Partitioner for controlling the partitioning of each RDD in the new - * DStream - * @tparam S State type - */ -@JvmName("updateStateByKeyArity2") -fun JavaDStreamLike, *, *>.updateStateByKey( - partitioner: Partitioner, - updateFunc: (List, S?) -> S?, -): JavaDStream> = - mapToPair { it.toTuple() } - .updateStateByKey( - { list: List, s: Optional -> - updateFunc(list, s.toNullable()).toOptional() - }, - partitioner, - ) - .map { it.toArity() } - -/** - * Return a new "state" DStream where the state for each key is updated by applying - * the given function on the previous state of the key and the new values of the key. - * org.apache.spark.Partitioner is used to control the partitioning of each RDD. - * @param updateFunc State update function. If `this` function returns None, then - * corresponding state key-value pair will be eliminated. - * @param partitioner Partitioner for controlling the partitioning of each RDD in the new - * DStream. - * @param initialRDD initial state value of each key. - * @tparam S State type - */ -@JvmName("updateStateByKeyArity2") -fun JavaDStreamLike, *, *>.updateStateByKey( - partitioner: Partitioner, - initialRDD: JavaRDD>, - updateFunc: (List, S?) -> S?, -): JavaDStream> = - mapToPair { it.toTuple() } - .updateStateByKey( - { list: List, s: Optional -> - updateFunc(list, s.toNullable()).toOptional() - }, - partitioner, - initialRDD.mapToPair { it.toTuple() }, - ) - .map { it.toArity() } - -/** - * Return a new DStream by applying a map function to the value of each key-value pairs in - * 'this' DStream without changing the key. - */ -@JvmName("mapValuesArity2") -fun JavaDStreamLike, *, *>.mapValues( - mapValuesFunc: (V) -> U, -): JavaDStream> = - mapToPair { it.toTuple() } - .mapValues(mapValuesFunc) - .map { it.toArity() } - -/** - * Return a new DStream by applying a flatmap function to the value of each key-value pairs in - * 'this' DStream without changing the key. - */ -@JvmName("flatMapValuesArity2") -fun JavaDStreamLike, *, *>.flatMapValues( - flatMapValuesFunc: (V) -> Iterator, -): JavaDStream> = - mapToPair { it.toTuple() } - .flatMapValues(flatMapValuesFunc) - .map { it.toArity() } - -/** - * Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream. - * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. - */ -@JvmName("cogroupArity2") -fun JavaDStreamLike, *, *>.cogroup( - other: JavaDStreamLike, *, *>, - numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -): JavaDStream, Iterable>>> = - mapToPair { it.toTuple() } - .cogroup( - other.mapToPair { it.toTuple() }, - numPartitions, - ) - .map { - c(it._1, it._2.toArity()) - } - -/** - * Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream. - * The supplied org.apache.spark.Partitioner is used to partition the generated RDDs. - */ -@JvmName("cogroupArity2") -fun JavaDStreamLike, *, *>.cogroup( - other: JavaDStreamLike, *, *>, - partitioner: Partitioner, -): JavaDStream, Iterable>>> = - mapToPair { it.toTuple() } - .cogroup( - other.mapToPair { it.toTuple() }, - partitioner, - ) - .map { - c(it._1, it._2.toArity()) - } - -/** - * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream. - * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. - */ -@JvmName("joinArity2") -fun JavaDStreamLike, *, *>.join( - other: JavaDStreamLike, *, *>, - numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -): JavaDStream>> = - mapToPair { it.toTuple() } - .join( - other.mapToPair { it.toTuple() }, - numPartitions, - ) - .map { - c(it._1, it._2.toArity()) - } - -/** - * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream. - * The supplied org.apache.spark.Partitioner is used to control the partitioning of each RDD. - */ -@JvmName("joinArity2") -fun JavaDStreamLike, *, *>.join( - other: JavaDStreamLike, *, *>, - partitioner: Partitioner, -): JavaDStream>> = - mapToPair { it.toTuple() } - .join( - other.mapToPair { it.toTuple() }, - partitioner, - ) - .map { - c(it._1, it._2.toArity()) - } - -/** - * Return a new DStream by applying 'left outer join' between RDDs of `this` DStream and - * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` - * partitions. - */ -@JvmName("leftOuterJoinArity2") -fun JavaDStreamLike, *, *>.leftOuterJoin( - other: JavaDStreamLike, *, *>, - numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -): JavaDStream>> = - mapToPair { it.toTuple() } - .leftOuterJoin( - other.mapToPair { it.toTuple() }, - numPartitions, - ) - .map { - c(it._1, c(it._2._1, it._2._2.toNullable())) - } - -/** - * Return a new DStream by applying 'left outer join' between RDDs of `this` DStream and - * `other` DStream. The supplied org.apache.spark.Partitioner is used to control - * the partitioning of each RDD. - */ -@JvmName("leftOuterJoinArity2") -fun JavaDStreamLike, *, *>.leftOuterJoin( - other: JavaDStreamLike, *, *>, - partitioner: Partitioner, -): JavaDStream>> = - mapToPair { it.toTuple() } - .leftOuterJoin( - other.mapToPair { it.toTuple() }, - partitioner, - ) - .map { - c(it._1, c(it._2._1, it._2._2.toNullable())) - } - -/** - * Return a new DStream by applying 'right outer join' between RDDs of `this` DStream and - * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` - * partitions. - */ -@JvmName("rightOuterJoinArity2") -fun JavaDStreamLike, *, *>.rightOuterJoin( - other: JavaDStreamLike, *, *>, - numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -): JavaDStream>> = - mapToPair { it.toTuple() } - .rightOuterJoin( - other.mapToPair { it.toTuple() }, - numPartitions, - ) - .map { - c(it._1, c(it._2._1.toNullable(), it._2._2)) - } - -/** - * Return a new DStream by applying 'right outer join' between RDDs of `this` DStream and - * `other` DStream. The supplied org.apache.spark.Partitioner is used to control - * the partitioning of each RDD. - */ -@JvmName("rightOuterJoinArity2") -fun JavaDStreamLike, *, *>.rightOuterJoin( - other: JavaDStreamLike, *, *>, - partitioner: Partitioner, -): JavaDStream>> = - mapToPair { it.toTuple() } - .rightOuterJoin( - other.mapToPair { it.toTuple() }, - partitioner, - ) - .map { - c(it._1, c(it._2._1.toNullable(), it._2._2)) - } - -/** - * Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and - * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` - * partitions. - */ -@JvmName("fullOuterJoinArity2") -fun JavaDStreamLike, *, *>.fullOuterJoin( - other: JavaDStreamLike, *, *>, - numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -): JavaDStream>> = - mapToPair { it.toTuple() } - .fullOuterJoin( - other.mapToPair { it.toTuple() }, - numPartitions, - ) - .map { - c(it._1, c(it._2._1.toNullable(), it._2._2.toNullable())) - } - -/** - * Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and - * `other` DStream. The supplied org.apache.spark.Partitioner is used to control - * the partitioning of each RDD. - */ -@JvmName("fullOuterJoinArity2") -fun JavaDStreamLike, *, *>.fullOuterJoin( - other: JavaDStreamLike, *, *>, - partitioner: Partitioner, -): JavaDStream>> = - mapToPair { it.toTuple() } - .fullOuterJoin( - other.mapToPair { it.toTuple() }, - partitioner, - ) - .map { - c(it._1, c(it._2._1.toNullable(), it._2._2.toNullable())) - } - -/** - * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is - * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix". - */ -@JvmName("saveAsHadoopFilesArity2") -fun JavaDStreamLike, *, *>.saveAsHadoopFiles( - prefix: String, suffix: String, -): Unit = - mapToPair { it.toTuple() } - .saveAsHadoopFiles(prefix, suffix) - -/** - * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is - * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix". - */ -@JvmName("saveAsNewAPIHadoopFilesArity2") -fun JavaDStreamLike, *, *>.saveAsNewAPIHadoopFiles( - prefix: String, suffix: String, -): Unit = - mapToPair { it.toTuple() } - .saveAsNewAPIHadoopFiles(prefix, suffix) - - - - - -/** - * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to - * generate the RDDs with `numPartitions` partitions. - */ -@JvmName("groupByKeyPair") -fun JavaDStreamLike, *, *>.groupByKey( - numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -): JavaDStream>> = - mapToPair { it.toTuple() } - .groupByKey(numPartitions) - .map { it.toPair() } - -/** - * Return a new DStream by applying `groupByKey` on each RDD. The supplied - * org.apache.spark.Partitioner is used to control the partitioning of each RDD. - */ -@JvmName("groupByKeyPair") -fun JavaDStreamLike, *, *>.groupByKey(partitioner: Partitioner): JavaDStream>> = - mapToPair { it.toTuple() } - .groupByKey(partitioner) - .map { it.toPair() } - -/** - * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are - * merged using the supplied reduce function. Hash partitioning is used to generate the RDDs - * with `numPartitions` partitions. - */ -@JvmName("reduceByKeyPair") -fun JavaDStreamLike, *, *>.reduceByKey( - numPartitions: Int = dstream().ssc().sc().defaultParallelism(), - reduceFunc: (V, V) -> V, -): JavaDStream> = - mapToPair { it.toTuple() } - .reduceByKey(reduceFunc, numPartitions) - .map { it.toPair() } - -/** - * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are - * merged using the supplied reduce function. org.apache.spark.Partitioner is used to control - * the partitioning of each RDD. - */ -@JvmName("reduceByKeyPair") -fun JavaDStreamLike, *, *>.reduceByKey( - partitioner: Partitioner, - reduceFunc: (V, V) -> V, -): JavaDStream> = - mapToPair { it.toTuple() } - .reduceByKey(reduceFunc, partitioner) - .map { it.toPair() } - -/** - * Combine elements of each key in DStream's RDDs using custom functions. This is similar to the - * combineByKey for RDDs. Please refer to combineByKey in - * org.apache.spark.rdd.PairRDDFunctions in the Spark core documentation for more information. - */ -@JvmName("combineByKeyPair") -fun JavaDStreamLike, *, *>.combineByKey( - createCombiner: (V) -> C, - mergeValue: (C, V) -> C, - mergeCombiner: (C, C) -> C, - partitioner: Partitioner, - mapSideCombine: Boolean = true, -): JavaDStream> = - mapToPair { it.toTuple() } - .combineByKey(createCombiner, mergeValue, mergeCombiner, partitioner, mapSideCombine) - .map { it.toPair() } - -/** - * Return a new DStream by applying `groupByKey` over a sliding window on `this` DStream. - * Similar to `DStream.groupByKey()`, but applies it over a sliding window. - * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. - * @param windowDuration width of the window; must be a multiple of this DStream's - * batching interval - * @param slideDuration sliding interval of the window (i.e., the interval after which - * the new DStream will generate RDDs); must be a multiple of this - * DStream's batching interval - * @param numPartitions number of partitions of each RDD in the new DStream; if not specified - * then Spark's default number of partitions will be used - */ -@JvmName("groupByKeyAndWindowPair") -fun JavaDStreamLike, *, *>.groupByKeyAndWindow( - windowDuration: Duration, - slideDuration: Duration = dstream().slideDuration(), - numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -): JavaDStream>> = - mapToPair { it.toTuple() } - .groupByKeyAndWindow(windowDuration, slideDuration, numPartitions) - .map { it.toPair() } - -/** - * Create a new DStream by applying `groupByKey` over a sliding window on `this` DStream. - * Similar to `DStream.groupByKey()`, but applies it over a sliding window. - * @param windowDuration width of the window; must be a multiple of this DStream's - * batching interval - * @param slideDuration sliding interval of the window (i.e., the interval after which - * the new DStream will generate RDDs); must be a multiple of this - * DStream's batching interval - * @param partitioner partitioner for controlling the partitioning of each RDD in the new - * DStream. - */ -@JvmName("groupByKeyAndWindowPair") -fun JavaDStreamLike, *, *>.groupByKeyAndWindow( - windowDuration: Duration, - slideDuration: Duration = dstream().slideDuration(), - partitioner: Partitioner, -): JavaDStream>> = - mapToPair { it.toTuple() } - .groupByKeyAndWindow(windowDuration, slideDuration, partitioner) - .map { it.toPair() } - -/** - * Return a new DStream by applying `reduceByKey` over a sliding window. This is similar to - * `DStream.reduceByKey()` but applies it over a sliding window. Hash partitioning is used to - * generate the RDDs with `numPartitions` partitions. - * @param reduceFunc associative and commutative reduce function - * @param windowDuration width of the window; must be a multiple of this DStream's - * batching interval - * @param slideDuration sliding interval of the window (i.e., the interval after which - * the new DStream will generate RDDs); must be a multiple of this - * DStream's batching interval - * @param numPartitions number of partitions of each RDD in the new DStream. - */ -@JvmName("reduceByKeyAndWindowPair") -fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( - windowDuration: Duration, - slideDuration: Duration = dstream().slideDuration(), - numPartitions: Int = dstream().ssc().sc().defaultParallelism(), - reduceFunc: (V, V) -> V, -): JavaDStream> = - mapToPair { it.toTuple() } - .reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, numPartitions) - .map { it.toPair() } - -/** - * Return a new DStream by applying `reduceByKey` over a sliding window. Similar to - * `DStream.reduceByKey()`, but applies it over a sliding window. - * @param reduceFunc associative and commutative reduce function - * @param windowDuration width of the window; must be a multiple of this DStream's - * batching interval - * @param slideDuration sliding interval of the window (i.e., the interval after which - * the new DStream will generate RDDs); must be a multiple of this - * DStream's batching interval - * @param partitioner partitioner for controlling the partitioning of each RDD - * in the new DStream. - */ -@JvmName("reduceByKeyAndWindowPair") -fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( - windowDuration: Duration, - slideDuration: Duration = dstream().slideDuration(), - partitioner: Partitioner, - reduceFunc: (V, V) -> V, -): JavaDStream> = - mapToPair { it.toTuple() } - .reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, partitioner) - .map { it.toPair() } - -/** - * Return a new DStream by applying incremental `reduceByKey` over a sliding window. - * The reduced value of over a new window is calculated using the old window's reduced value : - * 1. reduce the new values that entered the window (e.g., adding new counts) - * - * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) - * - * This is more efficient than reduceByKeyAndWindow without "inverse reduce" function. - * However, it is applicable to only "invertible reduce functions". - * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. - * @param reduceFunc associative and commutative reduce function - * @param invReduceFunc inverse reduce function; such that for all y, invertible x: - * `invReduceFunc(reduceFunc(x, y), x) = y` - * @param windowDuration width of the window; must be a multiple of this DStream's - * batching interval - * @param slideDuration sliding interval of the window (i.e., the interval after which - * the new DStream will generate RDDs); must be a multiple of this - * DStream's batching interval - * @param filterFunc Optional function to filter expired key-value pairs; - * only pairs that satisfy the function are retained - */ -@JvmName("reduceByKeyAndWindowPair") -fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( - invReduceFunc: (V, V) -> V, - windowDuration: Duration, - slideDuration: Duration = dstream().slideDuration(), - numPartitions: Int = dstream().ssc().sc().defaultParallelism(), - filterFunc: ((Pair) -> Boolean)? = null, - reduceFunc: (V, V) -> V, -): JavaDStream> = - mapToPair { it.toTuple() } - .reduceByKeyAndWindow( - reduceFunc, - invReduceFunc, - windowDuration, - slideDuration, - numPartitions, - filterFunc?.let { - { tuple -> - filterFunc(tuple.toPair()) - } - } - ) - .map { it.toPair() } - -/** - * Return a new DStream by applying incremental `reduceByKey` over a sliding window. - * The reduced value of over a new window is calculated using the old window's reduced value : - * 1. reduce the new values that entered the window (e.g., adding new counts) - * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) - * This is more efficient than reduceByKeyAndWindow without "inverse reduce" function. - * However, it is applicable to only "invertible reduce functions". - * @param reduceFunc associative and commutative reduce function - * @param invReduceFunc inverse reduce function - * @param windowDuration width of the window; must be a multiple of this DStream's - * batching interval - * @param slideDuration sliding interval of the window (i.e., the interval after which - * the new DStream will generate RDDs); must be a multiple of this - * DStream's batching interval - * @param partitioner partitioner for controlling the partitioning of each RDD in the new - * DStream. - * @param filterFunc Optional function to filter expired key-value pairs; - * only pairs that satisfy the function are retained - */ -@JvmName("reduceByKeyAndWindowPair") -fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( - invReduceFunc: (V, V) -> V, - windowDuration: Duration, - slideDuration: Duration = dstream().slideDuration(), - partitioner: Partitioner, - filterFunc: ((Pair) -> Boolean)? = null, - reduceFunc: (V, V) -> V, -): JavaDStream> = - mapToPair { it.toTuple() } - .reduceByKeyAndWindow( - reduceFunc, - invReduceFunc, - windowDuration, - slideDuration, - partitioner, - filterFunc?.let { - { tuple -> - filterFunc(tuple.toPair()) - } - } - ) - .map { it.toPair() } - -/** - * Return a [MapWithStateDStream] by applying a function to every key-value element of - * `this` stream, while maintaining some state data for each unique key. The mapping function - * and other specification (e.g. partitioners, timeouts, initial state data, etc.) of this - * transformation can be specified using `StateSpec` class. The state data is accessible in - * as a parameter of type `State` in the mapping function. - * - * Example of using `mapWithState`: - * {{{ - * // A mapping function that maintains an integer state and return a String - * def mappingFunction(key: String, value: Option[Int], state: State[Int]): Option[String] = { - * // Use state.exists(), state.get(), state.update() and state.remove() - * // to manage state, and return the necessary string - * } - * - * val spec = StateSpec.function(mappingFunction).numPartitions(10) - * - * val mapWithStateDStream = keyValueDStream.mapWithState[StateType, MappedType](spec) - * }}} - * - * @param spec Specification of this transformation - * @tparam StateType Class type of the state data - * @tparam MappedType Class type of the mapped data - */ -@JvmName("mapWithStatePair") -fun JavaDStreamLike, *, *>.mapWithState( - spec: StateSpec, -): JavaMapWithStateDStream = - mapToPair { it.toTuple() } - .mapWithState(spec) - -/** - * Return a new "state" DStream where the state for each key is updated by applying - * the given function on the previous state of the key and the new values of each key. - * In every batch the updateFunc will be called for each state even if there are no new values. - * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. - * @param updateFunc State update function. If `this` function returns None, then - * corresponding state key-value pair will be eliminated. - * @tparam S State type - */ -@JvmName("updateStateByKeyPair") -fun JavaDStreamLike, *, *>.updateStateByKey( - numPartitions: Int = dstream().ssc().sc().defaultParallelism(), - updateFunc: (List, S?) -> S?, -): JavaDStream> = - mapToPair { it.toTuple() } - .updateStateByKey( - { list: List, s: Optional -> - updateFunc(list, s.toNullable()).toOptional() - }, - numPartitions, - ) - .map { it.toPair() } - -/** - * Return a new "state" DStream where the state for each key is updated by applying - * the given function on the previous state of the key and the new values of each key. - * In every batch the updateFunc will be called for each state even if there are no new values. - * [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD. - * @param updateFunc State update function. Note, that this function may generate a different - * tuple with a different key than the input key. Therefore keys may be removed - * or added in this way. It is up to the developer to decide whether to - * remember the partitioner despite the key being changed. - * @param partitioner Partitioner for controlling the partitioning of each RDD in the new - * DStream - * @tparam S State type - */ -@JvmName("updateStateByKeyPair") -fun JavaDStreamLike, *, *>.updateStateByKey( - partitioner: Partitioner, - updateFunc: (List, S?) -> S?, -): JavaDStream> = - mapToPair { it.toTuple() } - .updateStateByKey( - { list: List, s: Optional -> - updateFunc(list, s.toNullable()).toOptional() - }, - partitioner, - ) - .map { it.toPair() } - -/** - * Return a new "state" DStream where the state for each key is updated by applying - * the given function on the previous state of the key and the new values of the key. - * org.apache.spark.Partitioner is used to control the partitioning of each RDD. - * @param updateFunc State update function. If `this` function returns None, then - * corresponding state key-value pair will be eliminated. - * @param partitioner Partitioner for controlling the partitioning of each RDD in the new - * DStream. - * @param initialRDD initial state value of each key. - * @tparam S State type - */ -@JvmName("updateStateByKeyPair") -fun JavaDStreamLike, *, *>.updateStateByKey( - partitioner: Partitioner, - initialRDD: JavaRDD>, - updateFunc: (List, S?) -> S?, -): JavaDStream> = - mapToPair { it.toTuple() } - .updateStateByKey( - { list: List, s: Optional -> - updateFunc(list, s.toNullable()).toOptional() - }, - partitioner, - initialRDD.mapToPair { it.toTuple() }, - ) - .map { it.toPair() } - -/** - * Return a new DStream by applying a map function to the value of each key-value pairs in - * 'this' DStream without changing the key. - */ -@JvmName("mapValuesPair") -fun JavaDStreamLike, *, *>.mapValues( - mapValuesFunc: (V) -> U, -): JavaDStream> = - mapToPair { it.toTuple() } - .mapValues(mapValuesFunc) - .map { it.toPair() } - -/** - * Return a new DStream by applying a flatmap function to the value of each key-value pairs in - * 'this' DStream without changing the key. - */ -@JvmName("flatMapValuesPair") -fun JavaDStreamLike, *, *>.flatMapValues( - flatMapValuesFunc: (V) -> Iterator, -): JavaDStream> = - mapToPair { it.toTuple() } - .flatMapValues(flatMapValuesFunc) - .map { it.toPair() } - -/** - * Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream. - * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. - */ -@JvmName("cogroupPair") -fun JavaDStreamLike, *, *>.cogroup( - other: JavaDStreamLike, *, *>, - numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -): JavaDStream, Iterable>>> = - mapToPair { it.toTuple() } - .cogroup( - other.mapToPair { it.toTuple() }, - numPartitions, - ) - .map { - Pair(it._1, it._2.toPair()) - } - -/** - * Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream. - * The supplied org.apache.spark.Partitioner is used to partition the generated RDDs. - */ -@JvmName("cogroupPair") -fun JavaDStreamLike, *, *>.cogroup( - other: JavaDStreamLike, *, *>, - partitioner: Partitioner, -): JavaDStream, Iterable>>> = - mapToPair { it.toTuple() } - .cogroup( - other.mapToPair { it.toTuple() }, - partitioner, - ) - .map { - Pair(it._1, it._2.toPair()) - } - -/** - * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream. - * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. - */ -@JvmName("joinPair") -fun JavaDStreamLike, *, *>.join( - other: JavaDStreamLike, *, *>, - numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -): JavaDStream>> = - mapToPair { it.toTuple() } - .join( - other.mapToPair { it.toTuple() }, - numPartitions, - ) - .map { - Pair(it._1, it._2.toPair()) - } - -/** - * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream. - * The supplied org.apache.spark.Partitioner is used to control the partitioning of each RDD. - */ -@JvmName("joinPair") -fun JavaDStreamLike, *, *>.join( - other: JavaDStreamLike, *, *>, - partitioner: Partitioner, -): JavaDStream>> = - mapToPair { it.toTuple() } - .join( - other.mapToPair { it.toTuple() }, - partitioner, - ) - .map { - Pair(it._1, it._2.toPair()) - } - -/** - * Return a new DStream by applying 'left outer join' between RDDs of `this` DStream and - * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` - * partitions. - */ -@JvmName("leftOuterJoinPair") -fun JavaDStreamLike, *, *>.leftOuterJoin( - other: JavaDStreamLike, *, *>, - numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -): JavaDStream>> = - mapToPair { it.toTuple() } - .leftOuterJoin( - other.mapToPair { it.toTuple() }, - numPartitions, - ) - .map { - Pair(it._1, Pair(it._2._1, it._2._2.toNullable())) - } - -/** - * Return a new DStream by applying 'left outer join' between RDDs of `this` DStream and - * `other` DStream. The supplied org.apache.spark.Partitioner is used to control - * the partitioning of each RDD. - */ -@JvmName("leftOuterJoinPair") -fun JavaDStreamLike, *, *>.leftOuterJoin( - other: JavaDStreamLike, *, *>, - partitioner: Partitioner, -): JavaDStream>> = - mapToPair { it.toTuple() } - .leftOuterJoin( - other.mapToPair { it.toTuple() }, - partitioner, - ) - .map { - Pair(it._1, Pair(it._2._1, it._2._2.toNullable())) - } - -/** - * Return a new DStream by applying 'right outer join' between RDDs of `this` DStream and - * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` - * partitions. - */ -@JvmName("rightOuterJoinPair") -fun JavaDStreamLike, *, *>.rightOuterJoin( - other: JavaDStreamLike, *, *>, - numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -): JavaDStream>> = - mapToPair { it.toTuple() } - .rightOuterJoin( - other.mapToPair { it.toTuple() }, - numPartitions, - ) - .map { - Pair(it._1, Pair(it._2._1.toNullable(), it._2._2)) - } - -/** - * Return a new DStream by applying 'right outer join' between RDDs of `this` DStream and - * `other` DStream. The supplied org.apache.spark.Partitioner is used to control - * the partitioning of each RDD. - */ -@JvmName("rightOuterJoinPair") -fun JavaDStreamLike, *, *>.rightOuterJoin( - other: JavaDStreamLike, *, *>, - partitioner: Partitioner, -): JavaDStream>> = - mapToPair { it.toTuple() } - .rightOuterJoin( - other.mapToPair { it.toTuple() }, - partitioner, - ) - .map { - Pair(it._1, Pair(it._2._1.toNullable(), it._2._2)) - } - -/** - * Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and - * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` - * partitions. - */ -@JvmName("fullOuterJoinPair") -fun JavaDStreamLike, *, *>.fullOuterJoin( - other: JavaDStreamLike, *, *>, - numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -): JavaDStream>> = - mapToPair { it.toTuple() } - .fullOuterJoin( - other.mapToPair { it.toTuple() }, - numPartitions, - ) - .map { - Pair(it._1, Pair(it._2._1.toNullable(), it._2._2.toNullable())) - } - -/** - * Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and - * `other` DStream. The supplied org.apache.spark.Partitioner is used to control - * the partitioning of each RDD. - */ -@JvmName("fullOuterJoinPair") -fun JavaDStreamLike, *, *>.fullOuterJoin( - other: JavaDStreamLike, *, *>, - partitioner: Partitioner, -): JavaDStream>> = - mapToPair { it.toTuple() } - .fullOuterJoin( - other.mapToPair { it.toTuple() }, - partitioner, - ) - .map { - Pair(it._1, Pair(it._2._1.toNullable(), it._2._2.toNullable())) - } - -/** - * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is - * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix". - */ -@JvmName("saveAsHadoopFilesPair") -fun JavaDStreamLike, *, *>.saveAsHadoopFiles( - prefix: String, suffix: String, -): Unit = - mapToPair { it.toTuple() } - .saveAsHadoopFiles(prefix, suffix) - -/** - * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is - * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix". - */ -@JvmName("saveAsNewAPIHadoopFilesPair") -fun JavaDStreamLike, *, *>.saveAsNewAPIHadoopFiles( - prefix: String, suffix: String, -): Unit = - mapToPair { it.toTuple() } - .saveAsNewAPIHadoopFiles(prefix, suffix) +fun JavaRDD>.toPairRDD(): JavaPairRDD = JavaPairRDD.fromJavaRDD(this) +@JvmName("arity2ToPairDStream") +fun JavaDStreamLike, *, *>.toPairDStream(): JavaPairDStream = + mapToPair(Arity2::toTuple) +@JvmName("pairToPairDStream") +fun JavaDStreamLike, *, *>.toPairDStream(): JavaPairDStream = + mapToPair(Pair::toTuple) +///** +// * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to +// * generate the RDDs with `numPartitions` partitions. +// */ +//@JvmName("groupByKeyArity2") +//fun JavaDStreamLike, *, *>.groupByKey( +// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +//): JavaDStream>> = +// mapToPair { it.toTuple() } +// .groupByKey(numPartitions) +// .map { it.toArity() } +// +///** +// * Return a new DStream by applying `groupByKey` on each RDD. The supplied +// * org.apache.spark.Partitioner is used to control the partitioning of each RDD. +// */ +//@JvmName("groupByKeyArity2") +//fun JavaDStreamLike, *, *>.groupByKey(partitioner: Partitioner): JavaDStream>> = +// mapToPair { it.toTuple() } +// .groupByKey(partitioner) +// .map { it.toArity() } +// +///** +// * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are +// * merged using the supplied reduce function. Hash partitioning is used to generate the RDDs +// * with `numPartitions` partitions. +// */ +//@JvmName("reduceByKeyArity2") +//fun JavaDStreamLike, *, *>.reduceByKey( +// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +// reduceFunc: (V, V) -> V, +//): JavaDStream> = +// mapToPair { it.toTuple() } +// .reduceByKey(reduceFunc, numPartitions) +// .map { it.toArity() } +// +///** +// * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are +// * merged using the supplied reduce function. org.apache.spark.Partitioner is used to control +// * the partitioning of each RDD. +// */ +//@JvmName("reduceByKeyArity2") +//fun JavaDStreamLike, *, *>.reduceByKey( +// partitioner: Partitioner, +// reduceFunc: (V, V) -> V, +//): JavaDStream> = +// mapToPair { it.toTuple() } +// .reduceByKey(reduceFunc, partitioner) +// .map { it.toArity() } +// +///** +// * Combine elements of each key in DStream's RDDs using custom functions. This is similar to the +// * combineByKey for RDDs. Please refer to combineByKey in +// * org.apache.spark.rdd.PairRDDFunctions in the Spark core documentation for more information. +// */ +//@JvmName("combineByKeyArity2") +//fun JavaDStreamLike, *, *>.combineByKey( +// createCombiner: (V) -> C, +// mergeValue: (C, V) -> C, +// mergeCombiner: (C, C) -> C, +// partitioner: Partitioner, +// mapSideCombine: Boolean = true, +//): JavaDStream> = +// mapToPair { it.toTuple() } +// .combineByKey(createCombiner, mergeValue, mergeCombiner, partitioner, mapSideCombine) +// .map { it.toArity() } +// +///** +// * Return a new DStream by applying `groupByKey` over a sliding window on `this` DStream. +// * Similar to `DStream.groupByKey()`, but applies it over a sliding window. +// * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. +// * @param windowDuration width of the window; must be a multiple of this DStream's +// * batching interval +// * @param slideDuration sliding interval of the window (i.e., the interval after which +// * the new DStream will generate RDDs); must be a multiple of this +// * DStream's batching interval +// * @param numPartitions number of partitions of each RDD in the new DStream; if not specified +// * then Spark's default number of partitions will be used +// */ +//@JvmName("groupByKeyAndWindowArity2") +//fun JavaDStreamLike, *, *>.groupByKeyAndWindow( +// windowDuration: Duration, +// slideDuration: Duration = dstream().slideDuration(), +// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +//): JavaDStream>> = +// mapToPair { it.toTuple() } +// .groupByKeyAndWindow(windowDuration, slideDuration, numPartitions) +// .map { it.toArity() } +// +///** +// * Create a new DStream by applying `groupByKey` over a sliding window on `this` DStream. +// * Similar to `DStream.groupByKey()`, but applies it over a sliding window. +// * @param windowDuration width of the window; must be a multiple of this DStream's +// * batching interval +// * @param slideDuration sliding interval of the window (i.e., the interval after which +// * the new DStream will generate RDDs); must be a multiple of this +// * DStream's batching interval +// * @param partitioner partitioner for controlling the partitioning of each RDD in the new +// * DStream. +// */ +//@JvmName("groupByKeyAndWindowArity2") +//fun JavaDStreamLike, *, *>.groupByKeyAndWindow( +// windowDuration: Duration, +// slideDuration: Duration = dstream().slideDuration(), +// partitioner: Partitioner, +//): JavaDStream>> = +// mapToPair { it.toTuple() } +// .groupByKeyAndWindow(windowDuration, slideDuration, partitioner) +// .map { it.toArity() } +// +///** +// * Return a new DStream by applying `reduceByKey` over a sliding window. This is similar to +// * `DStream.reduceByKey()` but applies it over a sliding window. Hash partitioning is used to +// * generate the RDDs with `numPartitions` partitions. +// * @param reduceFunc associative and commutative reduce function +// * @param windowDuration width of the window; must be a multiple of this DStream's +// * batching interval +// * @param slideDuration sliding interval of the window (i.e., the interval after which +// * the new DStream will generate RDDs); must be a multiple of this +// * DStream's batching interval +// * @param numPartitions number of partitions of each RDD in the new DStream. +// */ +//@JvmName("reduceByKeyAndWindowArity2") +//fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( +// windowDuration: Duration, +// slideDuration: Duration = dstream().slideDuration(), +// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +// reduceFunc: (V, V) -> V, +//): JavaDStream> = +// mapToPair { it.toTuple() } +// .reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, numPartitions) +// .map { it.toArity() } +// +///** +// * Return a new DStream by applying `reduceByKey` over a sliding window. Similar to +// * `DStream.reduceByKey()`, but applies it over a sliding window. +// * @param reduceFunc associative and commutative reduce function +// * @param windowDuration width of the window; must be a multiple of this DStream's +// * batching interval +// * @param slideDuration sliding interval of the window (i.e., the interval after which +// * the new DStream will generate RDDs); must be a multiple of this +// * DStream's batching interval +// * @param partitioner partitioner for controlling the partitioning of each RDD +// * in the new DStream. +// */ +//@JvmName("reduceByKeyAndWindowArity2") +//fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( +// windowDuration: Duration, +// slideDuration: Duration = dstream().slideDuration(), +// partitioner: Partitioner, +// reduceFunc: (V, V) -> V, +//): JavaDStream> = +// mapToPair { it.toTuple() } +// .reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, partitioner) +// .map { it.toArity() } +// +///** +// * Return a new DStream by applying incremental `reduceByKey` over a sliding window. +// * The reduced value of over a new window is calculated using the old window's reduced value : +// * 1. reduce the new values that entered the window (e.g., adding new counts) +// * +// * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) +// * +// * This is more efficient than reduceByKeyAndWindow without "inverse reduce" function. +// * However, it is applicable to only "invertible reduce functions". +// * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. +// * @param reduceFunc associative and commutative reduce function +// * @param invReduceFunc inverse reduce function; such that for all y, invertible x: +// * `invReduceFunc(reduceFunc(x, y), x) = y` +// * @param windowDuration width of the window; must be a multiple of this DStream's +// * batching interval +// * @param slideDuration sliding interval of the window (i.e., the interval after which +// * the new DStream will generate RDDs); must be a multiple of this +// * DStream's batching interval +// * @param filterFunc Optional function to filter expired key-value pairs; +// * only pairs that satisfy the function are retained +// */ +//@JvmName("reduceByKeyAndWindowArity2") +//fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( +// invReduceFunc: (V, V) -> V, +// windowDuration: Duration, +// slideDuration: Duration = dstream().slideDuration(), +// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +// filterFunc: ((Arity2) -> Boolean)? = null, +// reduceFunc: (V, V) -> V, +//): JavaDStream> = +// mapToPair { it.toTuple() } +// .reduceByKeyAndWindow( +// reduceFunc, +// invReduceFunc, +// windowDuration, +// slideDuration, +// numPartitions, +// filterFunc?.let { +// { tuple -> +// filterFunc(tuple.toArity()) +// } +// } +// ) +// .map { it.toArity() } +// +///** +// * Return a new DStream by applying incremental `reduceByKey` over a sliding window. +// * The reduced value of over a new window is calculated using the old window's reduced value : +// * 1. reduce the new values that entered the window (e.g., adding new counts) +// * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) +// * This is more efficient than reduceByKeyAndWindow without "inverse reduce" function. +// * However, it is applicable to only "invertible reduce functions". +// * @param reduceFunc associative and commutative reduce function +// * @param invReduceFunc inverse reduce function +// * @param windowDuration width of the window; must be a multiple of this DStream's +// * batching interval +// * @param slideDuration sliding interval of the window (i.e., the interval after which +// * the new DStream will generate RDDs); must be a multiple of this +// * DStream's batching interval +// * @param partitioner partitioner for controlling the partitioning of each RDD in the new +// * DStream. +// * @param filterFunc Optional function to filter expired key-value pairs; +// * only pairs that satisfy the function are retained +// */ +//@JvmName("reduceByKeyAndWindowArity2") +//fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( +// invReduceFunc: (V, V) -> V, +// windowDuration: Duration, +// slideDuration: Duration = dstream().slideDuration(), +// partitioner: Partitioner, +// filterFunc: ((Arity2) -> Boolean)? = null, +// reduceFunc: (V, V) -> V, +//): JavaDStream> = +// mapToPair { it.toTuple() } +// .reduceByKeyAndWindow( +// reduceFunc, +// invReduceFunc, +// windowDuration, +// slideDuration, +// partitioner, +// filterFunc?.let { +// { tuple -> +// filterFunc(tuple.toArity()) +// } +// } +// ) +// .map { it.toArity() } +// +///** +// * Return a [MapWithStateDStream] by applying a function to every key-value element of +// * `this` stream, while maintaining some state data for each unique key. The mapping function +// * and other specification (e.g. partitioners, timeouts, initial state data, etc.) of this +// * transformation can be specified using `StateSpec` class. The state data is accessible in +// * as a parameter of type `State` in the mapping function. +// * +// * Example of using `mapWithState`: +// * {{{ +// * // A mapping function that maintains an integer state and return a String +// * def mappingFunction(key: String, value: Option[Int], state: State[Int]): Option[String] = { +// * // Use state.exists(), state.get(), state.update() and state.remove() +// * // to manage state, and return the necessary string +// * } +// * +// * val spec = StateSpec.function(mappingFunction).numPartitions(10) +// * +// * val mapWithStateDStream = keyValueDStream.mapWithState[StateType, MappedType](spec) +// * }}} +// * +// * @param spec Specification of this transformation +// * @tparam StateType Class type of the state data +// * @tparam MappedType Class type of the mapped data +// */ +//@JvmName("mapWithStateArity2") +//fun JavaDStreamLike, *, *>.mapWithState( +// spec: StateSpec, +//): JavaMapWithStateDStream = +// mapToPair { it.toTuple() } +// .mapWithState(spec) +// +///** +// * Return a new "state" DStream where the state for each key is updated by applying +// * the given function on the previous state of the key and the new values of each key. +// * In every batch the updateFunc will be called for each state even if there are no new values. +// * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. +// * @param updateFunc State update function. If `this` function returns None, then +// * corresponding state key-value pair will be eliminated. +// * @tparam S State type +// */ +//@JvmName("updateStateByKeyArity2") +//fun JavaDStreamLike, *, *>.updateStateByKey( +// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +// updateFunc: (List, S?) -> S?, +//): JavaDStream> = +// mapToPair { it.toTuple() } +// .updateStateByKey( +// { list: List, s: Optional -> +// updateFunc(list, s.getOrNull()).asOptional() +// }, +// numPartitions, +// ) +// .map { it.toArity() } +// +///** +// * Return a new "state" DStream where the state for each key is updated by applying +// * the given function on the previous state of the key and the new values of each key. +// * In every batch the updateFunc will be called for each state even if there are no new values. +// * [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD. +// * @param updateFunc State update function. Note, that this function may generate a different +// * tuple with a different key than the input key. Therefore keys may be removed +// * or added in this way. It is up to the developer to decide whether to +// * remember the partitioner despite the key being changed. +// * @param partitioner Partitioner for controlling the partitioning of each RDD in the new +// * DStream +// * @tparam S State type +// */ +//@JvmName("updateStateByKeyArity2") +//fun JavaDStreamLike, *, *>.updateStateByKey( +// partitioner: Partitioner, +// updateFunc: (List, S?) -> S?, +//): JavaDStream> = +// mapToPair { it.toTuple() } +// .updateStateByKey( +// { list: List, s: Optional -> +// updateFunc(list, s.getOrNull()).asOptional() +// }, +// partitioner, +// ) +// .map { it.toArity() } +// +///** +// * Return a new "state" DStream where the state for each key is updated by applying +// * the given function on the previous state of the key and the new values of the key. +// * org.apache.spark.Partitioner is used to control the partitioning of each RDD. +// * @param updateFunc State update function. If `this` function returns None, then +// * corresponding state key-value pair will be eliminated. +// * @param partitioner Partitioner for controlling the partitioning of each RDD in the new +// * DStream. +// * @param initialRDD initial state value of each key. +// * @tparam S State type +// */ +//@JvmName("updateStateByKeyArity2") +//fun JavaDStreamLike, *, *>.updateStateByKey( +// partitioner: Partitioner, +// initialRDD: JavaRDD>, +// updateFunc: (List, S?) -> S?, +//): JavaDStream> = +// mapToPair { it.toTuple() } +// .updateStateByKey( +// { list: List, s: Optional -> +// updateFunc(list, s.getOrNull()).asOptional() +// }, +// partitioner, +// initialRDD.mapToPair { it.toTuple() }, +// ) +// .map { it.toArity() } +// +///** +// * Return a new DStream by applying a map function to the value of each key-value pairs in +// * 'this' DStream without changing the key. +// */ +//@JvmName("mapValuesArity2") +//fun JavaDStreamLike, *, *>.mapValues( +// mapValuesFunc: (V) -> U, +//): JavaDStream> = +// mapToPair { it.toTuple() } +// .mapValues(mapValuesFunc) +// .map { it.toArity() } +// +///** +// * Return a new DStream by applying a flatmap function to the value of each key-value pairs in +// * 'this' DStream without changing the key. +// */ +//@JvmName("flatMapValuesArity2") +//fun JavaDStreamLike, *, *>.flatMapValues( +// flatMapValuesFunc: (V) -> Iterator, +//): JavaDStream> = +// mapToPair { it.toTuple() } +// .flatMapValues(flatMapValuesFunc) +// .map { it.toArity() } +// +///** +// * Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream. +// * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. +// */ +//@JvmName("cogroupArity2") +//fun JavaDStreamLike, *, *>.cogroup( +// other: JavaDStreamLike, *, *>, +// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +//): JavaDStream, Iterable>>> = +// mapToPair { it.toTuple() } +// .cogroup( +// other.mapToPair { it.toTuple() }, +// numPartitions, +// ) +// .map { +// c(it._1, it._2.toArity()) +// } +// +///** +// * Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream. +// * The supplied org.apache.spark.Partitioner is used to partition the generated RDDs. +// */ +//@JvmName("cogroupArity2") +//fun JavaDStreamLike, *, *>.cogroup( +// other: JavaDStreamLike, *, *>, +// partitioner: Partitioner, +//): JavaDStream, Iterable>>> = +// mapToPair { it.toTuple() } +// .cogroup( +// other.mapToPair { it.toTuple() }, +// partitioner, +// ) +// .map { +// c(it._1, it._2.toArity()) +// } +// +///** +// * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream. +// * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. +// */ +//@JvmName("joinArity2") +//fun JavaDStreamLike, *, *>.join( +// other: JavaDStreamLike, *, *>, +// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +//): JavaDStream>> = +// mapToPair { it.toTuple() } +// .join( +// other.mapToPair { it.toTuple() }, +// numPartitions, +// ) +// .map { +// c(it._1, it._2.toArity()) +// } +// +///** +// * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream. +// * The supplied org.apache.spark.Partitioner is used to control the partitioning of each RDD. +// */ +//@JvmName("joinArity2") +//fun JavaDStreamLike, *, *>.join( +// other: JavaDStreamLike, *, *>, +// partitioner: Partitioner, +//): JavaDStream>> = +// mapToPair { it.toTuple() } +// .join( +// other.mapToPair { it.toTuple() }, +// partitioner, +// ) +// .map { +// c(it._1, it._2.toArity()) +// } +// +///** +// * Return a new DStream by applying 'left outer join' between RDDs of `this` DStream and +// * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` +// * partitions. +// */ +//@JvmName("leftOuterJoinArity2") +//fun JavaDStreamLike, *, *>.leftOuterJoin( +// other: JavaDStreamLike, *, *>, +// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +//): JavaDStream>> = +// mapToPair { it.toTuple() } +// .leftOuterJoin( +// other.mapToPair { it.toTuple() }, +// numPartitions, +// ) +// .map { +// c(it._1, c(it._2._1, it._2._2.getOrNull())) +// } +// +///** +// * Return a new DStream by applying 'left outer join' between RDDs of `this` DStream and +// * `other` DStream. The supplied org.apache.spark.Partitioner is used to control +// * the partitioning of each RDD. +// */ +//@JvmName("leftOuterJoinArity2") +//fun JavaDStreamLike, *, *>.leftOuterJoin( +// other: JavaDStreamLike, *, *>, +// partitioner: Partitioner, +//): JavaDStream>> = +// mapToPair { it.toTuple() } +// .leftOuterJoin( +// other.mapToPair { it.toTuple() }, +// partitioner, +// ) +// .map { +// c(it._1, c(it._2._1, it._2._2.getOrNull())) +// } +// +///** +// * Return a new DStream by applying 'right outer join' between RDDs of `this` DStream and +// * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` +// * partitions. +// */ +//@JvmName("rightOuterJoinArity2") +//fun JavaDStreamLike, *, *>.rightOuterJoin( +// other: JavaDStreamLike, *, *>, +// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +//): JavaDStream>> = +// mapToPair { it.toTuple() } +// .rightOuterJoin( +// other.mapToPair { it.toTuple() }, +// numPartitions, +// ) +// .map { +// c(it._1, c(it._2._1.getOrNull(), it._2._2)) +// } +// +///** +// * Return a new DStream by applying 'right outer join' between RDDs of `this` DStream and +// * `other` DStream. The supplied org.apache.spark.Partitioner is used to control +// * the partitioning of each RDD. +// */ +//@JvmName("rightOuterJoinArity2") +//fun JavaDStreamLike, *, *>.rightOuterJoin( +// other: JavaDStreamLike, *, *>, +// partitioner: Partitioner, +//): JavaDStream>> = +// mapToPair { it.toTuple() } +// .rightOuterJoin( +// other.mapToPair { it.toTuple() }, +// partitioner, +// ) +// .map { +// c(it._1, c(it._2._1.getOrNull(), it._2._2)) +// } +// +///** +// * Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and +// * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` +// * partitions. +// */ +//@JvmName("fullOuterJoinArity2") +//fun JavaDStreamLike, *, *>.fullOuterJoin( +// other: JavaDStreamLike, *, *>, +// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +//): JavaDStream>> = +// mapToPair { it.toTuple() } +// .fullOuterJoin( +// other.mapToPair { it.toTuple() }, +// numPartitions, +// ) +// .map { +// c(it._1, c(it._2._1.getOrNull(), it._2._2.getOrNull())) +// } +// +///** +// * Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and +// * `other` DStream. The supplied org.apache.spark.Partitioner is used to control +// * the partitioning of each RDD. +// */ +//@JvmName("fullOuterJoinArity2") +//fun JavaDStreamLike, *, *>.fullOuterJoin( +// other: JavaDStreamLike, *, *>, +// partitioner: Partitioner, +//): JavaDStream>> = +// mapToPair { it.toTuple() } +// .fullOuterJoin( +// other.mapToPair { it.toTuple() }, +// partitioner, +// ) +// .map { +// c(it._1, c(it._2._1.getOrNull(), it._2._2.getOrNull())) +// } +// +///** +// * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is +// * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix". +// */ +//@JvmName("saveAsHadoopFilesArity2") +//fun JavaDStreamLike, *, *>.saveAsHadoopFiles( +// prefix: String, suffix: String, +//): Unit = +// mapToPair { it.toTuple() } +// .saveAsHadoopFiles(prefix, suffix) +// +///** +// * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is +// * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix". +// */ +//@JvmName("saveAsNewAPIHadoopFilesArity2") +//fun JavaDStreamLike, *, *>.saveAsNewAPIHadoopFiles( +// prefix: String, suffix: String, +//): Unit = +// mapToPair { it.toTuple() } +// .saveAsNewAPIHadoopFiles(prefix, suffix) +// +// +///** +// * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to +// * generate the RDDs with `numPartitions` partitions. +// */ +//@JvmName("groupByKeyPair") +//fun JavaDStreamLike, *, *>.groupByKey( +// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +//): JavaDStream>> = +// mapToPair { it.toTuple() } +// .groupByKey(numPartitions) +// .map { it.toPair() } +// +///** +// * Return a new DStream by applying `groupByKey` on each RDD. The supplied +// * org.apache.spark.Partitioner is used to control the partitioning of each RDD. +// */ +//@JvmName("groupByKeyPair") +//fun JavaDStreamLike, *, *>.groupByKey(partitioner: Partitioner): JavaDStream>> = +// mapToPair { it.toTuple() } +// .groupByKey(partitioner) +// .map { it.toPair() } +// +///** +// * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are +// * merged using the supplied reduce function. Hash partitioning is used to generate the RDDs +// * with `numPartitions` partitions. +// */ +//@JvmName("reduceByKeyPair") +//fun JavaDStreamLike, *, *>.reduceByKey( +// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +// reduceFunc: (V, V) -> V, +//): JavaDStream> = +// mapToPair { it.toTuple() } +// .reduceByKey(reduceFunc, numPartitions) +// .map { it.toPair() } +// +///** +// * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are +// * merged using the supplied reduce function. org.apache.spark.Partitioner is used to control +// * the partitioning of each RDD. +// */ +//@JvmName("reduceByKeyPair") +//fun JavaDStreamLike, *, *>.reduceByKey( +// partitioner: Partitioner, +// reduceFunc: (V, V) -> V, +//): JavaDStream> = +// mapToPair { it.toTuple() } +// .reduceByKey(reduceFunc, partitioner) +// .map { it.toPair() } +// +///** +// * Combine elements of each key in DStream's RDDs using custom functions. This is similar to the +// * combineByKey for RDDs. Please refer to combineByKey in +// * org.apache.spark.rdd.PairRDDFunctions in the Spark core documentation for more information. +// */ +//@JvmName("combineByKeyPair") +//fun JavaDStreamLike, *, *>.combineByKey( +// createCombiner: (V) -> C, +// mergeValue: (C, V) -> C, +// mergeCombiner: (C, C) -> C, +// partitioner: Partitioner, +// mapSideCombine: Boolean = true, +//): JavaDStream> = +// mapToPair { it.toTuple() } +// .combineByKey(createCombiner, mergeValue, mergeCombiner, partitioner, mapSideCombine) +// .map { it.toPair() } +// +///** +// * Return a new DStream by applying `groupByKey` over a sliding window on `this` DStream. +// * Similar to `DStream.groupByKey()`, but applies it over a sliding window. +// * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. +// * @param windowDuration width of the window; must be a multiple of this DStream's +// * batching interval +// * @param slideDuration sliding interval of the window (i.e., the interval after which +// * the new DStream will generate RDDs); must be a multiple of this +// * DStream's batching interval +// * @param numPartitions number of partitions of each RDD in the new DStream; if not specified +// * then Spark's default number of partitions will be used +// */ +//@JvmName("groupByKeyAndWindowPair") +//fun JavaDStreamLike, *, *>.groupByKeyAndWindow( +// windowDuration: Duration, +// slideDuration: Duration = dstream().slideDuration(), +// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +//): JavaDStream>> = +// mapToPair { it.toTuple() } +// .groupByKeyAndWindow(windowDuration, slideDuration, numPartitions) +// .map { it.toPair() } +// +///** +// * Create a new DStream by applying `groupByKey` over a sliding window on `this` DStream. +// * Similar to `DStream.groupByKey()`, but applies it over a sliding window. +// * @param windowDuration width of the window; must be a multiple of this DStream's +// * batching interval +// * @param slideDuration sliding interval of the window (i.e., the interval after which +// * the new DStream will generate RDDs); must be a multiple of this +// * DStream's batching interval +// * @param partitioner partitioner for controlling the partitioning of each RDD in the new +// * DStream. +// */ +//@JvmName("groupByKeyAndWindowPair") +//fun JavaDStreamLike, *, *>.groupByKeyAndWindow( +// windowDuration: Duration, +// slideDuration: Duration = dstream().slideDuration(), +// partitioner: Partitioner, +//): JavaDStream>> = +// mapToPair { it.toTuple() } +// .groupByKeyAndWindow(windowDuration, slideDuration, partitioner) +// .map { it.toPair() } +// +///** +// * Return a new DStream by applying `reduceByKey` over a sliding window. This is similar to +// * `DStream.reduceByKey()` but applies it over a sliding window. Hash partitioning is used to +// * generate the RDDs with `numPartitions` partitions. +// * @param reduceFunc associative and commutative reduce function +// * @param windowDuration width of the window; must be a multiple of this DStream's +// * batching interval +// * @param slideDuration sliding interval of the window (i.e., the interval after which +// * the new DStream will generate RDDs); must be a multiple of this +// * DStream's batching interval +// * @param numPartitions number of partitions of each RDD in the new DStream. +// */ +//@JvmName("reduceByKeyAndWindowPair") +//fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( +// windowDuration: Duration, +// slideDuration: Duration = dstream().slideDuration(), +// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +// reduceFunc: (V, V) -> V, +//): JavaDStream> = +// mapToPair { it.toTuple() } +// .reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, numPartitions) +// .map { it.toPair() } +// +///** +// * Return a new DStream by applying `reduceByKey` over a sliding window. Similar to +// * `DStream.reduceByKey()`, but applies it over a sliding window. +// * @param reduceFunc associative and commutative reduce function +// * @param windowDuration width of the window; must be a multiple of this DStream's +// * batching interval +// * @param slideDuration sliding interval of the window (i.e., the interval after which +// * the new DStream will generate RDDs); must be a multiple of this +// * DStream's batching interval +// * @param partitioner partitioner for controlling the partitioning of each RDD +// * in the new DStream. +// */ +//@JvmName("reduceByKeyAndWindowPair") +//fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( +// windowDuration: Duration, +// slideDuration: Duration = dstream().slideDuration(), +// partitioner: Partitioner, +// reduceFunc: (V, V) -> V, +//): JavaDStream> = +// mapToPair { it.toTuple() } +// .reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, partitioner) +// .map { it.toPair() } +// +///** +// * Return a new DStream by applying incremental `reduceByKey` over a sliding window. +// * The reduced value of over a new window is calculated using the old window's reduced value : +// * 1. reduce the new values that entered the window (e.g., adding new counts) +// * +// * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) +// * +// * This is more efficient than reduceByKeyAndWindow without "inverse reduce" function. +// * However, it is applicable to only "invertible reduce functions". +// * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. +// * @param reduceFunc associative and commutative reduce function +// * @param invReduceFunc inverse reduce function; such that for all y, invertible x: +// * `invReduceFunc(reduceFunc(x, y), x) = y` +// * @param windowDuration width of the window; must be a multiple of this DStream's +// * batching interval +// * @param slideDuration sliding interval of the window (i.e., the interval after which +// * the new DStream will generate RDDs); must be a multiple of this +// * DStream's batching interval +// * @param filterFunc Optional function to filter expired key-value pairs; +// * only pairs that satisfy the function are retained +// */ +//@JvmName("reduceByKeyAndWindowPair") +//fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( +// invReduceFunc: (V, V) -> V, +// windowDuration: Duration, +// slideDuration: Duration = dstream().slideDuration(), +// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +// filterFunc: ((Pair) -> Boolean)? = null, +// reduceFunc: (V, V) -> V, +//): JavaDStream> = +// mapToPair { it.toTuple() } +// .reduceByKeyAndWindow( +// reduceFunc, +// invReduceFunc, +// windowDuration, +// slideDuration, +// numPartitions, +// filterFunc?.let { +// { tuple -> +// filterFunc(tuple.toPair()) +// } +// } +// ) +// .map { it.toPair() } +// +///** +// * Return a new DStream by applying incremental `reduceByKey` over a sliding window. +// * The reduced value of over a new window is calculated using the old window's reduced value : +// * 1. reduce the new values that entered the window (e.g., adding new counts) +// * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) +// * This is more efficient than reduceByKeyAndWindow without "inverse reduce" function. +// * However, it is applicable to only "invertible reduce functions". +// * @param reduceFunc associative and commutative reduce function +// * @param invReduceFunc inverse reduce function +// * @param windowDuration width of the window; must be a multiple of this DStream's +// * batching interval +// * @param slideDuration sliding interval of the window (i.e., the interval after which +// * the new DStream will generate RDDs); must be a multiple of this +// * DStream's batching interval +// * @param partitioner partitioner for controlling the partitioning of each RDD in the new +// * DStream. +// * @param filterFunc Optional function to filter expired key-value pairs; +// * only pairs that satisfy the function are retained +// */ +//@JvmName("reduceByKeyAndWindowPair") +//fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( +// invReduceFunc: (V, V) -> V, +// windowDuration: Duration, +// slideDuration: Duration = dstream().slideDuration(), +// partitioner: Partitioner, +// filterFunc: ((Pair) -> Boolean)? = null, +// reduceFunc: (V, V) -> V, +//): JavaDStream> = +// mapToPair { it.toTuple() } +// .reduceByKeyAndWindow( +// reduceFunc, +// invReduceFunc, +// windowDuration, +// slideDuration, +// partitioner, +// filterFunc?.let { +// { tuple -> +// filterFunc(tuple.toPair()) +// } +// } +// ) +// .map { it.toPair() } +// +///** +// * Return a [MapWithStateDStream] by applying a function to every key-value element of +// * `this` stream, while maintaining some state data for each unique key. The mapping function +// * and other specification (e.g. partitioners, timeouts, initial state data, etc.) of this +// * transformation can be specified using `StateSpec` class. The state data is accessible in +// * as a parameter of type `State` in the mapping function. +// * +// * Example of using `mapWithState`: +// * {{{ +// * // A mapping function that maintains an integer state and return a String +// * def mappingFunction(key: String, value: Option[Int], state: State[Int]): Option[String] = { +// * // Use state.exists(), state.get(), state.update() and state.remove() +// * // to manage state, and return the necessary string +// * } +// * +// * val spec = StateSpec.function(mappingFunction).numPartitions(10) +// * +// * val mapWithStateDStream = keyValueDStream.mapWithState[StateType, MappedType](spec) +// * }}} +// * +// * @param spec Specification of this transformation +// * @tparam StateType Class type of the state data +// * @tparam MappedType Class type of the mapped data +// */ +//@JvmName("mapWithStatePair") +//fun JavaDStreamLike, *, *>.mapWithState( +// spec: StateSpec, +//): JavaMapWithStateDStream = +// mapToPair { it.toTuple() } +// .mapWithState(spec) +// +///** +// * Return a new "state" DStream where the state for each key is updated by applying +// * the given function on the previous state of the key and the new values of each key. +// * In every batch the updateFunc will be called for each state even if there are no new values. +// * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. +// * @param updateFunc State update function. If `this` function returns None, then +// * corresponding state key-value pair will be eliminated. +// * @tparam S State type +// */ +//@JvmName("updateStateByKeyPair") +//fun JavaDStreamLike, *, *>.updateStateByKey( +// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +// updateFunc: (List, S?) -> S?, +//): JavaDStream> = +// mapToPair { it.toTuple() } +// .updateStateByKey( +// { list: List, s: Optional -> +// updateFunc(list, s.getOrNull()).asOptional() +// }, +// numPartitions, +// ) +// .map { it.toPair() } +// +///** +// * Return a new "state" DStream where the state for each key is updated by applying +// * the given function on the previous state of the key and the new values of each key. +// * In every batch the updateFunc will be called for each state even if there are no new values. +// * [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD. +// * @param updateFunc State update function. Note, that this function may generate a different +// * tuple with a different key than the input key. Therefore keys may be removed +// * or added in this way. It is up to the developer to decide whether to +// * remember the partitioner despite the key being changed. +// * @param partitioner Partitioner for controlling the partitioning of each RDD in the new +// * DStream +// * @tparam S State type +// */ +//@JvmName("updateStateByKeyPair") +//fun JavaDStreamLike, *, *>.updateStateByKey( +// partitioner: Partitioner, +// updateFunc: (List, S?) -> S?, +//): JavaDStream> = +// mapToPair { it.toTuple() } +// .updateStateByKey( +// { list: List, s: Optional -> +// updateFunc(list, s.getOrNull()).asOptional() +// }, +// partitioner, +// ) +// .map { it.toPair() } +// +///** +// * Return a new "state" DStream where the state for each key is updated by applying +// * the given function on the previous state of the key and the new values of the key. +// * org.apache.spark.Partitioner is used to control the partitioning of each RDD. +// * @param updateFunc State update function. If `this` function returns None, then +// * corresponding state key-value pair will be eliminated. +// * @param partitioner Partitioner for controlling the partitioning of each RDD in the new +// * DStream. +// * @param initialRDD initial state value of each key. +// * @tparam S State type +// */ +//@JvmName("updateStateByKeyPair") +//fun JavaDStreamLike, *, *>.updateStateByKey( +// partitioner: Partitioner, +// initialRDD: JavaRDD>, +// updateFunc: (List, S?) -> S?, +//): JavaDStream> = +// mapToPair { it.toTuple() } +// .updateStateByKey( +// { list: List, s: Optional -> +// updateFunc(list, s.getOrNull()).asOptional() +// }, +// partitioner, +// initialRDD.mapToPair { it.toTuple() }, +// ) +// .map { it.toPair() } +// +///** +// * Return a new DStream by applying a map function to the value of each key-value pairs in +// * 'this' DStream without changing the key. +// */ +//@JvmName("mapValuesPair") +//fun JavaDStreamLike, *, *>.mapValues( +// mapValuesFunc: (V) -> U, +//): JavaDStream> = +// mapToPair { it.toTuple() } +// .mapValues(mapValuesFunc) +// .map { it.toPair() } +// +///** +// * Return a new DStream by applying a flatmap function to the value of each key-value pairs in +// * 'this' DStream without changing the key. +// */ +//@JvmName("flatMapValuesPair") +//fun JavaDStreamLike, *, *>.flatMapValues( +// flatMapValuesFunc: (V) -> Iterator, +//): JavaDStream> = +// mapToPair { it.toTuple() } +// .flatMapValues(flatMapValuesFunc) +// .map { it.toPair() } +// +///** +// * Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream. +// * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. +// */ +//@JvmName("cogroupPair") +//fun JavaDStreamLike, *, *>.cogroup( +// other: JavaDStreamLike, *, *>, +// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +//): JavaDStream, Iterable>>> = +// mapToPair { it.toTuple() } +// .cogroup( +// other.mapToPair { it.toTuple() }, +// numPartitions, +// ) +// .map { +// Pair(it._1, it._2.toPair()) +// } +// +///** +// * Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream. +// * The supplied org.apache.spark.Partitioner is used to partition the generated RDDs. +// */ +//@JvmName("cogroupPair") +//fun JavaDStreamLike, *, *>.cogroup( +// other: JavaDStreamLike, *, *>, +// partitioner: Partitioner, +//): JavaDStream, Iterable>>> = +// mapToPair { it.toTuple() } +// .cogroup( +// other.mapToPair { it.toTuple() }, +// partitioner, +// ) +// .map { +// Pair(it._1, it._2.toPair()) +// } +// +///** +// * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream. +// * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. +// */ +//@JvmName("joinPair") +//fun JavaDStreamLike, *, *>.join( +// other: JavaDStreamLike, *, *>, +// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +//): JavaDStream>> = +// mapToPair { it.toTuple() } +// .join( +// other.mapToPair { it.toTuple() }, +// numPartitions, +// ) +// .map { +// Pair(it._1, it._2.toPair()) +// } +// +///** +// * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream. +// * The supplied org.apache.spark.Partitioner is used to control the partitioning of each RDD. +// */ +//@JvmName("joinPair") +//fun JavaDStreamLike, *, *>.join( +// other: JavaDStreamLike, *, *>, +// partitioner: Partitioner, +//): JavaDStream>> = +// mapToPair { it.toTuple() } +// .join( +// other.mapToPair { it.toTuple() }, +// partitioner, +// ) +// .map { +// Pair(it._1, it._2.toPair()) +// } +// +///** +// * Return a new DStream by applying 'left outer join' between RDDs of `this` DStream and +// * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` +// * partitions. +// */ +//@JvmName("leftOuterJoinPair") +//fun JavaDStreamLike, *, *>.leftOuterJoin( +// other: JavaDStreamLike, *, *>, +// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +//): JavaDStream>> = +// mapToPair { it.toTuple() } +// .leftOuterJoin( +// other.mapToPair { it.toTuple() }, +// numPartitions, +// ) +// .map { +// Pair(it._1, Pair(it._2._1, it._2._2.getOrNull())) +// } +// +///** +// * Return a new DStream by applying 'left outer join' between RDDs of `this` DStream and +// * `other` DStream. The supplied org.apache.spark.Partitioner is used to control +// * the partitioning of each RDD. +// */ +//@JvmName("leftOuterJoinPair") +//fun JavaDStreamLike, *, *>.leftOuterJoin( +// other: JavaDStreamLike, *, *>, +// partitioner: Partitioner, +//): JavaDStream>> = +// mapToPair { it.toTuple() } +// .leftOuterJoin( +// other.mapToPair { it.toTuple() }, +// partitioner, +// ) +// .map { +// Pair(it._1, Pair(it._2._1, it._2._2.getOrNull())) +// } +// +///** +// * Return a new DStream by applying 'right outer join' between RDDs of `this` DStream and +// * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` +// * partitions. +// */ +//@JvmName("rightOuterJoinPair") +//fun JavaDStreamLike, *, *>.rightOuterJoin( +// other: JavaDStreamLike, *, *>, +// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +//): JavaDStream>> = +// mapToPair { it.toTuple() } +// .rightOuterJoin( +// other.mapToPair { it.toTuple() }, +// numPartitions, +// ) +// .map { +// Pair(it._1, Pair(it._2._1.getOrNull(), it._2._2)) +// } +// +///** +// * Return a new DStream by applying 'right outer join' between RDDs of `this` DStream and +// * `other` DStream. The supplied org.apache.spark.Partitioner is used to control +// * the partitioning of each RDD. +// */ +//@JvmName("rightOuterJoinPair") +//fun JavaDStreamLike, *, *>.rightOuterJoin( +// other: JavaDStreamLike, *, *>, +// partitioner: Partitioner, +//): JavaDStream>> = +// mapToPair { it.toTuple() } +// .rightOuterJoin( +// other.mapToPair { it.toTuple() }, +// partitioner, +// ) +// .map { +// Pair(it._1, Pair(it._2._1.getOrNull(), it._2._2)) +// } +// +///** +// * Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and +// * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` +// * partitions. +// */ +//@JvmName("fullOuterJoinPair") +//fun JavaDStreamLike, *, *>.fullOuterJoin( +// other: JavaDStreamLike, *, *>, +// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +//): JavaDStream>> = +// mapToPair { it.toTuple() } +// .fullOuterJoin( +// other.mapToPair { it.toTuple() }, +// numPartitions, +// ) +// .map { +// Pair(it._1, Pair(it._2._1.getOrNull(), it._2._2.getOrNull())) +// } +// +///** +// * Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and +// * `other` DStream. The supplied org.apache.spark.Partitioner is used to control +// * the partitioning of each RDD. +// */ +//@JvmName("fullOuterJoinPair") +//fun JavaDStreamLike, *, *>.fullOuterJoin( +// other: JavaDStreamLike, *, *>, +// partitioner: Partitioner, +//): JavaDStream>> = +// mapToPair { it.toTuple() } +// .fullOuterJoin( +// other.mapToPair { it.toTuple() }, +// partitioner, +// ) +// .map { +// Pair(it._1, Pair(it._2._1.getOrNull(), it._2._2.getOrNull())) +// } +// +///** +// * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is +// * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix". +// */ +//@JvmName("saveAsHadoopFilesPair") +//fun JavaDStreamLike, *, *>.saveAsHadoopFiles( +// prefix: String, suffix: String, +//): Unit = +// mapToPair { it.toTuple() } +// .saveAsHadoopFiles(prefix, suffix) +// +///** +// * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is +// * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix". +// */ +//@JvmName("saveAsNewAPIHadoopFilesPair") +//fun JavaDStreamLike, *, *>.saveAsNewAPIHadoopFiles( +// prefix: String, suffix: String, +//): Unit = +// mapToPair { it.toTuple() } +// .saveAsNewAPIHadoopFiles(prefix, suffix) /** @@ -1231,22 +1222,22 @@ fun JavaDStreamLike, *, *>.saveAsNewAPIHadoopFiles( * generate the RDDs with `numPartitions` partitions. */ @JvmName("groupByKeyTuple2") -fun JavaDStreamLike, *, *>.groupByKey( +fun JavaDStream>.groupByKey( numPartitions: Int = dstream().ssc().sc().defaultParallelism(), ): JavaDStream>> = - mapToPair { it } + toPairDStream() .groupByKey(numPartitions) - .map { it } + .toJavaDStream() /** * Return a new DStream by applying `groupByKey` on each RDD. The supplied * org.apache.spark.Partitioner is used to control the partitioning of each RDD. */ @JvmName("groupByKeyTuple2") -fun JavaDStreamLike, *, *>.groupByKey(partitioner: Partitioner): JavaDStream>> = - mapToPair { it } +fun JavaDStream>.groupByKey(partitioner: Partitioner): JavaDStream>> = + toPairDStream() .groupByKey(partitioner) - .map { it } + .toJavaDStream() /** * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are @@ -1254,13 +1245,13 @@ fun JavaDStreamLike, *, *>.groupByKey(partitioner: Partition * with `numPartitions` partitions. */ @JvmName("reduceByKeyTuple2") -fun JavaDStreamLike, *, *>.reduceByKey( +fun JavaDStream>.reduceByKey( numPartitions: Int = dstream().ssc().sc().defaultParallelism(), reduceFunc: (V, V) -> V, ): JavaDStream> = - mapToPair { it } + toPairDStream() .reduceByKey(reduceFunc, numPartitions) - .map { it } + .toJavaDStream() /** * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are @@ -1268,13 +1259,13 @@ fun JavaDStreamLike, *, *>.reduceByKey( * the partitioning of each RDD. */ @JvmName("reduceByKeyTuple2") -fun JavaDStreamLike, *, *>.reduceByKey( +fun JavaDStream>.reduceByKey( partitioner: Partitioner, reduceFunc: (V, V) -> V, ): JavaDStream> = - mapToPair { it } + toPairDStream() .reduceByKey(reduceFunc, partitioner) - .map { it } + .toJavaDStream() /** * Combine elements of each key in DStream's RDDs using custom functions. This is similar to the @@ -1282,16 +1273,16 @@ fun JavaDStreamLike, *, *>.reduceByKey( * org.apache.spark.rdd.PairRDDFunctions in the Spark core documentation for more information. */ @JvmName("combineByKeyTuple2") -fun JavaDStreamLike, *, *>.combineByKey( +fun JavaDStream>.combineByKey( createCombiner: (V) -> C, mergeValue: (C, V) -> C, mergeCombiner: (C, C) -> C, partitioner: Partitioner, mapSideCombine: Boolean = true, ): JavaDStream> = - mapToPair { it } + toPairDStream() .combineByKey(createCombiner, mergeValue, mergeCombiner, partitioner, mapSideCombine) - .map { it } + .toJavaDStream() /** * Return a new DStream by applying `groupByKey` over a sliding window on `this` DStream. @@ -1306,14 +1297,14 @@ fun JavaDStreamLike, *, *>.combineByKey( * then Spark's default number of partitions will be used */ @JvmName("groupByKeyAndWindowTuple2") -fun JavaDStreamLike, *, *>.groupByKeyAndWindow( +fun JavaDStream>.groupByKeyAndWindow( windowDuration: Duration, slideDuration: Duration = dstream().slideDuration(), numPartitions: Int = dstream().ssc().sc().defaultParallelism(), ): JavaDStream>> = - mapToPair { it } + toPairDStream() .groupByKeyAndWindow(windowDuration, slideDuration, numPartitions) - .map { it } + .toJavaDStream() /** * Create a new DStream by applying `groupByKey` over a sliding window on `this` DStream. @@ -1327,14 +1318,14 @@ fun JavaDStreamLike, *, *>.groupByKeyAndWindow( * DStream. */ @JvmName("groupByKeyAndWindowTuple2") -fun JavaDStreamLike, *, *>.groupByKeyAndWindow( +fun JavaDStream>.groupByKeyAndWindow( windowDuration: Duration, slideDuration: Duration = dstream().slideDuration(), partitioner: Partitioner, ): JavaDStream>> = - mapToPair { it } + toPairDStream() .groupByKeyAndWindow(windowDuration, slideDuration, partitioner) - .map { it } + .toJavaDStream() /** * Return a new DStream by applying `reduceByKey` over a sliding window. This is similar to @@ -1349,15 +1340,15 @@ fun JavaDStreamLike, *, *>.groupByKeyAndWindow( * @param numPartitions number of partitions of each RDD in the new DStream. */ @JvmName("reduceByKeyAndWindowTuple2") -fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( +fun JavaDStream>.reduceByKeyAndWindow( windowDuration: Duration, slideDuration: Duration = dstream().slideDuration(), numPartitions: Int = dstream().ssc().sc().defaultParallelism(), reduceFunc: (V, V) -> V, ): JavaDStream> = - mapToPair { it } + toPairDStream() .reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, numPartitions) - .map { it } + .toJavaDStream() /** * Return a new DStream by applying `reduceByKey` over a sliding window. Similar to @@ -1372,15 +1363,15 @@ fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( * in the new DStream. */ @JvmName("reduceByKeyAndWindowTuple2") -fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( +fun JavaDStream>.reduceByKeyAndWindow( windowDuration: Duration, slideDuration: Duration = dstream().slideDuration(), partitioner: Partitioner, reduceFunc: (V, V) -> V, ): JavaDStream> = - mapToPair { it } + toPairDStream() .reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, partitioner) - .map { it } + .toJavaDStream() /** * Return a new DStream by applying incremental `reduceByKey` over a sliding window. @@ -1404,7 +1395,7 @@ fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( * only pairs that satisfy the function are retained */ @JvmName("reduceByKeyAndWindowTuple2") -fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( +fun JavaDStream>.reduceByKeyAndWindow( invReduceFunc: (V, V) -> V, windowDuration: Duration, slideDuration: Duration = dstream().slideDuration(), @@ -1412,20 +1403,20 @@ fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( filterFunc: ((Tuple2) -> Boolean)? = null, reduceFunc: (V, V) -> V, ): JavaDStream> = - mapToPair { it } + toPairDStream() .reduceByKeyAndWindow( - reduceFunc, - invReduceFunc, - windowDuration, - slideDuration, - numPartitions, - filterFunc?.let { + /* reduceFunc = */ reduceFunc, + /* invReduceFunc = */ invReduceFunc, + /* windowDuration = */ windowDuration, + /* slideDuration = */ slideDuration, + /* numPartitions = */ numPartitions, + /* filterFunc = */ filterFunc?.let { { tuple -> filterFunc(tuple) } } ) - .map { it } + .toJavaDStream() /** * Return a new DStream by applying incremental `reduceByKey` over a sliding window. @@ -1447,7 +1438,7 @@ fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( * only pairs that satisfy the function are retained */ @JvmName("reduceByKeyAndWindowTuple2") -fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( +fun JavaDStream>.reduceByKeyAndWindow( invReduceFunc: (V, V) -> V, windowDuration: Duration, slideDuration: Duration = dstream().slideDuration(), @@ -1455,20 +1446,20 @@ fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( filterFunc: ((Tuple2) -> Boolean)? = null, reduceFunc: (V, V) -> V, ): JavaDStream> = - mapToPair { it } + toPairDStream() .reduceByKeyAndWindow( - reduceFunc, - invReduceFunc, - windowDuration, - slideDuration, - partitioner, - filterFunc?.let { + /* reduceFunc = */ reduceFunc, + /* invReduceFunc = */ invReduceFunc, + /* windowDuration = */ windowDuration, + /* slideDuration = */ slideDuration, + /* partitioner = */ partitioner, + /* filterFunc = */ filterFunc?.let { { tuple -> filterFunc(tuple) } } ) - .map { it } + .toJavaDStream() /** * Return a [MapWithStateDStream] by applying a function to every key-value element of @@ -1495,34 +1486,33 @@ fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( * @tparam MappedType Class type of the mapped data */ @JvmName("mapWithStateTuple2") -fun JavaDStreamLike, *, *>.mapWithState( +fun JavaDStream>.mapWithState( spec: StateSpec, ): JavaMapWithStateDStream = - mapToPair { it } - .mapWithState(spec) + toPairDStream().mapWithState(spec) /** * Return a new "state" DStream where the state for each key is updated by applying * the given function on the previous state of the key and the new values of each key. * In every batch the updateFunc will be called for each state even if there are no new values. * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. - * @param updateFunc State update function. If `this` function returns None, then + * @param updateFunc State update function. If `this` function returns `null`, then * corresponding state key-value pair will be eliminated. * @tparam S State type */ @JvmName("updateStateByKeyTuple2") -fun JavaDStreamLike, *, *>.updateStateByKey( +fun JavaDStream>.updateStateByKey( numPartitions: Int = dstream().ssc().sc().defaultParallelism(), updateFunc: (List, S?) -> S?, ): JavaDStream> = - mapToPair { it } + toPairDStream() .updateStateByKey( { list: List, s: Optional -> - updateFunc(list, s.toNullable()).toOptional() + updateFunc(list, s.getOrNull()).asOptional() }, numPartitions, ) - .map { it } + .toJavaDStream() /** * Return a new "state" DStream where the state for each key is updated by applying @@ -1538,24 +1528,24 @@ fun JavaDStreamLike, *, *>.updateStateByKey( * @tparam S State type */ @JvmName("updateStateByKeyTuple2") -fun JavaDStreamLike, *, *>.updateStateByKey( +fun JavaDStream>.updateStateByKey( partitioner: Partitioner, updateFunc: (List, S?) -> S?, ): JavaDStream> = - mapToPair { it } + toPairDStream() .updateStateByKey( { list: List, s: Optional -> - updateFunc(list, s.toNullable()).toOptional() + updateFunc(list, s.getOrNull()).asOptional() }, partitioner, ) - .map { it } + .toJavaDStream() /** * Return a new "state" DStream where the state for each key is updated by applying * the given function on the previous state of the key and the new values of the key. * org.apache.spark.Partitioner is used to control the partitioning of each RDD. - * @param updateFunc State update function. If `this` function returns None, then + * @param updateFunc State update function. If `this` function returns `null`, then * corresponding state key-value pair will be eliminated. * @param partitioner Partitioner for controlling the partitioning of each RDD in the new * DStream. @@ -1563,116 +1553,110 @@ fun JavaDStreamLike, *, *>.updateStateByKey( * @tparam S State type */ @JvmName("updateStateByKeyTuple2") -fun JavaDStreamLike, *, *>.updateStateByKey( +fun JavaDStream>.updateStateByKey( partitioner: Partitioner, initialRDD: JavaRDD>, updateFunc: (List, S?) -> S?, ): JavaDStream> = - mapToPair { it } + toPairDStream() .updateStateByKey( { list: List, s: Optional -> - updateFunc(list, s.toNullable()).toOptional() + updateFunc(list, s.getOrNull()).asOptional() }, partitioner, - initialRDD.mapToPair { it }, + initialRDD.toPairRDD(), ) - .map { it } + .toJavaDStream() + /** * Return a new DStream by applying a map function to the value of each key-value pairs in * 'this' DStream without changing the key. */ @JvmName("mapValuesTuple2") -fun JavaDStreamLike, *, *>.mapValues( +fun JavaDStream>.mapValues( mapValuesFunc: (V) -> U, ): JavaDStream> = - mapToPair { it } + toPairDStream() .mapValues(mapValuesFunc) - .map { it } + .toJavaDStream() /** * Return a new DStream by applying a flatmap function to the value of each key-value pairs in * 'this' DStream without changing the key. */ @JvmName("flatMapValuesTuple2") -fun JavaDStreamLike, *, *>.flatMapValues( +fun JavaDStream>.flatMapValues( flatMapValuesFunc: (V) -> Iterator, ): JavaDStream> = - mapToPair { it } + toPairDStream() .flatMapValues(flatMapValuesFunc) - .map { it } + .toJavaDStream() /** * Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream. * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. */ @JvmName("cogroupTuple2") -fun JavaDStreamLike, *, *>.cogroup( - other: JavaDStreamLike, *, *>, +fun JavaDStream>.cogroup( + other: JavaDStream>, numPartitions: Int = dstream().ssc().sc().defaultParallelism(), ): JavaDStream, Iterable>>> = - mapToPair { it } + toPairDStream() .cogroup( - other.mapToPair { it }, + other.toPairDStream(), numPartitions, ) - .map { - Tuple2(it._1, it._2) - } + .toJavaDStream() + /** * Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream. * The supplied org.apache.spark.Partitioner is used to partition the generated RDDs. */ @JvmName("cogroupTuple2") -fun JavaDStreamLike, *, *>.cogroup( - other: JavaDStreamLike, *, *>, +fun JavaDStream>.cogroup( + other: JavaDStream>, partitioner: Partitioner, ): JavaDStream, Iterable>>> = - mapToPair { it } + toPairDStream() .cogroup( - other.mapToPair { it }, + other.toPairDStream(), partitioner, ) - .map { - Tuple2(it._1, it._2) - } + .toJavaDStream() /** * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream. * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. */ @JvmName("joinTuple2") -fun JavaDStreamLike, *, *>.join( - other: JavaDStreamLike, *, *>, +fun JavaDStream>.join( + other: JavaDStream>, numPartitions: Int = dstream().ssc().sc().defaultParallelism(), ): JavaDStream>> = - mapToPair { it } + toPairDStream() .join( - other.mapToPair { it }, + other.toPairDStream(), numPartitions, ) - .map { - Tuple2(it._1, it._2) - } + .toJavaDStream() /** * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream. * The supplied org.apache.spark.Partitioner is used to control the partitioning of each RDD. */ @JvmName("joinTuple2") -fun JavaDStreamLike, *, *>.join( - other: JavaDStreamLike, *, *>, +fun JavaDStream>.join( + other: JavaDStream>, partitioner: Partitioner, ): JavaDStream>> = - mapToPair { it } + toPairDStream() .join( - other.mapToPair { it }, + other.toPairDStream(), partitioner, ) - .map { - Tuple2(it._1, it._2) - } + .toJavaDStream() /** * Return a new DStream by applying 'left outer join' between RDDs of `this` DStream and @@ -1680,18 +1664,16 @@ fun JavaDStreamLike, *, *>.join( * partitions. */ @JvmName("leftOuterJoinTuple2") -fun JavaDStreamLike, *, *>.leftOuterJoin( - other: JavaDStreamLike, *, *>, +fun JavaDStream>.leftOuterJoin( + other: JavaDStream>, numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -): JavaDStream>> = - mapToPair { it } +): JavaDStream>>> = + toPairDStream() .leftOuterJoin( - other.mapToPair { it }, + other.toPairDStream(), numPartitions, ) - .map { - Tuple2(it._1, Tuple2(it._2._1, it._2._2.toNullable())) - } + .toJavaDStream() /** * Return a new DStream by applying 'left outer join' between RDDs of `this` DStream and @@ -1699,18 +1681,16 @@ fun JavaDStreamLike, *, *>.leftOuterJoin( * the partitioning of each RDD. */ @JvmName("leftOuterJoinTuple2") -fun JavaDStreamLike, *, *>.leftOuterJoin( - other: JavaDStreamLike, *, *>, +fun JavaDStream>.leftOuterJoin( + other: JavaDStream>, partitioner: Partitioner, -): JavaDStream>> = - mapToPair { it } +): JavaDStream>>> = + toPairDStream() .leftOuterJoin( - other.mapToPair { it }, + other.toPairDStream(), partitioner, ) - .map { - Tuple2(it._1, Tuple2(it._2._1, it._2._2.toNullable())) - } + .toJavaDStream() /** * Return a new DStream by applying 'right outer join' between RDDs of `this` DStream and @@ -1718,18 +1698,16 @@ fun JavaDStreamLike, *, *>.leftOuterJoin( * partitions. */ @JvmName("rightOuterJoinTuple2") -fun JavaDStreamLike, *, *>.rightOuterJoin( - other: JavaDStreamLike, *, *>, +fun JavaDStream>.rightOuterJoin( + other: JavaDStream>, numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -): JavaDStream>> = - mapToPair { it } +): JavaDStream, W>>> = + toPairDStream() .rightOuterJoin( - other.mapToPair { it }, + other.toPairDStream(), numPartitions, ) - .map { - Tuple2(it._1, Tuple2(it._2._1.toNullable(), it._2._2)) - } + .toJavaDStream() /** * Return a new DStream by applying 'right outer join' between RDDs of `this` DStream and @@ -1737,18 +1715,16 @@ fun JavaDStreamLike, *, *>.rightOuterJoin( * the partitioning of each RDD. */ @JvmName("rightOuterJoinTuple2") -fun JavaDStreamLike, *, *>.rightOuterJoin( - other: JavaDStreamLike, *, *>, +fun JavaDStream>.rightOuterJoin( + other: JavaDStream>, partitioner: Partitioner, -): JavaDStream>> = - mapToPair { it } +): JavaDStream, W>>> = + toPairDStream() .rightOuterJoin( - other.mapToPair { it }, + other.toPairDStream(), partitioner, ) - .map { - Tuple2(it._1, Tuple2(it._2._1.toNullable(), it._2._2)) - } + .toJavaDStream() /** * Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and @@ -1756,18 +1732,16 @@ fun JavaDStreamLike, *, *>.rightOuterJoin( * partitions. */ @JvmName("fullOuterJoinTuple2") -fun JavaDStreamLike, *, *>.fullOuterJoin( - other: JavaDStreamLike, *, *>, +fun JavaDStream>.fullOuterJoin( + other: JavaDStream>, numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -): JavaDStream>> = - mapToPair { it } +): JavaDStream, Optional>>> = + toPairDStream() .fullOuterJoin( - other.mapToPair { it }, + other.toPairDStream(), numPartitions, ) - .map { - Tuple2(it._1, Tuple2(it._2._1.toNullable(), it._2._2.toNullable())) - } + .toJavaDStream() /** * Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and @@ -1775,37 +1749,33 @@ fun JavaDStreamLike, *, *>.fullOuterJoin( * the partitioning of each RDD. */ @JvmName("fullOuterJoinTuple2") -fun JavaDStreamLike, *, *>.fullOuterJoin( - other: JavaDStreamLike, *, *>, +fun JavaDStream>.fullOuterJoin( + other: JavaDStream>, partitioner: Partitioner, -): JavaDStream>> = - mapToPair { it } +): JavaDStream, Optional>>> = + toPairDStream() .fullOuterJoin( - other.mapToPair { it }, + other.toPairDStream(), partitioner, ) - .map { - Tuple2(it._1, Tuple2(it._2._1.toNullable(), it._2._2.toNullable())) - } + .toJavaDStream() /** * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix". */ @JvmName("saveAsHadoopFilesTuple2") -fun JavaDStreamLike, *, *>.saveAsHadoopFiles( - prefix: String, suffix: String, -): Unit = - mapToPair { it } - .saveAsHadoopFiles(prefix, suffix) +fun JavaDStream>.saveAsHadoopFiles( + prefix: String, + suffix: String, +): Unit = toPairDStream().saveAsHadoopFiles(prefix, suffix) /** * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix". */ @JvmName("saveAsNewAPIHadoopFilesTuple2") -fun JavaDStreamLike, *, *>.saveAsNewAPIHadoopFiles( - prefix: String, suffix: String, -): Unit = - mapToPair { it } - .saveAsNewAPIHadoopFiles(prefix, suffix) +fun JavaDStream>.saveAsNewAPIHadoopFiles( + prefix: String, + suffix: String, +): Unit = toPairDStream().saveAsNewAPIHadoopFiles(prefix, suffix) From ea3e7d89858fff090bfcd3f802460ca51a832782 Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Thu, 31 Mar 2022 22:41:37 +0200 Subject: [PATCH 16/44] removed commented out code, fixed checkpointPath (needs cleaning up code), new notation for entering KSparkSession scope inside streaming scope --- ...t => KotlinRecoverableNetworkWordCount.kt} | 148 +-- .../kotlinx/spark/examples/Streaming.kt | 13 +- .../kotlinx/spark/api/SparkSession.kt | 126 +- .../jetbrains/kotlinx/spark/api/Streaming.kt | 1170 ----------------- 4 files changed, 142 insertions(+), 1315 deletions(-) rename examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/{JavaRecoverableNetworkWordCount.kt => KotlinRecoverableNetworkWordCount.kt} (60%) diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/JavaRecoverableNetworkWordCount.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/KotlinRecoverableNetworkWordCount.kt similarity index 60% rename from examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/JavaRecoverableNetworkWordCount.kt rename to examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/KotlinRecoverableNetworkWordCount.kt index 9b64149e..10f57a28 100644 --- a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/JavaRecoverableNetworkWordCount.kt +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/KotlinRecoverableNetworkWordCount.kt @@ -22,21 +22,13 @@ package org.jetbrains.kotlinx.spark.examples import com.google.common.io.Files -import org.apache.spark.api.java.JavaPairRDD import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.broadcast.Broadcast -import org.apache.spark.streaming.Duration import org.apache.spark.streaming.Durations import org.apache.spark.streaming.Time -import org.apache.spark.streaming.api.java.JavaDStream -import org.apache.spark.streaming.api.java.JavaPairDStream -import org.apache.spark.streaming.dstream.DStream -import org.apache.spark.streaming.dstream.PairDStreamFunctions import org.apache.spark.util.LongAccumulator import org.jetbrains.kotlinx.spark.api.* -import scala.Tuple1 import scala.Tuple2 -import scala.reflect.ClassTag import java.io.File import java.nio.charset.Charset import java.util.regex.Pattern @@ -49,13 +41,13 @@ import kotlin.time.measureTimedValue /** * Use this singleton to get or register a Broadcast variable. */ -internal object JavaWordExcludeList { +internal object KotlinWordExcludeList { @Volatile private var instance: Broadcast>? = null fun getInstance(sc: JavaSparkContext): Broadcast> { - if (instance == null) synchronized(JavaWordExcludeList::class.java) { + if (instance == null) synchronized(KotlinWordExcludeList::class) { if (instance == null) { val wordExcludeList = listOf("a", "b", "c") instance = sc.broadcast(wordExcludeList) @@ -68,13 +60,13 @@ internal object JavaWordExcludeList { /** * Use this singleton to get or register an Accumulator. */ -internal object JavaDroppedWordsCounter { +internal object KotlinDroppedWordsCounter { @Volatile private var instance: LongAccumulator? = null fun getInstance(sc: JavaSparkContext): LongAccumulator { - if (instance == null) synchronized(JavaDroppedWordsCounter::class.java) { + if (instance == null) synchronized(KotlinDroppedWordsCounter::class) { if (instance == null) instance = sc.sc().longAccumulator("DroppedWordsCounter") } @@ -87,7 +79,7 @@ internal object JavaDroppedWordsCounter { * shows how to use lazily instantiated singleton instances for Accumulator and Broadcast so that * they can be registered on driver failures. * - * Usage: JavaRecoverableNetworkWordCount + * Usage: KotlinRecoverableNetworkWordCount * and describe the TCP server that Spark Streaming would connect to receive * data. directory to HDFS-compatible file system which checkpoint data * file to which the word counts will be appended @@ -100,7 +92,7 @@ internal object JavaDroppedWordsCounter { * * and run the example as * - * `$ ./bin/run-example org.apache.spark.examples.streaming.JavaRecoverableNetworkWordCount \ + * `$ ./bin/run-example org.apache.spark.examples.streaming.KotlinRecoverableNetworkWordCount \ * localhost 9999 ~/checkpoint/ ~/out` * * If the directory ~/checkpoint/ does not exist (e.g. running for the first time), it will create @@ -110,7 +102,7 @@ internal object JavaDroppedWordsCounter { * * Refer to the online documentation for more details. */ -object JavaRecoverableNetworkWordCount { +object KotlinRecoverableNetworkWordCount { private val SPACE = Pattern.compile(" ") @@ -125,7 +117,7 @@ object JavaRecoverableNetworkWordCount { if (args.size != 4 && args.isNotEmpty()) { System.err.println("You arguments were " + listOf(*args)) System.err.println( - """Usage: JavaRecoverableNetworkWordCount + """Usage: KotlinRecoverableNetworkWordCount . and describe the TCP server that Spark Streaming would connect to receive data. directory to HDFS-compatible file system which checkpoint data file to which @@ -141,12 +133,13 @@ object JavaRecoverableNetworkWordCount { val checkpointDirectory = args.getOrElse(2) { DEFAULT_CHECKPOINT_DIRECTORY } val outputPath = args.getOrElse(3) { DEFAULT_OUTPUT_PATH } + // (used to detect the new context) // Create the context with a 1 second batch size or load from checkpointDirectory withSparkStreaming( -// checkpointPath = checkpointDirectory, TODO + checkpointPath = checkpointDirectory, batchDuration = Durations.seconds(1), - appName = "JavaRecoverableNetworkWordCount", + appName = "KotlinRecoverableNetworkWordCount", ) { createContext( ip = ip, @@ -167,8 +160,12 @@ object JavaRecoverableNetworkWordCount { // If you do not see this printed, that means the StreamingContext has been loaded // from the new checkpoint println("Creating new context") - val outputFile = File(outputPath) - if (outputFile.exists()) outputFile.delete() + val outputFile = File(outputPath).apply { + if (exists()) delete() + parentFile.mkdirs() + createNewFile() + } + // Create a socket stream on target ip:port and count the // words in input stream of \n delimited text (e.g. generated by 'nc') @@ -176,86 +173,55 @@ object JavaRecoverableNetworkWordCount { val words = lines.flatMap { it.split(SPACE).iterator() } -// val wordCounts = words -// .map { c(it, 1) } -// .reduceByKey { a, b -> a + b } -// .reduceByKey { a, b -> a + b } -// .reduceByKey { a, b -> a + b } -// .reduceByKey { a, b -> a + b } -// .reduceByKey { a, b -> a + b } -// .reduceByKey { a, b -> a + b } -// .reduceByKey { a, b -> a + b } -// .reduceByKey { a, b -> a + b } - - val wordCounts4 = words - .mapToPair { Tuple2(it, 1) } - .reduceByKey { a, b -> a + b } - .reduceByKey { a, b -> a + b } - .reduceByKey { a, b -> a + b } - .reduceByKey { a, b -> a + b } - .reduceByKey { a, b -> a + b } - .reduceByKey { a, b -> a + b } - .reduceByKey { a, b -> a + b } - .reduceByKey { a, b -> a + b } - - -// val wordCounts2 = words -// .map { it to 1 } -// .reduceByKey { a, b -> a + b } - val wordCounts3 = words .map { Tuple2(it, 1) } .reduceByKey { a, b -> a + b } - .reduceByKey { a, b -> a + b } - .reduceByKey { a, b -> a + b } - .reduceByKey { a, b -> a + b } - .reduceByKey { a, b -> a + b } - .reduceByKey { a, b -> a + b } - .reduceByKey { a, b -> a + b } - .reduceByKey { a, b -> a + b } -// val wordCounts5 = words -// .dstream() -// .map({ Tuple2(it, 1) }, fakeClassTag()) -// .let { DStream.toPairDStreamFunctions(it, fakeClassTag(), fakeClassTag(), null) } -// .reduceByKey { a, b -> a + b } -// .let { JavaDStream(it, fakeClassTag()) } + // in normal streaming context we can create a SparkSession from ssc: JavaStreamingContext + // normally `ssc.sparkContext().conf` + withSpark(ssc) { + listOf(1, 2, 3).toDS().show() + } wordCounts3.foreachRDD { rdd, time: Time -> - val sc = JavaSparkContext(rdd.context()) - - // Get or register the excludeList Broadcast - val excludeList = JavaWordExcludeList.getInstance(sc) - - // Get or register the droppedWordsCounter Accumulator - val droppedWordsCounter = JavaDroppedWordsCounter.getInstance(sc) - - // Use excludeList to drop words and use droppedWordsCounter to count them - val (counts, duration) = measureTimedValue { - rdd.filter { wordCount -> - if (excludeList.value().contains(wordCount._1)) { - droppedWordsCounter.add(wordCount._2.toLong()) - false - } else { - true - } - }.collect() - } - - println("Debug: ${rdd.toDebugString()}") - - val output = "Counts at time $time $counts\n$duration" - println(output) - println("Dropped ${droppedWordsCounter.value()} word(s) totally") - println("Appending to " + outputFile.absolutePath) - Files.append( - """ + // but in foreachRDD we must obtain this conf from the RDD + // like `rdd.context().conf` + withSpark(rdd) { + + rdd.toDS().show() + + // Get or register the excludeList Broadcast + val excludeList = KotlinWordExcludeList.getInstance(sc) + + // Get or register the droppedWordsCounter Accumulator + val droppedWordsCounter = KotlinDroppedWordsCounter.getInstance(sc) + + // Use excludeList to drop words and use droppedWordsCounter to count them + val (counts, duration) = measureTimedValue { + rdd.filter { wordCount -> + if (excludeList.value().contains(wordCount._1)) { + droppedWordsCounter.add(wordCount._2.toLong()) + false + } else { + true + } + }.collect() + } + + + val output = "Counts at time $time $counts\n$duration" + println(output) + println("Dropped ${droppedWordsCounter.value()} word(s) totally") + println("Appending to " + outputFile.absolutePath) + Files.append( + """ $output """.trimIndent(), - outputFile, - Charset.defaultCharset(), - ) + outputFile, + Charset.defaultCharset(), + ) + } } } diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Streaming.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Streaming.kt index de77c96f..07e06af9 100644 --- a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Streaming.kt +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Streaming.kt @@ -37,12 +37,15 @@ fun main() = withSparkStreaming(Durations.seconds(1), timeout = 10_000) { words.foreachRDD { rdd, time -> - val dataframe: Dataset = rdd.map { TestRow(it) }.toDS() + withSpark(rdd) { - dataframe - .groupByKey { it.word } - .count() - .show() + val dataframe: Dataset = rdd.map { TestRow(it) }.toDS() + + dataframe + .groupByKey { it.word } + .count() + .show() + } } diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt index d4481f1d..53d6a62e 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt @@ -22,6 +22,8 @@ * This file contains the main entry points and wrappers for the Kotlin Spark API. */ +@file:Suppress("UsePropertyAccessSyntax") + package org.jetbrains.kotlinx.spark.api import org.apache.spark.SparkConf @@ -38,13 +40,14 @@ import org.apache.spark.streaming.Durations import org.apache.spark.streaming.api.java.JavaStreamingContext import org.jetbrains.kotlinx.spark.api.SparkLogLevel.ERROR import org.jetbrains.kotlinx.spark.extensions.KSparkExtensions +import scala.Tuple2 /** * This wrapper over [SparkSession] which provides several additional methods to create [org.apache.spark.sql.Dataset]. * * @param spark The current [SparkSession] to wrap */ -open class KSparkSession(val spark: SparkSession) { +class KSparkSession(val spark: SparkSession) { /** Lazy instance of [JavaSparkContext] wrapper around [sparkContext]. */ val sc: JavaSparkContext by lazy { JavaSparkContext(spark.sparkContext) } @@ -82,13 +85,44 @@ open class KSparkSession(val spark: SparkSession) { /** * This wrapper over [SparkSession] and [JavaStreamingContext] provides several additional methods to create [org.apache.spark.sql.Dataset] */ -class KSparkStreamingSession(spark: SparkSession, val ssc: JavaStreamingContext) : KSparkSession(spark) { +class KSparkStreamingSession(val ssc: JavaStreamingContext) { /** Can be overwritten to be run after the streaming session has started and before it's terminated. */ var runAfterStart: KSparkStreamingSession.() -> Unit = {} -} + + fun invokeRunAfterStart(): Unit = runAfterStart() + + + fun withSpark(sc: SparkConf, func: KSparkSession.() -> Unit) { + val spark = SparkSession.builder().config(sc).getOrCreate() + KSparkSession(spark).apply(func) + } + + /** + * Helper function to enter Spark scope from [ssc] like + * ```kotlin + * ssc.withSpark { // this: KSparkSession + * + * } + * ``` + */ + fun withSpark(ssc: JavaStreamingContext, func: KSparkSession.() -> Unit) = withSpark(ssc.sparkContext().conf, func) + + + /** + * Helper function to enter Spark scope from a provided like + * when using the `foreachRDD` function. + * ```kotlin + * withSpark(rdd) { // this: KSparkSession + * + * } + * ``` + */ + fun withSpark(rdd: JavaRDDLike<*, *>, func: KSparkSession.() -> Unit) = withSpark(rdd.context().conf, func) +} + /** * The entry point to programming Spark with the Dataset and DataFrame API. @@ -160,7 +194,7 @@ inline fun withSpark( * @param logLevel Control our logLevel. This overrides any user-defined log settings. * @param func function which will be executed in context of [KSparkSession] (it means that `this` inside block will point to [KSparkSession]) */ -@Suppress("UsePropertyAccessSyntax") + @JvmOverloads inline fun withSpark(builder: Builder, logLevel: SparkLogLevel = ERROR, func: KSparkSession.() -> Unit) { builder @@ -212,7 +246,6 @@ inline fun withSpark(sparkConf: SparkConf, logLevel: SparkLogLevel = ERROR, func * @param logLevel Control our logLevel. This overrides any user-defined log settings. * @param timeout The time in milliseconds to wait for the stream to terminate without input. -1 by default, this means no timeout. * @param func function which will be executed in context of [KSparkStreamingSession] (it means that `this` inside block will point to [KSparkStreamingSession]) - * todo: provide alternatives with path instead of batchDuration etc */ @JvmOverloads inline fun withSparkStreaming( @@ -221,61 +254,56 @@ inline fun withSparkStreaming( props: Map = emptyMap(), master: String = SparkConf().get("spark.master", "local[*]"), appName: String = "Kotlin Spark Sample", - logLevel: SparkLogLevel = SparkLogLevel.ERROR, timeout: Long = -1L, - func: KSparkStreamingSession.() -> Unit, + crossinline func: KSparkStreamingSession.() -> Unit, ) { if (checkpointPath != null) { - TODO() -// var kSparkStreamingSession: KSparkStreamingSession? = null -// val ssc = JavaStreamingContext.getOrCreate(checkpointPath) { -// val jssc = JavaStreamingContext( -// SparkConf() -// .setAppName(appName) -// .setMaster(master) -// .setAll(props.map { (key, value) -> -// c(key, value.toString()).toTuple() -// }.asScalaIterable()), -// batchDuration, -// ) -// jssc.sparkContext().sc().setLogLevel(logLevel) -// jssc.checkpoint(checkpointPath) -// kSparkStreamingSession = KSparkStreamingSession( -// spark = SparkSession -// .builder() -// .sparkContext(jssc.sparkContext().sc()) -// .getOrCreate(), -// ssc = jssc, -// ).apply { func() } -// -// jssc -// } -// ssc.start() -// kSparkStreamingSession?.apply { runAfterStart() } -// ssc.awaitTerminationOrTimeout(timeout) -// ssc.stop() - } else { + var kSparkStreamingSession: KSparkStreamingSession? = null + val ssc = JavaStreamingContext.getOrCreate(checkpointPath) { + val sc = SparkConf() + .setAppName(appName) + .setMaster(master) + .setAll( + props + .map { (key, value) -> Tuple2(key, value.toString()) } + .asScalaIterable() + ) - withSpark( - props = props, - master = master, - appName = appName, - logLevel = logLevel, - ) { val ssc = JavaStreamingContext(sc, batchDuration) - KSparkStreamingSession(spark = spark, ssc = ssc).apply { - func() - ssc.start() - runAfterStart() - } + ssc.checkpoint(checkpointPath) - ssc.awaitTerminationOrTimeout(timeout) - ssc.stop() + kSparkStreamingSession = KSparkStreamingSession(ssc) + func(kSparkStreamingSession!!) + + ssc } + ssc.start() + kSparkStreamingSession?.invokeRunAfterStart() + ssc.awaitTerminationOrTimeout(timeout) + ssc.stop() + } else { + val sc = SparkConf() + .setAppName(appName) + .setMaster(master) + .setAll( + props + .map { (key, value) -> Tuple2(key, value.toString()) } + .asScalaIterable() + ) + val ssc = JavaStreamingContext(sc, batchDuration) + val kSparkStreamingSession = KSparkStreamingSession(ssc) + + func(kSparkStreamingSession) + ssc.start() + kSparkStreamingSession.invokeRunAfterStart() + + ssc.awaitTerminationOrTimeout(timeout) + ssc.stop() } } + /** * Broadcast a read-only variable to the cluster, returning a * [org.apache.spark.broadcast.Broadcast] object for reading it in distributed functions. diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Streaming.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Streaming.kt index 74aaa520..ccba0e01 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Streaming.kt +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Streaming.kt @@ -47,1176 +47,6 @@ fun JavaDStreamLike, *, *>.toPairDStream(): JavaPairDStream< fun JavaDStreamLike, *, *>.toPairDStream(): JavaPairDStream = mapToPair(Pair::toTuple) -///** -// * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to -// * generate the RDDs with `numPartitions` partitions. -// */ -//@JvmName("groupByKeyArity2") -//fun JavaDStreamLike, *, *>.groupByKey( -// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -//): JavaDStream>> = -// mapToPair { it.toTuple() } -// .groupByKey(numPartitions) -// .map { it.toArity() } -// -///** -// * Return a new DStream by applying `groupByKey` on each RDD. The supplied -// * org.apache.spark.Partitioner is used to control the partitioning of each RDD. -// */ -//@JvmName("groupByKeyArity2") -//fun JavaDStreamLike, *, *>.groupByKey(partitioner: Partitioner): JavaDStream>> = -// mapToPair { it.toTuple() } -// .groupByKey(partitioner) -// .map { it.toArity() } -// -///** -// * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are -// * merged using the supplied reduce function. Hash partitioning is used to generate the RDDs -// * with `numPartitions` partitions. -// */ -//@JvmName("reduceByKeyArity2") -//fun JavaDStreamLike, *, *>.reduceByKey( -// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -// reduceFunc: (V, V) -> V, -//): JavaDStream> = -// mapToPair { it.toTuple() } -// .reduceByKey(reduceFunc, numPartitions) -// .map { it.toArity() } -// -///** -// * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are -// * merged using the supplied reduce function. org.apache.spark.Partitioner is used to control -// * the partitioning of each RDD. -// */ -//@JvmName("reduceByKeyArity2") -//fun JavaDStreamLike, *, *>.reduceByKey( -// partitioner: Partitioner, -// reduceFunc: (V, V) -> V, -//): JavaDStream> = -// mapToPair { it.toTuple() } -// .reduceByKey(reduceFunc, partitioner) -// .map { it.toArity() } -// -///** -// * Combine elements of each key in DStream's RDDs using custom functions. This is similar to the -// * combineByKey for RDDs. Please refer to combineByKey in -// * org.apache.spark.rdd.PairRDDFunctions in the Spark core documentation for more information. -// */ -//@JvmName("combineByKeyArity2") -//fun JavaDStreamLike, *, *>.combineByKey( -// createCombiner: (V) -> C, -// mergeValue: (C, V) -> C, -// mergeCombiner: (C, C) -> C, -// partitioner: Partitioner, -// mapSideCombine: Boolean = true, -//): JavaDStream> = -// mapToPair { it.toTuple() } -// .combineByKey(createCombiner, mergeValue, mergeCombiner, partitioner, mapSideCombine) -// .map { it.toArity() } -// -///** -// * Return a new DStream by applying `groupByKey` over a sliding window on `this` DStream. -// * Similar to `DStream.groupByKey()`, but applies it over a sliding window. -// * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. -// * @param windowDuration width of the window; must be a multiple of this DStream's -// * batching interval -// * @param slideDuration sliding interval of the window (i.e., the interval after which -// * the new DStream will generate RDDs); must be a multiple of this -// * DStream's batching interval -// * @param numPartitions number of partitions of each RDD in the new DStream; if not specified -// * then Spark's default number of partitions will be used -// */ -//@JvmName("groupByKeyAndWindowArity2") -//fun JavaDStreamLike, *, *>.groupByKeyAndWindow( -// windowDuration: Duration, -// slideDuration: Duration = dstream().slideDuration(), -// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -//): JavaDStream>> = -// mapToPair { it.toTuple() } -// .groupByKeyAndWindow(windowDuration, slideDuration, numPartitions) -// .map { it.toArity() } -// -///** -// * Create a new DStream by applying `groupByKey` over a sliding window on `this` DStream. -// * Similar to `DStream.groupByKey()`, but applies it over a sliding window. -// * @param windowDuration width of the window; must be a multiple of this DStream's -// * batching interval -// * @param slideDuration sliding interval of the window (i.e., the interval after which -// * the new DStream will generate RDDs); must be a multiple of this -// * DStream's batching interval -// * @param partitioner partitioner for controlling the partitioning of each RDD in the new -// * DStream. -// */ -//@JvmName("groupByKeyAndWindowArity2") -//fun JavaDStreamLike, *, *>.groupByKeyAndWindow( -// windowDuration: Duration, -// slideDuration: Duration = dstream().slideDuration(), -// partitioner: Partitioner, -//): JavaDStream>> = -// mapToPair { it.toTuple() } -// .groupByKeyAndWindow(windowDuration, slideDuration, partitioner) -// .map { it.toArity() } -// -///** -// * Return a new DStream by applying `reduceByKey` over a sliding window. This is similar to -// * `DStream.reduceByKey()` but applies it over a sliding window. Hash partitioning is used to -// * generate the RDDs with `numPartitions` partitions. -// * @param reduceFunc associative and commutative reduce function -// * @param windowDuration width of the window; must be a multiple of this DStream's -// * batching interval -// * @param slideDuration sliding interval of the window (i.e., the interval after which -// * the new DStream will generate RDDs); must be a multiple of this -// * DStream's batching interval -// * @param numPartitions number of partitions of each RDD in the new DStream. -// */ -//@JvmName("reduceByKeyAndWindowArity2") -//fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( -// windowDuration: Duration, -// slideDuration: Duration = dstream().slideDuration(), -// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -// reduceFunc: (V, V) -> V, -//): JavaDStream> = -// mapToPair { it.toTuple() } -// .reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, numPartitions) -// .map { it.toArity() } -// -///** -// * Return a new DStream by applying `reduceByKey` over a sliding window. Similar to -// * `DStream.reduceByKey()`, but applies it over a sliding window. -// * @param reduceFunc associative and commutative reduce function -// * @param windowDuration width of the window; must be a multiple of this DStream's -// * batching interval -// * @param slideDuration sliding interval of the window (i.e., the interval after which -// * the new DStream will generate RDDs); must be a multiple of this -// * DStream's batching interval -// * @param partitioner partitioner for controlling the partitioning of each RDD -// * in the new DStream. -// */ -//@JvmName("reduceByKeyAndWindowArity2") -//fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( -// windowDuration: Duration, -// slideDuration: Duration = dstream().slideDuration(), -// partitioner: Partitioner, -// reduceFunc: (V, V) -> V, -//): JavaDStream> = -// mapToPair { it.toTuple() } -// .reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, partitioner) -// .map { it.toArity() } -// -///** -// * Return a new DStream by applying incremental `reduceByKey` over a sliding window. -// * The reduced value of over a new window is calculated using the old window's reduced value : -// * 1. reduce the new values that entered the window (e.g., adding new counts) -// * -// * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) -// * -// * This is more efficient than reduceByKeyAndWindow without "inverse reduce" function. -// * However, it is applicable to only "invertible reduce functions". -// * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. -// * @param reduceFunc associative and commutative reduce function -// * @param invReduceFunc inverse reduce function; such that for all y, invertible x: -// * `invReduceFunc(reduceFunc(x, y), x) = y` -// * @param windowDuration width of the window; must be a multiple of this DStream's -// * batching interval -// * @param slideDuration sliding interval of the window (i.e., the interval after which -// * the new DStream will generate RDDs); must be a multiple of this -// * DStream's batching interval -// * @param filterFunc Optional function to filter expired key-value pairs; -// * only pairs that satisfy the function are retained -// */ -//@JvmName("reduceByKeyAndWindowArity2") -//fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( -// invReduceFunc: (V, V) -> V, -// windowDuration: Duration, -// slideDuration: Duration = dstream().slideDuration(), -// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -// filterFunc: ((Arity2) -> Boolean)? = null, -// reduceFunc: (V, V) -> V, -//): JavaDStream> = -// mapToPair { it.toTuple() } -// .reduceByKeyAndWindow( -// reduceFunc, -// invReduceFunc, -// windowDuration, -// slideDuration, -// numPartitions, -// filterFunc?.let { -// { tuple -> -// filterFunc(tuple.toArity()) -// } -// } -// ) -// .map { it.toArity() } -// -///** -// * Return a new DStream by applying incremental `reduceByKey` over a sliding window. -// * The reduced value of over a new window is calculated using the old window's reduced value : -// * 1. reduce the new values that entered the window (e.g., adding new counts) -// * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) -// * This is more efficient than reduceByKeyAndWindow without "inverse reduce" function. -// * However, it is applicable to only "invertible reduce functions". -// * @param reduceFunc associative and commutative reduce function -// * @param invReduceFunc inverse reduce function -// * @param windowDuration width of the window; must be a multiple of this DStream's -// * batching interval -// * @param slideDuration sliding interval of the window (i.e., the interval after which -// * the new DStream will generate RDDs); must be a multiple of this -// * DStream's batching interval -// * @param partitioner partitioner for controlling the partitioning of each RDD in the new -// * DStream. -// * @param filterFunc Optional function to filter expired key-value pairs; -// * only pairs that satisfy the function are retained -// */ -//@JvmName("reduceByKeyAndWindowArity2") -//fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( -// invReduceFunc: (V, V) -> V, -// windowDuration: Duration, -// slideDuration: Duration = dstream().slideDuration(), -// partitioner: Partitioner, -// filterFunc: ((Arity2) -> Boolean)? = null, -// reduceFunc: (V, V) -> V, -//): JavaDStream> = -// mapToPair { it.toTuple() } -// .reduceByKeyAndWindow( -// reduceFunc, -// invReduceFunc, -// windowDuration, -// slideDuration, -// partitioner, -// filterFunc?.let { -// { tuple -> -// filterFunc(tuple.toArity()) -// } -// } -// ) -// .map { it.toArity() } -// -///** -// * Return a [MapWithStateDStream] by applying a function to every key-value element of -// * `this` stream, while maintaining some state data for each unique key. The mapping function -// * and other specification (e.g. partitioners, timeouts, initial state data, etc.) of this -// * transformation can be specified using `StateSpec` class. The state data is accessible in -// * as a parameter of type `State` in the mapping function. -// * -// * Example of using `mapWithState`: -// * {{{ -// * // A mapping function that maintains an integer state and return a String -// * def mappingFunction(key: String, value: Option[Int], state: State[Int]): Option[String] = { -// * // Use state.exists(), state.get(), state.update() and state.remove() -// * // to manage state, and return the necessary string -// * } -// * -// * val spec = StateSpec.function(mappingFunction).numPartitions(10) -// * -// * val mapWithStateDStream = keyValueDStream.mapWithState[StateType, MappedType](spec) -// * }}} -// * -// * @param spec Specification of this transformation -// * @tparam StateType Class type of the state data -// * @tparam MappedType Class type of the mapped data -// */ -//@JvmName("mapWithStateArity2") -//fun JavaDStreamLike, *, *>.mapWithState( -// spec: StateSpec, -//): JavaMapWithStateDStream = -// mapToPair { it.toTuple() } -// .mapWithState(spec) -// -///** -// * Return a new "state" DStream where the state for each key is updated by applying -// * the given function on the previous state of the key and the new values of each key. -// * In every batch the updateFunc will be called for each state even if there are no new values. -// * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. -// * @param updateFunc State update function. If `this` function returns None, then -// * corresponding state key-value pair will be eliminated. -// * @tparam S State type -// */ -//@JvmName("updateStateByKeyArity2") -//fun JavaDStreamLike, *, *>.updateStateByKey( -// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -// updateFunc: (List, S?) -> S?, -//): JavaDStream> = -// mapToPair { it.toTuple() } -// .updateStateByKey( -// { list: List, s: Optional -> -// updateFunc(list, s.getOrNull()).asOptional() -// }, -// numPartitions, -// ) -// .map { it.toArity() } -// -///** -// * Return a new "state" DStream where the state for each key is updated by applying -// * the given function on the previous state of the key and the new values of each key. -// * In every batch the updateFunc will be called for each state even if there are no new values. -// * [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD. -// * @param updateFunc State update function. Note, that this function may generate a different -// * tuple with a different key than the input key. Therefore keys may be removed -// * or added in this way. It is up to the developer to decide whether to -// * remember the partitioner despite the key being changed. -// * @param partitioner Partitioner for controlling the partitioning of each RDD in the new -// * DStream -// * @tparam S State type -// */ -//@JvmName("updateStateByKeyArity2") -//fun JavaDStreamLike, *, *>.updateStateByKey( -// partitioner: Partitioner, -// updateFunc: (List, S?) -> S?, -//): JavaDStream> = -// mapToPair { it.toTuple() } -// .updateStateByKey( -// { list: List, s: Optional -> -// updateFunc(list, s.getOrNull()).asOptional() -// }, -// partitioner, -// ) -// .map { it.toArity() } -// -///** -// * Return a new "state" DStream where the state for each key is updated by applying -// * the given function on the previous state of the key and the new values of the key. -// * org.apache.spark.Partitioner is used to control the partitioning of each RDD. -// * @param updateFunc State update function. If `this` function returns None, then -// * corresponding state key-value pair will be eliminated. -// * @param partitioner Partitioner for controlling the partitioning of each RDD in the new -// * DStream. -// * @param initialRDD initial state value of each key. -// * @tparam S State type -// */ -//@JvmName("updateStateByKeyArity2") -//fun JavaDStreamLike, *, *>.updateStateByKey( -// partitioner: Partitioner, -// initialRDD: JavaRDD>, -// updateFunc: (List, S?) -> S?, -//): JavaDStream> = -// mapToPair { it.toTuple() } -// .updateStateByKey( -// { list: List, s: Optional -> -// updateFunc(list, s.getOrNull()).asOptional() -// }, -// partitioner, -// initialRDD.mapToPair { it.toTuple() }, -// ) -// .map { it.toArity() } -// -///** -// * Return a new DStream by applying a map function to the value of each key-value pairs in -// * 'this' DStream without changing the key. -// */ -//@JvmName("mapValuesArity2") -//fun JavaDStreamLike, *, *>.mapValues( -// mapValuesFunc: (V) -> U, -//): JavaDStream> = -// mapToPair { it.toTuple() } -// .mapValues(mapValuesFunc) -// .map { it.toArity() } -// -///** -// * Return a new DStream by applying a flatmap function to the value of each key-value pairs in -// * 'this' DStream without changing the key. -// */ -//@JvmName("flatMapValuesArity2") -//fun JavaDStreamLike, *, *>.flatMapValues( -// flatMapValuesFunc: (V) -> Iterator, -//): JavaDStream> = -// mapToPair { it.toTuple() } -// .flatMapValues(flatMapValuesFunc) -// .map { it.toArity() } -// -///** -// * Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream. -// * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. -// */ -//@JvmName("cogroupArity2") -//fun JavaDStreamLike, *, *>.cogroup( -// other: JavaDStreamLike, *, *>, -// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -//): JavaDStream, Iterable>>> = -// mapToPair { it.toTuple() } -// .cogroup( -// other.mapToPair { it.toTuple() }, -// numPartitions, -// ) -// .map { -// c(it._1, it._2.toArity()) -// } -// -///** -// * Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream. -// * The supplied org.apache.spark.Partitioner is used to partition the generated RDDs. -// */ -//@JvmName("cogroupArity2") -//fun JavaDStreamLike, *, *>.cogroup( -// other: JavaDStreamLike, *, *>, -// partitioner: Partitioner, -//): JavaDStream, Iterable>>> = -// mapToPair { it.toTuple() } -// .cogroup( -// other.mapToPair { it.toTuple() }, -// partitioner, -// ) -// .map { -// c(it._1, it._2.toArity()) -// } -// -///** -// * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream. -// * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. -// */ -//@JvmName("joinArity2") -//fun JavaDStreamLike, *, *>.join( -// other: JavaDStreamLike, *, *>, -// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -//): JavaDStream>> = -// mapToPair { it.toTuple() } -// .join( -// other.mapToPair { it.toTuple() }, -// numPartitions, -// ) -// .map { -// c(it._1, it._2.toArity()) -// } -// -///** -// * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream. -// * The supplied org.apache.spark.Partitioner is used to control the partitioning of each RDD. -// */ -//@JvmName("joinArity2") -//fun JavaDStreamLike, *, *>.join( -// other: JavaDStreamLike, *, *>, -// partitioner: Partitioner, -//): JavaDStream>> = -// mapToPair { it.toTuple() } -// .join( -// other.mapToPair { it.toTuple() }, -// partitioner, -// ) -// .map { -// c(it._1, it._2.toArity()) -// } -// -///** -// * Return a new DStream by applying 'left outer join' between RDDs of `this` DStream and -// * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` -// * partitions. -// */ -//@JvmName("leftOuterJoinArity2") -//fun JavaDStreamLike, *, *>.leftOuterJoin( -// other: JavaDStreamLike, *, *>, -// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -//): JavaDStream>> = -// mapToPair { it.toTuple() } -// .leftOuterJoin( -// other.mapToPair { it.toTuple() }, -// numPartitions, -// ) -// .map { -// c(it._1, c(it._2._1, it._2._2.getOrNull())) -// } -// -///** -// * Return a new DStream by applying 'left outer join' between RDDs of `this` DStream and -// * `other` DStream. The supplied org.apache.spark.Partitioner is used to control -// * the partitioning of each RDD. -// */ -//@JvmName("leftOuterJoinArity2") -//fun JavaDStreamLike, *, *>.leftOuterJoin( -// other: JavaDStreamLike, *, *>, -// partitioner: Partitioner, -//): JavaDStream>> = -// mapToPair { it.toTuple() } -// .leftOuterJoin( -// other.mapToPair { it.toTuple() }, -// partitioner, -// ) -// .map { -// c(it._1, c(it._2._1, it._2._2.getOrNull())) -// } -// -///** -// * Return a new DStream by applying 'right outer join' between RDDs of `this` DStream and -// * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` -// * partitions. -// */ -//@JvmName("rightOuterJoinArity2") -//fun JavaDStreamLike, *, *>.rightOuterJoin( -// other: JavaDStreamLike, *, *>, -// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -//): JavaDStream>> = -// mapToPair { it.toTuple() } -// .rightOuterJoin( -// other.mapToPair { it.toTuple() }, -// numPartitions, -// ) -// .map { -// c(it._1, c(it._2._1.getOrNull(), it._2._2)) -// } -// -///** -// * Return a new DStream by applying 'right outer join' between RDDs of `this` DStream and -// * `other` DStream. The supplied org.apache.spark.Partitioner is used to control -// * the partitioning of each RDD. -// */ -//@JvmName("rightOuterJoinArity2") -//fun JavaDStreamLike, *, *>.rightOuterJoin( -// other: JavaDStreamLike, *, *>, -// partitioner: Partitioner, -//): JavaDStream>> = -// mapToPair { it.toTuple() } -// .rightOuterJoin( -// other.mapToPair { it.toTuple() }, -// partitioner, -// ) -// .map { -// c(it._1, c(it._2._1.getOrNull(), it._2._2)) -// } -// -///** -// * Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and -// * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` -// * partitions. -// */ -//@JvmName("fullOuterJoinArity2") -//fun JavaDStreamLike, *, *>.fullOuterJoin( -// other: JavaDStreamLike, *, *>, -// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -//): JavaDStream>> = -// mapToPair { it.toTuple() } -// .fullOuterJoin( -// other.mapToPair { it.toTuple() }, -// numPartitions, -// ) -// .map { -// c(it._1, c(it._2._1.getOrNull(), it._2._2.getOrNull())) -// } -// -///** -// * Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and -// * `other` DStream. The supplied org.apache.spark.Partitioner is used to control -// * the partitioning of each RDD. -// */ -//@JvmName("fullOuterJoinArity2") -//fun JavaDStreamLike, *, *>.fullOuterJoin( -// other: JavaDStreamLike, *, *>, -// partitioner: Partitioner, -//): JavaDStream>> = -// mapToPair { it.toTuple() } -// .fullOuterJoin( -// other.mapToPair { it.toTuple() }, -// partitioner, -// ) -// .map { -// c(it._1, c(it._2._1.getOrNull(), it._2._2.getOrNull())) -// } -// -///** -// * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is -// * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix". -// */ -//@JvmName("saveAsHadoopFilesArity2") -//fun JavaDStreamLike, *, *>.saveAsHadoopFiles( -// prefix: String, suffix: String, -//): Unit = -// mapToPair { it.toTuple() } -// .saveAsHadoopFiles(prefix, suffix) -// -///** -// * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is -// * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix". -// */ -//@JvmName("saveAsNewAPIHadoopFilesArity2") -//fun JavaDStreamLike, *, *>.saveAsNewAPIHadoopFiles( -// prefix: String, suffix: String, -//): Unit = -// mapToPair { it.toTuple() } -// .saveAsNewAPIHadoopFiles(prefix, suffix) -// -// -///** -// * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to -// * generate the RDDs with `numPartitions` partitions. -// */ -//@JvmName("groupByKeyPair") -//fun JavaDStreamLike, *, *>.groupByKey( -// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -//): JavaDStream>> = -// mapToPair { it.toTuple() } -// .groupByKey(numPartitions) -// .map { it.toPair() } -// -///** -// * Return a new DStream by applying `groupByKey` on each RDD. The supplied -// * org.apache.spark.Partitioner is used to control the partitioning of each RDD. -// */ -//@JvmName("groupByKeyPair") -//fun JavaDStreamLike, *, *>.groupByKey(partitioner: Partitioner): JavaDStream>> = -// mapToPair { it.toTuple() } -// .groupByKey(partitioner) -// .map { it.toPair() } -// -///** -// * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are -// * merged using the supplied reduce function. Hash partitioning is used to generate the RDDs -// * with `numPartitions` partitions. -// */ -//@JvmName("reduceByKeyPair") -//fun JavaDStreamLike, *, *>.reduceByKey( -// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -// reduceFunc: (V, V) -> V, -//): JavaDStream> = -// mapToPair { it.toTuple() } -// .reduceByKey(reduceFunc, numPartitions) -// .map { it.toPair() } -// -///** -// * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are -// * merged using the supplied reduce function. org.apache.spark.Partitioner is used to control -// * the partitioning of each RDD. -// */ -//@JvmName("reduceByKeyPair") -//fun JavaDStreamLike, *, *>.reduceByKey( -// partitioner: Partitioner, -// reduceFunc: (V, V) -> V, -//): JavaDStream> = -// mapToPair { it.toTuple() } -// .reduceByKey(reduceFunc, partitioner) -// .map { it.toPair() } -// -///** -// * Combine elements of each key in DStream's RDDs using custom functions. This is similar to the -// * combineByKey for RDDs. Please refer to combineByKey in -// * org.apache.spark.rdd.PairRDDFunctions in the Spark core documentation for more information. -// */ -//@JvmName("combineByKeyPair") -//fun JavaDStreamLike, *, *>.combineByKey( -// createCombiner: (V) -> C, -// mergeValue: (C, V) -> C, -// mergeCombiner: (C, C) -> C, -// partitioner: Partitioner, -// mapSideCombine: Boolean = true, -//): JavaDStream> = -// mapToPair { it.toTuple() } -// .combineByKey(createCombiner, mergeValue, mergeCombiner, partitioner, mapSideCombine) -// .map { it.toPair() } -// -///** -// * Return a new DStream by applying `groupByKey` over a sliding window on `this` DStream. -// * Similar to `DStream.groupByKey()`, but applies it over a sliding window. -// * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. -// * @param windowDuration width of the window; must be a multiple of this DStream's -// * batching interval -// * @param slideDuration sliding interval of the window (i.e., the interval after which -// * the new DStream will generate RDDs); must be a multiple of this -// * DStream's batching interval -// * @param numPartitions number of partitions of each RDD in the new DStream; if not specified -// * then Spark's default number of partitions will be used -// */ -//@JvmName("groupByKeyAndWindowPair") -//fun JavaDStreamLike, *, *>.groupByKeyAndWindow( -// windowDuration: Duration, -// slideDuration: Duration = dstream().slideDuration(), -// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -//): JavaDStream>> = -// mapToPair { it.toTuple() } -// .groupByKeyAndWindow(windowDuration, slideDuration, numPartitions) -// .map { it.toPair() } -// -///** -// * Create a new DStream by applying `groupByKey` over a sliding window on `this` DStream. -// * Similar to `DStream.groupByKey()`, but applies it over a sliding window. -// * @param windowDuration width of the window; must be a multiple of this DStream's -// * batching interval -// * @param slideDuration sliding interval of the window (i.e., the interval after which -// * the new DStream will generate RDDs); must be a multiple of this -// * DStream's batching interval -// * @param partitioner partitioner for controlling the partitioning of each RDD in the new -// * DStream. -// */ -//@JvmName("groupByKeyAndWindowPair") -//fun JavaDStreamLike, *, *>.groupByKeyAndWindow( -// windowDuration: Duration, -// slideDuration: Duration = dstream().slideDuration(), -// partitioner: Partitioner, -//): JavaDStream>> = -// mapToPair { it.toTuple() } -// .groupByKeyAndWindow(windowDuration, slideDuration, partitioner) -// .map { it.toPair() } -// -///** -// * Return a new DStream by applying `reduceByKey` over a sliding window. This is similar to -// * `DStream.reduceByKey()` but applies it over a sliding window. Hash partitioning is used to -// * generate the RDDs with `numPartitions` partitions. -// * @param reduceFunc associative and commutative reduce function -// * @param windowDuration width of the window; must be a multiple of this DStream's -// * batching interval -// * @param slideDuration sliding interval of the window (i.e., the interval after which -// * the new DStream will generate RDDs); must be a multiple of this -// * DStream's batching interval -// * @param numPartitions number of partitions of each RDD in the new DStream. -// */ -//@JvmName("reduceByKeyAndWindowPair") -//fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( -// windowDuration: Duration, -// slideDuration: Duration = dstream().slideDuration(), -// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -// reduceFunc: (V, V) -> V, -//): JavaDStream> = -// mapToPair { it.toTuple() } -// .reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, numPartitions) -// .map { it.toPair() } -// -///** -// * Return a new DStream by applying `reduceByKey` over a sliding window. Similar to -// * `DStream.reduceByKey()`, but applies it over a sliding window. -// * @param reduceFunc associative and commutative reduce function -// * @param windowDuration width of the window; must be a multiple of this DStream's -// * batching interval -// * @param slideDuration sliding interval of the window (i.e., the interval after which -// * the new DStream will generate RDDs); must be a multiple of this -// * DStream's batching interval -// * @param partitioner partitioner for controlling the partitioning of each RDD -// * in the new DStream. -// */ -//@JvmName("reduceByKeyAndWindowPair") -//fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( -// windowDuration: Duration, -// slideDuration: Duration = dstream().slideDuration(), -// partitioner: Partitioner, -// reduceFunc: (V, V) -> V, -//): JavaDStream> = -// mapToPair { it.toTuple() } -// .reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, partitioner) -// .map { it.toPair() } -// -///** -// * Return a new DStream by applying incremental `reduceByKey` over a sliding window. -// * The reduced value of over a new window is calculated using the old window's reduced value : -// * 1. reduce the new values that entered the window (e.g., adding new counts) -// * -// * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) -// * -// * This is more efficient than reduceByKeyAndWindow without "inverse reduce" function. -// * However, it is applicable to only "invertible reduce functions". -// * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. -// * @param reduceFunc associative and commutative reduce function -// * @param invReduceFunc inverse reduce function; such that for all y, invertible x: -// * `invReduceFunc(reduceFunc(x, y), x) = y` -// * @param windowDuration width of the window; must be a multiple of this DStream's -// * batching interval -// * @param slideDuration sliding interval of the window (i.e., the interval after which -// * the new DStream will generate RDDs); must be a multiple of this -// * DStream's batching interval -// * @param filterFunc Optional function to filter expired key-value pairs; -// * only pairs that satisfy the function are retained -// */ -//@JvmName("reduceByKeyAndWindowPair") -//fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( -// invReduceFunc: (V, V) -> V, -// windowDuration: Duration, -// slideDuration: Duration = dstream().slideDuration(), -// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -// filterFunc: ((Pair) -> Boolean)? = null, -// reduceFunc: (V, V) -> V, -//): JavaDStream> = -// mapToPair { it.toTuple() } -// .reduceByKeyAndWindow( -// reduceFunc, -// invReduceFunc, -// windowDuration, -// slideDuration, -// numPartitions, -// filterFunc?.let { -// { tuple -> -// filterFunc(tuple.toPair()) -// } -// } -// ) -// .map { it.toPair() } -// -///** -// * Return a new DStream by applying incremental `reduceByKey` over a sliding window. -// * The reduced value of over a new window is calculated using the old window's reduced value : -// * 1. reduce the new values that entered the window (e.g., adding new counts) -// * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) -// * This is more efficient than reduceByKeyAndWindow without "inverse reduce" function. -// * However, it is applicable to only "invertible reduce functions". -// * @param reduceFunc associative and commutative reduce function -// * @param invReduceFunc inverse reduce function -// * @param windowDuration width of the window; must be a multiple of this DStream's -// * batching interval -// * @param slideDuration sliding interval of the window (i.e., the interval after which -// * the new DStream will generate RDDs); must be a multiple of this -// * DStream's batching interval -// * @param partitioner partitioner for controlling the partitioning of each RDD in the new -// * DStream. -// * @param filterFunc Optional function to filter expired key-value pairs; -// * only pairs that satisfy the function are retained -// */ -//@JvmName("reduceByKeyAndWindowPair") -//fun JavaDStreamLike, *, *>.reduceByKeyAndWindow( -// invReduceFunc: (V, V) -> V, -// windowDuration: Duration, -// slideDuration: Duration = dstream().slideDuration(), -// partitioner: Partitioner, -// filterFunc: ((Pair) -> Boolean)? = null, -// reduceFunc: (V, V) -> V, -//): JavaDStream> = -// mapToPair { it.toTuple() } -// .reduceByKeyAndWindow( -// reduceFunc, -// invReduceFunc, -// windowDuration, -// slideDuration, -// partitioner, -// filterFunc?.let { -// { tuple -> -// filterFunc(tuple.toPair()) -// } -// } -// ) -// .map { it.toPair() } -// -///** -// * Return a [MapWithStateDStream] by applying a function to every key-value element of -// * `this` stream, while maintaining some state data for each unique key. The mapping function -// * and other specification (e.g. partitioners, timeouts, initial state data, etc.) of this -// * transformation can be specified using `StateSpec` class. The state data is accessible in -// * as a parameter of type `State` in the mapping function. -// * -// * Example of using `mapWithState`: -// * {{{ -// * // A mapping function that maintains an integer state and return a String -// * def mappingFunction(key: String, value: Option[Int], state: State[Int]): Option[String] = { -// * // Use state.exists(), state.get(), state.update() and state.remove() -// * // to manage state, and return the necessary string -// * } -// * -// * val spec = StateSpec.function(mappingFunction).numPartitions(10) -// * -// * val mapWithStateDStream = keyValueDStream.mapWithState[StateType, MappedType](spec) -// * }}} -// * -// * @param spec Specification of this transformation -// * @tparam StateType Class type of the state data -// * @tparam MappedType Class type of the mapped data -// */ -//@JvmName("mapWithStatePair") -//fun JavaDStreamLike, *, *>.mapWithState( -// spec: StateSpec, -//): JavaMapWithStateDStream = -// mapToPair { it.toTuple() } -// .mapWithState(spec) -// -///** -// * Return a new "state" DStream where the state for each key is updated by applying -// * the given function on the previous state of the key and the new values of each key. -// * In every batch the updateFunc will be called for each state even if there are no new values. -// * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. -// * @param updateFunc State update function. If `this` function returns None, then -// * corresponding state key-value pair will be eliminated. -// * @tparam S State type -// */ -//@JvmName("updateStateByKeyPair") -//fun JavaDStreamLike, *, *>.updateStateByKey( -// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -// updateFunc: (List, S?) -> S?, -//): JavaDStream> = -// mapToPair { it.toTuple() } -// .updateStateByKey( -// { list: List, s: Optional -> -// updateFunc(list, s.getOrNull()).asOptional() -// }, -// numPartitions, -// ) -// .map { it.toPair() } -// -///** -// * Return a new "state" DStream where the state for each key is updated by applying -// * the given function on the previous state of the key and the new values of each key. -// * In every batch the updateFunc will be called for each state even if there are no new values. -// * [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD. -// * @param updateFunc State update function. Note, that this function may generate a different -// * tuple with a different key than the input key. Therefore keys may be removed -// * or added in this way. It is up to the developer to decide whether to -// * remember the partitioner despite the key being changed. -// * @param partitioner Partitioner for controlling the partitioning of each RDD in the new -// * DStream -// * @tparam S State type -// */ -//@JvmName("updateStateByKeyPair") -//fun JavaDStreamLike, *, *>.updateStateByKey( -// partitioner: Partitioner, -// updateFunc: (List, S?) -> S?, -//): JavaDStream> = -// mapToPair { it.toTuple() } -// .updateStateByKey( -// { list: List, s: Optional -> -// updateFunc(list, s.getOrNull()).asOptional() -// }, -// partitioner, -// ) -// .map { it.toPair() } -// -///** -// * Return a new "state" DStream where the state for each key is updated by applying -// * the given function on the previous state of the key and the new values of the key. -// * org.apache.spark.Partitioner is used to control the partitioning of each RDD. -// * @param updateFunc State update function. If `this` function returns None, then -// * corresponding state key-value pair will be eliminated. -// * @param partitioner Partitioner for controlling the partitioning of each RDD in the new -// * DStream. -// * @param initialRDD initial state value of each key. -// * @tparam S State type -// */ -//@JvmName("updateStateByKeyPair") -//fun JavaDStreamLike, *, *>.updateStateByKey( -// partitioner: Partitioner, -// initialRDD: JavaRDD>, -// updateFunc: (List, S?) -> S?, -//): JavaDStream> = -// mapToPair { it.toTuple() } -// .updateStateByKey( -// { list: List, s: Optional -> -// updateFunc(list, s.getOrNull()).asOptional() -// }, -// partitioner, -// initialRDD.mapToPair { it.toTuple() }, -// ) -// .map { it.toPair() } -// -///** -// * Return a new DStream by applying a map function to the value of each key-value pairs in -// * 'this' DStream without changing the key. -// */ -//@JvmName("mapValuesPair") -//fun JavaDStreamLike, *, *>.mapValues( -// mapValuesFunc: (V) -> U, -//): JavaDStream> = -// mapToPair { it.toTuple() } -// .mapValues(mapValuesFunc) -// .map { it.toPair() } -// -///** -// * Return a new DStream by applying a flatmap function to the value of each key-value pairs in -// * 'this' DStream without changing the key. -// */ -//@JvmName("flatMapValuesPair") -//fun JavaDStreamLike, *, *>.flatMapValues( -// flatMapValuesFunc: (V) -> Iterator, -//): JavaDStream> = -// mapToPair { it.toTuple() } -// .flatMapValues(flatMapValuesFunc) -// .map { it.toPair() } -// -///** -// * Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream. -// * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. -// */ -//@JvmName("cogroupPair") -//fun JavaDStreamLike, *, *>.cogroup( -// other: JavaDStreamLike, *, *>, -// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -//): JavaDStream, Iterable>>> = -// mapToPair { it.toTuple() } -// .cogroup( -// other.mapToPair { it.toTuple() }, -// numPartitions, -// ) -// .map { -// Pair(it._1, it._2.toPair()) -// } -// -///** -// * Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream. -// * The supplied org.apache.spark.Partitioner is used to partition the generated RDDs. -// */ -//@JvmName("cogroupPair") -//fun JavaDStreamLike, *, *>.cogroup( -// other: JavaDStreamLike, *, *>, -// partitioner: Partitioner, -//): JavaDStream, Iterable>>> = -// mapToPair { it.toTuple() } -// .cogroup( -// other.mapToPair { it.toTuple() }, -// partitioner, -// ) -// .map { -// Pair(it._1, it._2.toPair()) -// } -// -///** -// * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream. -// * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. -// */ -//@JvmName("joinPair") -//fun JavaDStreamLike, *, *>.join( -// other: JavaDStreamLike, *, *>, -// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -//): JavaDStream>> = -// mapToPair { it.toTuple() } -// .join( -// other.mapToPair { it.toTuple() }, -// numPartitions, -// ) -// .map { -// Pair(it._1, it._2.toPair()) -// } -// -///** -// * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream. -// * The supplied org.apache.spark.Partitioner is used to control the partitioning of each RDD. -// */ -//@JvmName("joinPair") -//fun JavaDStreamLike, *, *>.join( -// other: JavaDStreamLike, *, *>, -// partitioner: Partitioner, -//): JavaDStream>> = -// mapToPair { it.toTuple() } -// .join( -// other.mapToPair { it.toTuple() }, -// partitioner, -// ) -// .map { -// Pair(it._1, it._2.toPair()) -// } -// -///** -// * Return a new DStream by applying 'left outer join' between RDDs of `this` DStream and -// * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` -// * partitions. -// */ -//@JvmName("leftOuterJoinPair") -//fun JavaDStreamLike, *, *>.leftOuterJoin( -// other: JavaDStreamLike, *, *>, -// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -//): JavaDStream>> = -// mapToPair { it.toTuple() } -// .leftOuterJoin( -// other.mapToPair { it.toTuple() }, -// numPartitions, -// ) -// .map { -// Pair(it._1, Pair(it._2._1, it._2._2.getOrNull())) -// } -// -///** -// * Return a new DStream by applying 'left outer join' between RDDs of `this` DStream and -// * `other` DStream. The supplied org.apache.spark.Partitioner is used to control -// * the partitioning of each RDD. -// */ -//@JvmName("leftOuterJoinPair") -//fun JavaDStreamLike, *, *>.leftOuterJoin( -// other: JavaDStreamLike, *, *>, -// partitioner: Partitioner, -//): JavaDStream>> = -// mapToPair { it.toTuple() } -// .leftOuterJoin( -// other.mapToPair { it.toTuple() }, -// partitioner, -// ) -// .map { -// Pair(it._1, Pair(it._2._1, it._2._2.getOrNull())) -// } -// -///** -// * Return a new DStream by applying 'right outer join' between RDDs of `this` DStream and -// * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` -// * partitions. -// */ -//@JvmName("rightOuterJoinPair") -//fun JavaDStreamLike, *, *>.rightOuterJoin( -// other: JavaDStreamLike, *, *>, -// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -//): JavaDStream>> = -// mapToPair { it.toTuple() } -// .rightOuterJoin( -// other.mapToPair { it.toTuple() }, -// numPartitions, -// ) -// .map { -// Pair(it._1, Pair(it._2._1.getOrNull(), it._2._2)) -// } -// -///** -// * Return a new DStream by applying 'right outer join' between RDDs of `this` DStream and -// * `other` DStream. The supplied org.apache.spark.Partitioner is used to control -// * the partitioning of each RDD. -// */ -//@JvmName("rightOuterJoinPair") -//fun JavaDStreamLike, *, *>.rightOuterJoin( -// other: JavaDStreamLike, *, *>, -// partitioner: Partitioner, -//): JavaDStream>> = -// mapToPair { it.toTuple() } -// .rightOuterJoin( -// other.mapToPair { it.toTuple() }, -// partitioner, -// ) -// .map { -// Pair(it._1, Pair(it._2._1.getOrNull(), it._2._2)) -// } -// -///** -// * Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and -// * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` -// * partitions. -// */ -//@JvmName("fullOuterJoinPair") -//fun JavaDStreamLike, *, *>.fullOuterJoin( -// other: JavaDStreamLike, *, *>, -// numPartitions: Int = dstream().ssc().sc().defaultParallelism(), -//): JavaDStream>> = -// mapToPair { it.toTuple() } -// .fullOuterJoin( -// other.mapToPair { it.toTuple() }, -// numPartitions, -// ) -// .map { -// Pair(it._1, Pair(it._2._1.getOrNull(), it._2._2.getOrNull())) -// } -// -///** -// * Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and -// * `other` DStream. The supplied org.apache.spark.Partitioner is used to control -// * the partitioning of each RDD. -// */ -//@JvmName("fullOuterJoinPair") -//fun JavaDStreamLike, *, *>.fullOuterJoin( -// other: JavaDStreamLike, *, *>, -// partitioner: Partitioner, -//): JavaDStream>> = -// mapToPair { it.toTuple() } -// .fullOuterJoin( -// other.mapToPair { it.toTuple() }, -// partitioner, -// ) -// .map { -// Pair(it._1, Pair(it._2._1.getOrNull(), it._2._2.getOrNull())) -// } -// -///** -// * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is -// * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix". -// */ -//@JvmName("saveAsHadoopFilesPair") -//fun JavaDStreamLike, *, *>.saveAsHadoopFiles( -// prefix: String, suffix: String, -//): Unit = -// mapToPair { it.toTuple() } -// .saveAsHadoopFiles(prefix, suffix) -// -///** -// * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is -// * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix". -// */ -//@JvmName("saveAsNewAPIHadoopFilesPair") -//fun JavaDStreamLike, *, *>.saveAsNewAPIHadoopFiles( -// prefix: String, suffix: String, -//): Unit = -// mapToPair { it.toTuple() } -// .saveAsNewAPIHadoopFiles(prefix, suffix) - - /** * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to * generate the RDDs with `numPartitions` partitions. From f523d9d1bd24f4deb6af67cb42f1a571a1a79024 Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Fri, 1 Apr 2022 14:39:12 +0200 Subject: [PATCH 17/44] small streaming updates --- .../kotlinx/spark/api/SparkSession.kt | 9 +-- .../jetbrains/kotlinx/spark/api/Streaming.kt | 67 +++++++++---------- .../kotlinx/spark/api/StreamingTest.kt | 20 ++++-- 3 files changed, 51 insertions(+), 45 deletions(-) diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt index 53d6a62e..85f43841 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt @@ -93,9 +93,10 @@ class KSparkStreamingSession(val ssc: JavaStreamingContext) { fun invokeRunAfterStart(): Unit = runAfterStart() - fun withSpark(sc: SparkConf, func: KSparkSession.() -> Unit) { + fun withSpark(sc: SparkConf, func: KSparkSession.() -> T): T { val spark = SparkSession.builder().config(sc).getOrCreate() - KSparkSession(spark).apply(func) + + return with(KSparkSession(spark), func) } /** @@ -106,7 +107,7 @@ class KSparkStreamingSession(val ssc: JavaStreamingContext) { * } * ``` */ - fun withSpark(ssc: JavaStreamingContext, func: KSparkSession.() -> Unit) = withSpark(ssc.sparkContext().conf, func) + fun withSpark(ssc: JavaStreamingContext, func: KSparkSession.() -> T): T = withSpark(ssc.sparkContext().conf, func) /** @@ -118,7 +119,7 @@ class KSparkStreamingSession(val ssc: JavaStreamingContext) { * } * ``` */ - fun withSpark(rdd: JavaRDDLike<*, *>, func: KSparkSession.() -> Unit) = withSpark(rdd.context().conf, func) + fun withSpark(rdd: JavaRDDLike<*, *>, func: KSparkSession.() -> T): T = withSpark(rdd.context().conf, func) } diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Streaming.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Streaming.kt index ccba0e01..ea9b4df4 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Streaming.kt +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Streaming.kt @@ -33,19 +33,18 @@ import org.apache.spark.streaming.dstream.DStream import scala.Tuple2 -@JvmName("tuple2ToPairDStream") fun JavaDStream>.toPairDStream(): JavaPairDStream = JavaPairDStream.fromJavaDStream(this) -fun JavaRDD>.toPairRDD(): JavaPairRDD = JavaPairRDD.fromJavaRDD(this) +fun JavaPairDStream.toTupleDStream(): JavaDStream> = + toJavaDStream() -@JvmName("arity2ToPairDStream") -fun JavaDStreamLike, *, *>.toPairDStream(): JavaPairDStream = - mapToPair(Arity2::toTuple) +fun JavaRDD>.toPairRDD(): JavaPairRDD = + JavaPairRDD.fromJavaRDD(this) + +fun JavaPairRDD.toTupleRDD(): JavaRDD> = + JavaPairRDD.toRDD(this).toJavaRDD() -@JvmName("pairToPairDStream") -fun JavaDStreamLike, *, *>.toPairDStream(): JavaPairDStream = - mapToPair(Pair::toTuple) /** * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to @@ -57,7 +56,7 @@ fun JavaDStream>.groupByKey( ): JavaDStream>> = toPairDStream() .groupByKey(numPartitions) - .toJavaDStream() + .toTupleDStream() /** * Return a new DStream by applying `groupByKey` on each RDD. The supplied @@ -67,7 +66,7 @@ fun JavaDStream>.groupByKey( fun JavaDStream>.groupByKey(partitioner: Partitioner): JavaDStream>> = toPairDStream() .groupByKey(partitioner) - .toJavaDStream() + .toTupleDStream() /** * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are @@ -81,7 +80,7 @@ fun JavaDStream>.reduceByKey( ): JavaDStream> = toPairDStream() .reduceByKey(reduceFunc, numPartitions) - .toJavaDStream() + .toTupleDStream() /** * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are @@ -95,7 +94,7 @@ fun JavaDStream>.reduceByKey( ): JavaDStream> = toPairDStream() .reduceByKey(reduceFunc, partitioner) - .toJavaDStream() + .toTupleDStream() /** * Combine elements of each key in DStream's RDDs using custom functions. This is similar to the @@ -112,7 +111,7 @@ fun JavaDStream>.combineByKey( ): JavaDStream> = toPairDStream() .combineByKey(createCombiner, mergeValue, mergeCombiner, partitioner, mapSideCombine) - .toJavaDStream() + .toTupleDStream() /** * Return a new DStream by applying `groupByKey` over a sliding window on `this` DStream. @@ -134,7 +133,7 @@ fun JavaDStream>.groupByKeyAndWindow( ): JavaDStream>> = toPairDStream() .groupByKeyAndWindow(windowDuration, slideDuration, numPartitions) - .toJavaDStream() + .toTupleDStream() /** * Create a new DStream by applying `groupByKey` over a sliding window on `this` DStream. @@ -155,7 +154,7 @@ fun JavaDStream>.groupByKeyAndWindow( ): JavaDStream>> = toPairDStream() .groupByKeyAndWindow(windowDuration, slideDuration, partitioner) - .toJavaDStream() + .toTupleDStream() /** * Return a new DStream by applying `reduceByKey` over a sliding window. This is similar to @@ -178,7 +177,7 @@ fun JavaDStream>.reduceByKeyAndWindow( ): JavaDStream> = toPairDStream() .reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, numPartitions) - .toJavaDStream() + .toTupleDStream() /** * Return a new DStream by applying `reduceByKey` over a sliding window. Similar to @@ -201,7 +200,7 @@ fun JavaDStream>.reduceByKeyAndWindow( ): JavaDStream> = toPairDStream() .reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, partitioner) - .toJavaDStream() + .toTupleDStream() /** * Return a new DStream by applying incremental `reduceByKey` over a sliding window. @@ -246,7 +245,7 @@ fun JavaDStream>.reduceByKeyAndWindow( } } ) - .toJavaDStream() + .toTupleDStream() /** * Return a new DStream by applying incremental `reduceByKey` over a sliding window. @@ -289,7 +288,7 @@ fun JavaDStream>.reduceByKeyAndWindow( } } ) - .toJavaDStream() + .toTupleDStream() /** * Return a [MapWithStateDStream] by applying a function to every key-value element of @@ -342,7 +341,7 @@ fun JavaDStream>.updateStateByKey( }, numPartitions, ) - .toJavaDStream() + .toTupleDStream() /** * Return a new "state" DStream where the state for each key is updated by applying @@ -369,7 +368,7 @@ fun JavaDStream>.updateStateByKey( }, partitioner, ) - .toJavaDStream() + .toTupleDStream() /** * Return a new "state" DStream where the state for each key is updated by applying @@ -396,7 +395,7 @@ fun JavaDStream>.updateStateByKey( partitioner, initialRDD.toPairRDD(), ) - .toJavaDStream() + .toTupleDStream() /** @@ -409,7 +408,7 @@ fun JavaDStream>.mapValues( ): JavaDStream> = toPairDStream() .mapValues(mapValuesFunc) - .toJavaDStream() + .toTupleDStream() /** * Return a new DStream by applying a flatmap function to the value of each key-value pairs in @@ -421,7 +420,7 @@ fun JavaDStream>.flatMapValues( ): JavaDStream> = toPairDStream() .flatMapValues(flatMapValuesFunc) - .toJavaDStream() + .toTupleDStream() /** * Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream. @@ -437,7 +436,7 @@ fun JavaDStream>.cogroup( other.toPairDStream(), numPartitions, ) - .toJavaDStream() + .toTupleDStream() /** @@ -454,7 +453,7 @@ fun JavaDStream>.cogroup( other.toPairDStream(), partitioner, ) - .toJavaDStream() + .toTupleDStream() /** * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream. @@ -470,7 +469,7 @@ fun JavaDStream>.join( other.toPairDStream(), numPartitions, ) - .toJavaDStream() + .toTupleDStream() /** * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream. @@ -486,7 +485,7 @@ fun JavaDStream>.join( other.toPairDStream(), partitioner, ) - .toJavaDStream() + .toTupleDStream() /** * Return a new DStream by applying 'left outer join' between RDDs of `this` DStream and @@ -503,7 +502,7 @@ fun JavaDStream>.leftOuterJoin( other.toPairDStream(), numPartitions, ) - .toJavaDStream() + .toTupleDStream() /** * Return a new DStream by applying 'left outer join' between RDDs of `this` DStream and @@ -520,7 +519,7 @@ fun JavaDStream>.leftOuterJoin( other.toPairDStream(), partitioner, ) - .toJavaDStream() + .toTupleDStream() /** * Return a new DStream by applying 'right outer join' between RDDs of `this` DStream and @@ -537,7 +536,7 @@ fun JavaDStream>.rightOuterJoin( other.toPairDStream(), numPartitions, ) - .toJavaDStream() + .toTupleDStream() /** * Return a new DStream by applying 'right outer join' between RDDs of `this` DStream and @@ -554,7 +553,7 @@ fun JavaDStream>.rightOuterJoin( other.toPairDStream(), partitioner, ) - .toJavaDStream() + .toTupleDStream() /** * Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and @@ -571,7 +570,7 @@ fun JavaDStream>.fullOuterJoin( other.toPairDStream(), numPartitions, ) - .toJavaDStream() + .toTupleDStream() /** * Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and @@ -588,7 +587,7 @@ fun JavaDStream>.fullOuterJoin( other.toPairDStream(), partitioner, ) - .toJavaDStream() + .toTupleDStream() /** * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is diff --git a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt index d93c7d23..47d7c408 100644 --- a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt +++ b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt @@ -22,9 +22,11 @@ package org.jetbrains.kotlinx.spark.api import io.kotest.core.spec.style.ShouldSpec import io.kotest.matchers.collections.shouldBeIn import io.kotest.matchers.shouldBe +import org.apache.spark.api.java.JavaRDD import org.apache.spark.streaming.Duration import java.io.Serializable import org.jetbrains.kotlinx.spark.api.* +import org.jetbrains.kotlinx.spark.api.tuples.* import java.util.LinkedList @@ -40,19 +42,23 @@ class StreamingTest : ShouldSpec({ } withSparkStreaming(Duration(10), timeout = 1000) { - val resultsBroadcast = spark.broadcast(results) - - val rdd = sc.parallelize(input) - val queue = LinkedList(listOf(rdd)) + val (resultsBroadcast, queue) = withSpark(ssc) { + val resultsBroadcast = spark.broadcast(results) + val rdd = sc.parallelize(input) + resultsBroadcast X LinkedList(listOf(rdd)) + } val inputStream = ssc.queueStream(queue) inputStream.foreachRDD { rdd, _ -> - rdd.toDS().forEach { - it shouldBeIn input - resultsBroadcast.value.counter++ + withSpark(rdd) { + rdd.toDS().forEach { + it shouldBeIn input + resultsBroadcast.value.counter++ + } } } + } results.counter shouldBe input.size From 2708a5c2054e07225c2b1a351fbf51ba66d7cb0b Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Fri, 1 Apr 2022 17:41:30 +0200 Subject: [PATCH 18/44] created temporary branch merging tuples with streaming. Will merge into normal "exploring-streaming" after vacation --- .../examples/KotlinDirectKafkaWordCount.kt | 8 +++++-- .../kotlinx/spark/api/SparkSession.kt | 21 ++++++++++------- .../jetbrains/kotlinx/spark/api/ApiTest.kt | 23 ------------------- .../kotlinx/spark/api/StreamingTest.kt | 14 ++++++----- 4 files changed, 27 insertions(+), 39 deletions(-) diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/KotlinDirectKafkaWordCount.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/KotlinDirectKafkaWordCount.kt index eea40720..897a9176 100644 --- a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/KotlinDirectKafkaWordCount.kt +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/KotlinDirectKafkaWordCount.kt @@ -30,8 +30,11 @@ import org.apache.spark.streaming.kafka010.ConsumerStrategies import org.apache.spark.streaming.kafka010.KafkaUtils import org.apache.spark.streaming.kafka010.LocationStrategies import org.jetbrains.kotlinx.spark.api.c +import org.jetbrains.kotlinx.spark.api.reduceByKey import org.jetbrains.kotlinx.spark.api.toTuple +import org.jetbrains.kotlinx.spark.api.tuples.* import org.jetbrains.kotlinx.spark.api.withSparkStreaming +import scala.Tuple2 import java.io.Serializable import java.util.regex.Pattern import kotlin.system.exitProcess @@ -56,6 +59,7 @@ import kotlin.system.exitProcess * consumer-group topic1,topic2 */ object KotlinDirectKafkaWordCount { + private val SPACE = Pattern.compile(" ") private const val DEFAULT_BROKER = "localhost:9092" @@ -102,8 +106,8 @@ object KotlinDirectKafkaWordCount { val lines: JavaDStream = messages.map { it.value() } val words: JavaDStream = lines.flatMap { it.split(SPACE).iterator() } - val wordCounts: JavaPairDStream = words - .mapToPair { c(it, 1).toTuple() } + val wordCounts: JavaDStream> = words + .map { it X 1 } .reduceByKey { a: Int, b: Int -> a + b } wordCounts.print() diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt index 85f43841..18d88fea 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt @@ -92,22 +92,26 @@ class KSparkStreamingSession(val ssc: JavaStreamingContext) { fun invokeRunAfterStart(): Unit = runAfterStart() + fun getSpark(sc: SparkConf): SparkSession = + SparkSession + .builder() + .config(sc) + .getOrCreate() - fun withSpark(sc: SparkConf, func: KSparkSession.() -> T): T { - val spark = SparkSession.builder().config(sc).getOrCreate() + fun withSpark(sc: SparkConf, func: KSparkSession.() -> T): T = + KSparkSession(getSpark(sc)).func() - return with(KSparkSession(spark), func) - } /** - * Helper function to enter Spark scope from [ssc] like + * Helper function to enter Spark scope from [sscForConf] like * ```kotlin - * ssc.withSpark { // this: KSparkSession + * withSpark(ssc) { // this: KSparkSession * * } * ``` */ - fun withSpark(ssc: JavaStreamingContext, func: KSparkSession.() -> T): T = withSpark(ssc.sparkContext().conf, func) + fun withSpark(sscForConf: JavaStreamingContext, func: KSparkSession.() -> T): T = + withSpark(sscForConf.sparkContext().conf, func) /** @@ -119,7 +123,8 @@ class KSparkStreamingSession(val ssc: JavaStreamingContext) { * } * ``` */ - fun withSpark(rdd: JavaRDDLike<*, *>, func: KSparkSession.() -> T): T = withSpark(rdd.context().conf, func) + fun withSpark(rddForConf: JavaRDDLike<*, *>, func: KSparkSession.() -> T): T = + withSpark(rddForConf.context().conf, func) } diff --git a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/ApiTest.kt b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/ApiTest.kt index b4e08216..e95a65f7 100644 --- a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/ApiTest.kt +++ b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/ApiTest.kt @@ -20,32 +20,9 @@ package org.jetbrains.kotlinx.spark.api/*- import ch.tutteli.atrium.api.fluent.en_GB.* import ch.tutteli.atrium.api.verbs.expect import io.kotest.core.spec.style.ShouldSpec -import io.kotest.matchers.should import io.kotest.matchers.shouldBe -import org.apache.spark.api.java.JavaDoubleRDD -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.rdd.RDD -import org.apache.spark.sql.Dataset -import org.apache.spark.sql.functions.* -import org.apache.spark.sql.streaming.GroupState -import org.apache.spark.sql.streaming.GroupStateTimeout -import org.apache.spark.sql.types.Decimal -import org.apache.spark.unsafe.types.CalendarInterval -import scala.Product -import scala.Tuple1 -import scala.Tuple2 -import scala.Tuple3 import scala.collection.Seq import java.io.Serializable -import java.math.BigDecimal -import java.sql.Date -import java.sql.Timestamp -import java.time.Duration -import java.time.Instant -import java.time.LocalDate -import java.time.Period import kotlin.collections.Iterator import scala.collection.Iterator as ScalaIterator import scala.collection.Map as ScalaMap diff --git a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt index 47d7c408..405181ce 100644 --- a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt +++ b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt @@ -42,16 +42,18 @@ class StreamingTest : ShouldSpec({ } withSparkStreaming(Duration(10), timeout = 1000) { - val (resultsBroadcast, queue) = withSpark(ssc) { - val resultsBroadcast = spark.broadcast(results) - val rdd = sc.parallelize(input) + val (resultsBroadcast, queue) = + withSpark(sscForConf = ssc) { + val resultsBroadcast = spark.broadcast(results) + val rdd = sc.parallelize(input) + + resultsBroadcast X LinkedList(listOf(rdd)) + } - resultsBroadcast X LinkedList(listOf(rdd)) - } val inputStream = ssc.queueStream(queue) inputStream.foreachRDD { rdd, _ -> - withSpark(rdd) { + withSpark(rddForConf = rdd) { rdd.toDS().forEach { it shouldBeIn input resultsBroadcast.value.counter++ From c70c0f3705385531c39fc837c05b625044e4f4d4 Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Tue, 12 Apr 2022 17:23:58 +0200 Subject: [PATCH 19/44] working on testing, not yet finished --- .../KotlinRecoverableNetworkWordCount.kt | 32 +-- kotlin-spark-api/3.2/pom_2.12.xml | 27 +++ .../kotlinx/spark/api/SparkSession.kt | 140 +++++++----- .../kotlinx/spark/api/StreamingTest.kt | 208 ++++++++++++++++-- pom.xml | 1 + 5 files changed, 318 insertions(+), 90 deletions(-) diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/KotlinRecoverableNetworkWordCount.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/KotlinRecoverableNetworkWordCount.kt index 10f57a28..7c4873f0 100644 --- a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/KotlinRecoverableNetworkWordCount.kt +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/KotlinRecoverableNetworkWordCount.kt @@ -28,8 +28,10 @@ import org.apache.spark.streaming.Durations import org.apache.spark.streaming.Time import org.apache.spark.util.LongAccumulator import org.jetbrains.kotlinx.spark.api.* +import org.jetbrains.kotlinx.spark.api.tuples.* import scala.Tuple2 import java.io.File +import java.io.Serializable import java.nio.charset.Charset import java.util.regex.Pattern import kotlin.experimental.ExperimentalTypeInference @@ -174,7 +176,7 @@ object KotlinRecoverableNetworkWordCount { val words = lines.flatMap { it.split(SPACE).iterator() } val wordCounts3 = words - .map { Tuple2(it, 1) } + .map { t(it, 1) } .reduceByKey { a, b -> a + b } // in normal streaming context we can create a SparkSession from ssc: JavaStreamingContext @@ -183,6 +185,10 @@ object KotlinRecoverableNetworkWordCount { listOf(1, 2, 3).toDS().show() } + setRunAfterStart { + println("Context is created and started running!") + } + wordCounts3.foreachRDD { rdd, time: Time -> // but in foreachRDD we must obtain this conf from the RDD // like `rdd.context().conf` @@ -197,19 +203,17 @@ object KotlinRecoverableNetworkWordCount { val droppedWordsCounter = KotlinDroppedWordsCounter.getInstance(sc) // Use excludeList to drop words and use droppedWordsCounter to count them - val (counts, duration) = measureTimedValue { - rdd.filter { wordCount -> - if (excludeList.value().contains(wordCount._1)) { - droppedWordsCounter.add(wordCount._2.toLong()) - false - } else { - true - } - }.collect() - } - - - val output = "Counts at time $time $counts\n$duration" + val counts = rdd.filter { (word, count) -> + if (excludeList.value().contains(word)) { + droppedWordsCounter.add(count.toLong()) + false + } else { + true + } + }.collect() + + + val output = "Counts at time $time $counts" println(output) println("Dropped ${droppedWordsCounter.value()} word(s) totally") println("Appending to " + outputFile.absolutePath) diff --git a/kotlin-spark-api/3.2/pom_2.12.xml b/kotlin-spark-api/3.2/pom_2.12.xml index cf0a68c4..586a89cc 100644 --- a/kotlin-spark-api/3.2/pom_2.12.xml +++ b/kotlin-spark-api/3.2/pom_2.12.xml @@ -63,6 +63,25 @@ provided + + + + + + + + + + + + + + org.apache.hadoop + hadoop-client + ${hadoop.version} + provided + + io.kotest @@ -88,6 +107,14 @@ ${atrium.version} test + + org.apache.spark + spark-streaming_${scala.compat.version} + ${spark3.version} + tests + test + + diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt index 18d88fea..465f5780 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt @@ -26,11 +26,14 @@ 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.JavaRDDLike import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.broadcast.Broadcast +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.rdd.RDD import org.apache.spark.sql.Dataset import org.apache.spark.sql.SparkSession.Builder @@ -39,8 +42,9 @@ import org.apache.spark.streaming.Duration import org.apache.spark.streaming.Durations import org.apache.spark.streaming.api.java.JavaStreamingContext import org.jetbrains.kotlinx.spark.api.SparkLogLevel.ERROR +import org.jetbrains.kotlinx.spark.api.tuples.* import org.jetbrains.kotlinx.spark.extensions.KSparkExtensions -import scala.Tuple2 +import java.io.Serializable /** * This wrapper over [SparkSession] which provides several additional methods to create [org.apache.spark.sql.Dataset]. @@ -85,51 +89,58 @@ class KSparkSession(val spark: SparkSession) { /** * This wrapper over [SparkSession] and [JavaStreamingContext] provides several additional methods to create [org.apache.spark.sql.Dataset] */ -class KSparkStreamingSession(val ssc: JavaStreamingContext) { - - /** Can be overwritten to be run after the streaming session has started and before it's terminated. */ - var runAfterStart: KSparkStreamingSession.() -> Unit = {} +class KSparkStreamingSession(@Transient val ssc: JavaStreamingContext) : Serializable { + // Serializable and Transient to that [withSpark] works inside [foreachRDD] and other Spark functions that serialize - fun invokeRunAfterStart(): Unit = runAfterStart() + private var runAfterStart: KSparkStreamingSession.() -> Unit = {} - fun getSpark(sc: SparkConf): SparkSession = - SparkSession - .builder() - .config(sc) - .getOrCreate() + /** Will be run after the streaming session has started and before it's terminated. */ + fun setRunAfterStart(block: KSparkStreamingSession.() -> Unit) { + runAfterStart = block + } - fun withSpark(sc: SparkConf, func: KSparkSession.() -> T): T = - KSparkSession(getSpark(sc)).func() + internal fun invokeRunAfterStart(): Unit = runAfterStart() /** - * Helper function to enter Spark scope from [sscForConf] like + * Helper function to enter Spark scope from a provided like + * when using the `foreachRDD` function. * ```kotlin - * withSpark(ssc) { // this: KSparkSession + * withSpark(rdd) { // this: KSparkSession * * } * ``` */ - fun withSpark(sscForConf: JavaStreamingContext, func: KSparkSession.() -> T): T = - withSpark(sscForConf.sparkContext().conf, func) + fun withSpark(rddForConf: JavaRDDLike<*, *>, func: KSparkSession.() -> T): T = + withSpark(rddForConf.context().conf, func) + + + fun getSpark(sc: SparkConf): SparkSession = SparkSession + .builder() + .config(sc) + .getOrCreate() + + fun withSpark(sc: SparkConf, func: KSparkSession.() -> T): T = + KSparkSession(getSpark(sc)).func() /** - * Helper function to enter Spark scope from a provided like - * when using the `foreachRDD` function. + * Helper function to enter Spark scope from [sscForConf] like * ```kotlin - * withSpark(rdd) { // this: KSparkSession + * withSpark(ssc) { // this: KSparkSession * * } * ``` */ - fun withSpark(rddForConf: JavaRDDLike<*, *>, func: KSparkSession.() -> T): T = - withSpark(rddForConf.context().conf, func) + fun withSpark(sscForConf: JavaStreamingContext, func: KSparkSession.() -> T): T = + withSpark(sscForConf.sparkContext().conf, func) } + + /** * The entry point to programming Spark with the Dataset and DataFrame API. * @@ -239,52 +250,67 @@ inline fun withSpark(sparkConf: SparkConf, logLevel: SparkLogLevel = ERROR, func * recreated from the checkpoint data. If the data does not exist, then the provided factory * will be used to create a JavaStreamingContext. * - * @param batchDuration The time interval at which streaming data will be divided into batches. Defaults to 1 second. - * @param checkpointPath If checkpoint data exists in the provided `checkpointPath`, then StreamingContext will be - * recreated from the checkpoint data. If the data does not exist (or `null` is provided), then the streaming context will be built using - * the other provided parameters. - * @param props spark options, value types are runtime-checked for type-correctness - * @param master Sets the Spark master URL to connect to, such as "local" to run locally, "local[4]" to - * run locally with 4 cores, or "spark://master:7077" to run on a Spark standalone cluster. By default, it - * tries to get the system value "spark.master", otherwise it uses "local[*]" - * @param appName Sets a name for the application, which will be shown in the Spark web UI. - * If no application name is set, a randomly generated name will be used. - * @param logLevel Control our logLevel. This overrides any user-defined log settings. - * @param timeout The time in milliseconds to wait for the stream to terminate without input. -1 by default, this means no timeout. - * @param func function which will be executed in context of [KSparkStreamingSession] (it means that `this` inside block will point to [KSparkStreamingSession]) + * @param batchDuration The time interval at which streaming data will be divided into batches. Defaults to 1 + * second. + * @param checkpointPath If checkpoint data exists in the provided `checkpointPath`, then StreamingContext will be + * recreated from the checkpoint data. If the data does not exist (or `null` is provided), + * then the streaming context will be built using the other provided parameters. + * @param hadoopConf Only used if [checkpointPath] is given. Hadoop configuration if necessary for reading from + * any HDFS compatible file system. + * @param createOnError Only used if [checkpointPath] is given. Whether to create a new JavaStreamingContext if + * there is an error in reading checkpoint data. + * @param props Spark options, value types are runtime-checked for type-correctness. + * @param master Sets the Spark master URL to connect to, such as "local" to run locally, "local[4]" to + * run locally with 4 cores, or "spark://master:7077" to run on a Spark standalone cluster. + * By default, it tries to get the system value "spark.master", otherwise it uses "local[*]". + * @param appName Sets a name for the application, which will be shown in the Spark web UI. + * If no application name is set, a randomly generated name will be used. + * @param timeout The time in milliseconds to wait for the stream to terminate without input. -1 by default, + * this means no timeout. + * @param func Function which will be executed in context of [KSparkStreamingSession] (it means that + * `this` inside block will point to [KSparkStreamingSession]) */ @JvmOverloads -inline fun withSparkStreaming( +fun withSparkStreaming( batchDuration: Duration = Durations.seconds(1L), checkpointPath: String? = null, + hadoopConf: Configuration = SparkHadoopUtil.get().conf(), + createOnError: Boolean = false, props: Map = emptyMap(), master: String = SparkConf().get("spark.master", "local[*]"), appName: String = "Kotlin Spark Sample", timeout: Long = -1L, - crossinline func: KSparkStreamingSession.() -> Unit, + startStreamingContext: Boolean = true, + func: KSparkStreamingSession.() -> Unit, ) { if (checkpointPath != null) { - var kSparkStreamingSession: KSparkStreamingSession? = null - val ssc = JavaStreamingContext.getOrCreate(checkpointPath) { - val sc = SparkConf() - .setAppName(appName) - .setMaster(master) - .setAll( - props - .map { (key, value) -> Tuple2(key, value.toString()) } - .asScalaIterable() - ) - val ssc = JavaStreamingContext(sc, batchDuration) - ssc.checkpoint(checkpointPath) - - kSparkStreamingSession = KSparkStreamingSession(ssc) - - func(kSparkStreamingSession!!) + var kSparkStreamingSession: KSparkStreamingSession? = null - ssc - } - ssc.start() + val ssc = JavaStreamingContext.getOrCreate( + /* checkpointPath = */ checkpointPath, + /* creatingFunc = */ { + val sc = SparkConf() + .setAppName(appName) + .setMaster(master) + .setAll( + props + .map { (key, value) -> key X value.toString() } + .asScalaIterable() + ) + + val ssc = JavaStreamingContext(sc, batchDuration) + ssc.checkpoint(checkpointPath) + + kSparkStreamingSession = KSparkStreamingSession(ssc) + func(kSparkStreamingSession!!) + + ssc + }, + /* hadoopConf = */ hadoopConf, + /* createOnError = */ createOnError + ) + if (startStreamingContext) ssc.start() kSparkStreamingSession?.invokeRunAfterStart() ssc.awaitTerminationOrTimeout(timeout) ssc.stop() @@ -294,7 +320,7 @@ inline fun withSparkStreaming( .setMaster(master) .setAll( props - .map { (key, value) -> Tuple2(key, value.toString()) } + .map { (key, value) -> key X value.toString() } .asScalaIterable() ) val ssc = JavaStreamingContext(sc, batchDuration) diff --git a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt index 405181ce..3f3515ec 100644 --- a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt +++ b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt @@ -22,12 +22,22 @@ package org.jetbrains.kotlinx.spark.api import io.kotest.core.spec.style.ShouldSpec import io.kotest.matchers.collections.shouldBeIn import io.kotest.matchers.shouldBe -import org.apache.spark.api.java.JavaRDD -import org.apache.spark.streaming.Duration +import org.apache.commons.io.FileUtils +import org.apache.hadoop.fs.FileSystem +import org.apache.spark.SparkConf +import org.apache.spark.api.java.JavaSparkContext +import org.apache.spark.streaming.* +import org.apache.spark.streaming.api.java.JavaStreamingContext +import org.apache.spark.util.Utils +import org.jetbrains.kotlinx.spark.api.tuples.X +import org.jetbrains.kotlinx.spark.api.tuples.component1 +import org.jetbrains.kotlinx.spark.api.tuples.component2 +import java.io.File import java.io.Serializable -import org.jetbrains.kotlinx.spark.api.* -import org.jetbrains.kotlinx.spark.api.tuples.* -import java.util.LinkedList +import java.net.ConnectException +import java.nio.charset.StandardCharsets +import java.util.* +import java.util.concurrent.atomic.AtomicBoolean class StreamingTest : ShouldSpec({ @@ -35,35 +45,195 @@ class StreamingTest : ShouldSpec({ should("stream") { val input = listOf("aaa", "bbb", "aaa", "ccc") - - val results = object : Serializable { - @Volatile - var counter = 0 - } + val counter = Counter(0) withSparkStreaming(Duration(10), timeout = 1000) { - val (resultsBroadcast, queue) = - withSpark(sscForConf = ssc) { - val resultsBroadcast = spark.broadcast(results) - val rdd = sc.parallelize(input) - resultsBroadcast X LinkedList(listOf(rdd)) - } + val (counterBroadcast, queue) = withSpark(ssc) { + spark.broadcast(counter) X LinkedList(listOf(sc.parallelize(input))) + } val inputStream = ssc.queueStream(queue) inputStream.foreachRDD { rdd, _ -> - withSpark(rddForConf = rdd) { + withSpark(rdd) { rdd.toDS().forEach { it shouldBeIn input - resultsBroadcast.value.counter++ + counterBroadcast.value.value++ } } } + } + + counter.value shouldBe input.size + + } + +// should("checkpoint") { +// +// val emptyDir: File = Files.createTempDir() +// emptyDir.deleteOnExit() +// val contextSuite = StreamingContextSuite() +// val corruptedCheckpointDir: String = contextSuite.createCorruptedCheckpoint() +// val checkpointDir: String = contextSuite.createValidCheckpoint() +// +// // Function to create JavaStreamingContext without any output operations +// // (used to detect the new context) +// +// // Function to create JavaStreamingContext without any output operations +// // (used to detect the new context) +// val newContextCreated = AtomicBoolean(false) +// val creatingFunc: Function0 = { +// newContextCreated.set(true) +// JavaStreamingContext(conf, Seconds.apply(1)) +// } +// +// newContextCreated.set(false) +// ssc = JavaStreamingContext.getOrCreate(emptyDir.absolutePath, creatingFunc) +// Assert.assertTrue("new context not created", newContextCreated.get()) +// ssc.stop() +// +// newContextCreated.set(false) +// ssc = JavaStreamingContext.getOrCreate( +// corruptedCheckpointDir, creatingFunc, +// Configuration(), true +// ) +// Assert.assertTrue("new context not created", newContextCreated.get()) +// ssc.stop() +// +// newContextCreated.set(false) +// ssc = JavaStreamingContext.getOrCreate( +// checkpointDir, creatingFunc, +// Configuration() +// ) +// Assert.assertTrue("old context not recovered", !newContextCreated.get()) +// ssc.stop() +// +// newContextCreated.set(false) +// val sc = JavaSparkContext(conf) +// ssc = JavaStreamingContext.getOrCreate( +// checkpointDir, creatingFunc, +// Configuration() +// ) +// Assert.assertTrue("old context not recovered", !newContextCreated.get()) +// ssc.stop() +// } + + should("Work with checkpointpath") { + + + val conf = SparkConf() + .setMaster("local[*]") + .setAppName("Kotlin Spark Sample") + .set("newContext", "true") + + val emptyDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark") + emptyDir.deleteOnExit() + + val batchDuration = Durations.seconds(1) + val timeout = Durations.seconds(1).milliseconds() + + val corruptedCheckpointDir = createCorruptedCheckpoint() + val checkpointDir = createValidCheckpoint(conf, batchDuration) + + val newContextCreated = AtomicBoolean(false) + + val creatingFun: KSparkStreamingSession.() -> Unit = { +// if (conf == null) conf = ssc.sparkContext().conf + + println("created new context") + newContextCreated.set(true) +// setRunAfterStart { +// ssc.stop() +// } } - results.counter shouldBe input.size + + + newContextCreated.set(false) + withSparkStreaming( + batchDuration = batchDuration, + checkpointPath = emptyDir.absolutePath, + props = mapOf("newContext" to true), + timeout = timeout, + func = creatingFun, + startStreamingContext = false, + ) + newContextCreated.get() shouldBe true + + + + newContextCreated.set(false) + withSparkStreaming( + batchDuration = batchDuration, + checkpointPath = corruptedCheckpointDir, + props = mapOf("newContext" to true), + timeout = timeout, + func = creatingFun, + startStreamingContext = false, + createOnError = true, + ) + newContextCreated.get() shouldBe true + + newContextCreated.set(false) + withSparkStreaming( + batchDuration = batchDuration, + checkpointPath = checkpointDir, + props = mapOf("newContext" to true), + timeout = timeout, + func = creatingFun, + startStreamingContext = false, + ) + newContextCreated.get() shouldBe false + + + newContextCreated.set(false) +// val sc = JavaSparkContext( +// SparkConf() +// .setAppName("test") +// .setMaster("local[*]") +// ) + withSparkStreaming( + batchDuration = batchDuration, + checkpointPath = checkpointDir, + props = mapOf("newContext" to true), + timeout = timeout, + func = creatingFun, + startStreamingContext = false, + ) + newContextCreated.get() shouldBe false + // todo do something with checkpoint again, check that it doesn't create new instance + + + // TODO clean up checkpoint } } }) + +private fun createCorruptedCheckpoint(): String { + val checkpointDirectory = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark").absolutePath + val fakeCheckpointFile = Checkpoint.checkpointFile(checkpointDirectory, Time(1000)) + FileUtils.write(File(fakeCheckpointFile.toString()), "blablabla", StandardCharsets.UTF_8) + assert(Checkpoint.getCheckpointFiles(checkpointDirectory, (null as FileSystem?).asOption()).nonEmpty()) + return checkpointDirectory +} + +private fun createValidCheckpoint(conf: SparkConf, batchDuration: Duration): String { + val testDirectory = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark").absolutePath + val checkpointDirectory = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark").absolutePath + val ssc = JavaStreamingContext( + conf.clone().set("someKey", "someValue"), + batchDuration, + ) + + ssc.checkpoint(checkpointDirectory) + ssc.textFileStream(testDirectory).foreachRDD { rdd, _ -> rdd.count() } + ssc.start() + ssc.stop() + + return checkpointDirectory +} + +class Counter(@Volatile var value: Int) : Serializable + diff --git a/pom.xml b/pom.xml index 0df3adac..5d76c6df 100644 --- a/pom.xml +++ b/pom.xml @@ -16,6 +16,7 @@ 4.6.0 1.0.1 3.2.1 + 3.3.1 2.10.0 From 02d50ccf1d869b43339b92c9e6bf1f41a76aa31c Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Wed, 13 Apr 2022 16:16:17 +0200 Subject: [PATCH 20/44] updated to kotlin 1.6.20, refactored withSparkStreaming and fixed tests --- .../kotlinx/spark/api/SparkSession.kt | 148 +++++++++--------- .../kotlinx/spark/api/StreamingTest.kt | 146 +++++------------ pom.xml | 2 +- 3 files changed, 110 insertions(+), 186 deletions(-) diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt index 465f5780..1e6e5025 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt @@ -90,11 +90,12 @@ class KSparkSession(val spark: SparkSession) { * This wrapper over [SparkSession] and [JavaStreamingContext] provides several additional methods to create [org.apache.spark.sql.Dataset] */ class KSparkStreamingSession(@Transient val ssc: JavaStreamingContext) : Serializable { - // Serializable and Transient to that [withSpark] works inside [foreachRDD] and other Spark functions that serialize + // Serializable and Transient so that [withSpark] works inside [foreachRDD] and other Spark functions that serialize private var runAfterStart: KSparkStreamingSession.() -> Unit = {} - /** Will be run after the streaming session has started and before it's terminated. */ + /** [block] will be run after the streaming session has started from a new context (so not when loading from a checkpoint) + * and before it's terminated. */ fun setRunAfterStart(block: KSparkStreamingSession.() -> Unit) { runAfterStart = block } @@ -102,6 +103,30 @@ class KSparkStreamingSession(@Transient val ssc: JavaStreamingContext) : Seriali internal fun invokeRunAfterStart(): Unit = runAfterStart() + /** Creates new spark session from given [sc]. */ + fun getSpark(sc: SparkConf): SparkSession = + SparkSession + .builder() + .config(sc) + .getOrCreate() + + /** Creates new spark session from context of given JavaRDD, [rddForConf]. */ + fun getSpark(rddForConf: JavaRDDLike<*, *>): SparkSession = getSpark(rddForConf.context().conf) + + /** Creates new spark session from context of given JavaStreamingContext, [sscForConf] */ + fun getSpark(sscForConf: JavaStreamingContext): SparkSession = getSpark(sscForConf.sparkContext().conf) + + /** + * Helper function to enter Spark scope from [sc] like + * ```kotlin + * withSpark(sc) { // this: KSparkSession + * + * } + * ``` + */ + fun withSpark(sc: SparkConf, func: KSparkSession.() -> T): T = + KSparkSession(getSpark(sc)).func() + /** * Helper function to enter Spark scope from a provided like * when using the `foreachRDD` function. @@ -112,17 +137,7 @@ class KSparkStreamingSession(@Transient val ssc: JavaStreamingContext) : Seriali * ``` */ fun withSpark(rddForConf: JavaRDDLike<*, *>, func: KSparkSession.() -> T): T = - withSpark(rddForConf.context().conf, func) - - - fun getSpark(sc: SparkConf): SparkSession = SparkSession - .builder() - .config(sc) - .getOrCreate() - - fun withSpark(sc: SparkConf, func: KSparkSession.() -> T): T = - KSparkSession(getSpark(sc)).func() - + KSparkSession(getSpark(rddForConf)).func() /** * Helper function to enter Spark scope from [sscForConf] like @@ -133,14 +148,10 @@ class KSparkStreamingSession(@Transient val ssc: JavaStreamingContext) : Seriali * ``` */ fun withSpark(sscForConf: JavaStreamingContext, func: KSparkSession.() -> T): T = - withSpark(sscForConf.sparkContext().conf, func) - - + KSparkSession(getSpark(sscForConf)).func() } - - /** * The entry point to programming Spark with the Dataset and DataFrame API. * @@ -250,25 +261,26 @@ inline fun withSpark(sparkConf: SparkConf, logLevel: SparkLogLevel = ERROR, func * recreated from the checkpoint data. If the data does not exist, then the provided factory * will be used to create a JavaStreamingContext. * - * @param batchDuration The time interval at which streaming data will be divided into batches. Defaults to 1 - * second. - * @param checkpointPath If checkpoint data exists in the provided `checkpointPath`, then StreamingContext will be - * recreated from the checkpoint data. If the data does not exist (or `null` is provided), - * then the streaming context will be built using the other provided parameters. - * @param hadoopConf Only used if [checkpointPath] is given. Hadoop configuration if necessary for reading from - * any HDFS compatible file system. - * @param createOnError Only used if [checkpointPath] is given. Whether to create a new JavaStreamingContext if - * there is an error in reading checkpoint data. - * @param props Spark options, value types are runtime-checked for type-correctness. - * @param master Sets the Spark master URL to connect to, such as "local" to run locally, "local[4]" to - * run locally with 4 cores, or "spark://master:7077" to run on a Spark standalone cluster. - * By default, it tries to get the system value "spark.master", otherwise it uses "local[*]". - * @param appName Sets a name for the application, which will be shown in the Spark web UI. - * If no application name is set, a randomly generated name will be used. - * @param timeout The time in milliseconds to wait for the stream to terminate without input. -1 by default, - * this means no timeout. - * @param func Function which will be executed in context of [KSparkStreamingSession] (it means that - * `this` inside block will point to [KSparkStreamingSession]) + * @param batchDuration The time interval at which streaming data will be divided into batches. Defaults to 1 + * second. + * @param checkpointPath If checkpoint data exists in the provided `checkpointPath`, then StreamingContext will be + * recreated from the checkpoint data. If the data does not exist (or `null` is provided), + * then the streaming context will be built using the other provided parameters. + * @param hadoopConf Only used if [checkpointPath] is given. Hadoop configuration if necessary for reading from + * any HDFS compatible file system. + * @param createOnError Only used if [checkpointPath] is given. Whether to create a new JavaStreamingContext if + * there is an error in reading checkpoint data. + * @param props Spark options, value types are runtime-checked for type-correctness. + * @param master Sets the Spark master URL to connect to, such as "local" to run locally, "local[4]" to + * run locally with 4 cores, or "spark://master:7077" to run on a Spark standalone cluster. + * By default, it tries to get the system value "spark.master", otherwise it uses "local[*]". + * @param appName Sets a name for the application, which will be shown in the Spark web UI. + * If no application name is set, a randomly generated name will be used. + * @param timeout The time in milliseconds to wait for the stream to terminate without input. -1 by default, + * this means no timeout. + * @param startStreamingContext Defaults to `true`. If set to `false`, then the streaming context will not be started. + * @param func Function which will be executed in context of [KSparkStreamingSession] (it means that + * `this` inside block will point to [KSparkStreamingSession]) */ @JvmOverloads fun withSparkStreaming( @@ -283,38 +295,11 @@ fun withSparkStreaming( startStreamingContext: Boolean = true, func: KSparkStreamingSession.() -> Unit, ) { - if (checkpointPath != null) { - - var kSparkStreamingSession: KSparkStreamingSession? = null - - val ssc = JavaStreamingContext.getOrCreate( - /* checkpointPath = */ checkpointPath, - /* creatingFunc = */ { - val sc = SparkConf() - .setAppName(appName) - .setMaster(master) - .setAll( - props - .map { (key, value) -> key X value.toString() } - .asScalaIterable() - ) - - val ssc = JavaStreamingContext(sc, batchDuration) - ssc.checkpoint(checkpointPath) - - kSparkStreamingSession = KSparkStreamingSession(ssc) - func(kSparkStreamingSession!!) - - ssc - }, - /* hadoopConf = */ hadoopConf, - /* createOnError = */ createOnError - ) - if (startStreamingContext) ssc.start() - kSparkStreamingSession?.invokeRunAfterStart() - ssc.awaitTerminationOrTimeout(timeout) - ssc.stop() - } else { + + // will only be set when a new context is created + var kSparkStreamingSession: KSparkStreamingSession? = null + + val creatingFunc = { val sc = SparkConf() .setAppName(appName) .setMaster(master) @@ -323,16 +308,29 @@ fun withSparkStreaming( .map { (key, value) -> key X value.toString() } .asScalaIterable() ) + val ssc = JavaStreamingContext(sc, batchDuration) - val kSparkStreamingSession = KSparkStreamingSession(ssc) + ssc.checkpoint(checkpointPath) - func(kSparkStreamingSession) - ssc.start() - kSparkStreamingSession.invokeRunAfterStart() + kSparkStreamingSession = KSparkStreamingSession(ssc) + func(kSparkStreamingSession!!) + + ssc + } + + val ssc: JavaStreamingContext = when { + checkpointPath != null -> + JavaStreamingContext.getOrCreate(checkpointPath, creatingFunc, hadoopConf, createOnError) - ssc.awaitTerminationOrTimeout(timeout) - ssc.stop() + else -> creatingFunc() + } + + if (startStreamingContext) { + ssc.start() + kSparkStreamingSession?.invokeRunAfterStart() } + ssc.awaitTerminationOrTimeout(timeout) + ssc.stop() } diff --git a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt index 3f3515ec..686f414f 100644 --- a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt +++ b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt @@ -19,12 +19,16 @@ */ package org.jetbrains.kotlinx.spark.api +import io.kotest.assertions.throwables.shouldThrow +import io.kotest.assertions.timing.eventually import io.kotest.core.spec.style.ShouldSpec import io.kotest.matchers.collections.shouldBeIn import io.kotest.matchers.shouldBe +import kotlinx.coroutines.runBlocking import org.apache.commons.io.FileUtils import org.apache.hadoop.fs.FileSystem import org.apache.spark.SparkConf +import org.apache.spark.SparkException import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.streaming.* import org.apache.spark.streaming.api.java.JavaStreamingContext @@ -32,12 +36,15 @@ import org.apache.spark.util.Utils import org.jetbrains.kotlinx.spark.api.tuples.X import org.jetbrains.kotlinx.spark.api.tuples.component1 import org.jetbrains.kotlinx.spark.api.tuples.component2 +import scala.Option import java.io.File import java.io.Serializable import java.net.ConnectException import java.nio.charset.StandardCharsets import java.util.* import java.util.concurrent.atomic.AtomicBoolean +import kotlin.time.Duration.Companion.seconds +import kotlin.time.ExperimentalTime class StreamingTest : ShouldSpec({ @@ -69,88 +76,30 @@ class StreamingTest : ShouldSpec({ } -// should("checkpoint") { -// -// val emptyDir: File = Files.createTempDir() -// emptyDir.deleteOnExit() -// val contextSuite = StreamingContextSuite() -// val corruptedCheckpointDir: String = contextSuite.createCorruptedCheckpoint() -// val checkpointDir: String = contextSuite.createValidCheckpoint() -// -// // Function to create JavaStreamingContext without any output operations -// // (used to detect the new context) -// -// // Function to create JavaStreamingContext without any output operations -// // (used to detect the new context) -// val newContextCreated = AtomicBoolean(false) -// val creatingFunc: Function0 = { -// newContextCreated.set(true) -// JavaStreamingContext(conf, Seconds.apply(1)) -// } -// -// newContextCreated.set(false) -// ssc = JavaStreamingContext.getOrCreate(emptyDir.absolutePath, creatingFunc) -// Assert.assertTrue("new context not created", newContextCreated.get()) -// ssc.stop() -// -// newContextCreated.set(false) -// ssc = JavaStreamingContext.getOrCreate( -// corruptedCheckpointDir, creatingFunc, -// Configuration(), true -// ) -// Assert.assertTrue("new context not created", newContextCreated.get()) -// ssc.stop() -// -// newContextCreated.set(false) -// ssc = JavaStreamingContext.getOrCreate( -// checkpointDir, creatingFunc, -// Configuration() -// ) -// Assert.assertTrue("old context not recovered", !newContextCreated.get()) -// ssc.stop() -// -// newContextCreated.set(false) -// val sc = JavaSparkContext(conf) -// ssc = JavaStreamingContext.getOrCreate( -// checkpointDir, creatingFunc, -// Configuration() -// ) -// Assert.assertTrue("old context not recovered", !newContextCreated.get()) -// ssc.stop() -// } - should("Work with checkpointpath") { - - - val conf = SparkConf() - .setMaster("local[*]") - .setAppName("Kotlin Spark Sample") - .set("newContext", "true") - - val emptyDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark") + val emptyDir = createTempDir() emptyDir.deleteOnExit() val batchDuration = Durations.seconds(1) val timeout = Durations.seconds(1).milliseconds() + val testDirectory = createTempDir() + testDirectory.deleteOnExit() + val corruptedCheckpointDir = createCorruptedCheckpoint() - val checkpointDir = createValidCheckpoint(conf, batchDuration) val newContextCreated = AtomicBoolean(false) val creatingFun: KSparkStreamingSession.() -> Unit = { -// if (conf == null) conf = ssc.sparkContext().conf - println("created new context") newContextCreated.set(true) -// setRunAfterStart { -// ssc.stop() -// } - } - + // closing statement + ssc.textFileStream(testDirectory.absolutePath).foreachRDD { rdd, _ -> rdd.count() } + } + // fill emptyDir with checkpoint newContextCreated.set(false) withSparkStreaming( batchDuration = batchDuration, @@ -158,82 +107,59 @@ class StreamingTest : ShouldSpec({ props = mapOf("newContext" to true), timeout = timeout, func = creatingFun, - startStreamingContext = false, ) newContextCreated.get() shouldBe true - - + // check that creatingFun isn't executed when checkpoint is present newContextCreated.set(false) withSparkStreaming( batchDuration = batchDuration, - checkpointPath = corruptedCheckpointDir, + checkpointPath = emptyDir.absolutePath, props = mapOf("newContext" to true), timeout = timeout, func = creatingFun, - startStreamingContext = false, - createOnError = true, ) - newContextCreated.get() shouldBe true + newContextCreated.get() shouldBe false + // check that creatingFun is not executed when createOnError = false using corrupted checkpoint newContextCreated.set(false) - withSparkStreaming( - batchDuration = batchDuration, - checkpointPath = checkpointDir, - props = mapOf("newContext" to true), - timeout = timeout, - func = creatingFun, - startStreamingContext = false, - ) + shouldThrow { + withSparkStreaming( + batchDuration = batchDuration, + checkpointPath = corruptedCheckpointDir, + props = mapOf("newContext" to true), + timeout = timeout, + func = creatingFun, + createOnError = false, + ) + } newContextCreated.get() shouldBe false - + // check that creatingFun is executed when createOnError = true using corrupted checkpoint newContextCreated.set(false) -// val sc = JavaSparkContext( -// SparkConf() -// .setAppName("test") -// .setMaster("local[*]") -// ) withSparkStreaming( batchDuration = batchDuration, - checkpointPath = checkpointDir, + checkpointPath = corruptedCheckpointDir, props = mapOf("newContext" to true), timeout = timeout, func = creatingFun, - startStreamingContext = false, + createOnError = true, ) - newContextCreated.get() shouldBe false - // todo do something with checkpoint again, check that it doesn't create new instance - - - // TODO clean up checkpoint + newContextCreated.get() shouldBe true } } }) +private fun createTempDir() = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark") + private fun createCorruptedCheckpoint(): String { - val checkpointDirectory = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark").absolutePath + val checkpointDirectory = createTempDir().absolutePath val fakeCheckpointFile = Checkpoint.checkpointFile(checkpointDirectory, Time(1000)) FileUtils.write(File(fakeCheckpointFile.toString()), "blablabla", StandardCharsets.UTF_8) assert(Checkpoint.getCheckpointFiles(checkpointDirectory, (null as FileSystem?).asOption()).nonEmpty()) return checkpointDirectory } -private fun createValidCheckpoint(conf: SparkConf, batchDuration: Duration): String { - val testDirectory = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark").absolutePath - val checkpointDirectory = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark").absolutePath - val ssc = JavaStreamingContext( - conf.clone().set("someKey", "someValue"), - batchDuration, - ) - - ssc.checkpoint(checkpointDirectory) - ssc.textFileStream(testDirectory).foreachRDD { rdd, _ -> rdd.count() } - ssc.start() - ssc.stop() - - return checkpointDirectory -} class Counter(@Volatile var value: Int) : Serializable diff --git a/pom.xml b/pom.xml index 5d76c6df..acee0ac5 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ pom - 1.5.30 + 1.6.20 1.6.10 0.16.0 4.6.0 From 219b949dba1fd498adacab5a60752c23bb896493 Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Thu, 14 Apr 2022 13:03:40 +0200 Subject: [PATCH 21/44] added conversions for Option(al), State. Added DataStreamWriter forEachBatch helper added KotlinStatefulNetworkCount.kt cleaning up --- examples/pom-3.2_2.12.xml | 8 ++ .../KotlinDirectKafkaWordCount.kt | 7 +- .../KotlinRecoverableNetworkWordCount.kt | 20 ++--- .../streaming/KotlinStatefulNetworkCount.kt | 88 +++++++++++++++++++ .../examples/{ => streaming}/Streaming.kt | 10 ++- kotlin-spark-api/3.2/pom_2.12.xml | 8 ++ .../kotlinx/spark/api/Conversions.kt | 24 ++++- .../kotlinx/spark/api/DataStreamWriter.kt | 23 +++++ .../kotlinx/spark/api/SparkSession.kt | 2 +- .../{Streaming.kt => StreamingKeyValues.kt} | 47 ++-------- .../kotlinx/spark/api/StreamingTest.kt | 20 +---- 11 files changed, 176 insertions(+), 81 deletions(-) rename examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/{ => streaming}/KotlinDirectKafkaWordCount.kt (95%) rename examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/{ => streaming}/KotlinRecoverableNetworkWordCount.kt (95%) create mode 100644 examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/KotlinStatefulNetworkCount.kt rename examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/{ => streaming}/Streaming.kt (88%) create mode 100644 kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/DataStreamWriter.kt rename kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/{Streaming.kt => StreamingKeyValues.kt} (93%) diff --git a/examples/pom-3.2_2.12.xml b/examples/pom-3.2_2.12.xml index 5f214b69..58d9856c 100644 --- a/examples/pom-3.2_2.12.xml +++ b/examples/pom-3.2_2.12.xml @@ -90,6 +90,14 @@ true + + org.apache.maven.plugins + maven-compiler-plugin + + 8 + 8 + + diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/KotlinDirectKafkaWordCount.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/KotlinDirectKafkaWordCount.kt similarity index 95% rename from examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/KotlinDirectKafkaWordCount.kt rename to examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/KotlinDirectKafkaWordCount.kt index 897a9176..476815e2 100644 --- a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/KotlinDirectKafkaWordCount.kt +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/KotlinDirectKafkaWordCount.kt @@ -17,7 +17,7 @@ * limitations under the License. * =LICENSEEND= */ -package org.jetbrains.kotlinx.spark.examples +package org.jetbrains.kotlinx.spark.examples.streaming import org.apache.kafka.clients.consumer.ConsumerConfig.* import org.apache.kafka.clients.consumer.ConsumerRecord @@ -25,13 +25,10 @@ import org.apache.kafka.common.serialization.StringDeserializer import org.apache.spark.streaming.Durations import org.apache.spark.streaming.api.java.JavaDStream import org.apache.spark.streaming.api.java.JavaInputDStream -import org.apache.spark.streaming.api.java.JavaPairDStream import org.apache.spark.streaming.kafka010.ConsumerStrategies import org.apache.spark.streaming.kafka010.KafkaUtils import org.apache.spark.streaming.kafka010.LocationStrategies -import org.jetbrains.kotlinx.spark.api.c import org.jetbrains.kotlinx.spark.api.reduceByKey -import org.jetbrains.kotlinx.spark.api.toTuple import org.jetbrains.kotlinx.spark.api.tuples.* import org.jetbrains.kotlinx.spark.api.withSparkStreaming import scala.Tuple2 @@ -84,7 +81,7 @@ object KotlinDirectKafkaWordCount { val topics: String = args.getOrElse(2) { DEFAULT_TOPIC } // Create context with a 2 seconds batch interval - withSparkStreaming(batchDuration = Durations.seconds(2), appName = "JavaDirectKafkaWordCount") { + withSparkStreaming(batchDuration = Durations.seconds(2), appName = "KotlinDirectKafkaWordCount") { val topicsSet: Set = topics.split(',').toSet() diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/KotlinRecoverableNetworkWordCount.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/KotlinRecoverableNetworkWordCount.kt similarity index 95% rename from examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/KotlinRecoverableNetworkWordCount.kt rename to examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/KotlinRecoverableNetworkWordCount.kt index 7c4873f0..f25571b0 100644 --- a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/KotlinRecoverableNetworkWordCount.kt +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/KotlinRecoverableNetworkWordCount.kt @@ -19,7 +19,7 @@ */ @file:OptIn(ExperimentalTime::class) -package org.jetbrains.kotlinx.spark.examples +package org.jetbrains.kotlinx.spark.examples.streaming import com.google.common.io.Files import org.apache.spark.api.java.JavaSparkContext @@ -29,15 +29,11 @@ import org.apache.spark.streaming.Time import org.apache.spark.util.LongAccumulator import org.jetbrains.kotlinx.spark.api.* import org.jetbrains.kotlinx.spark.api.tuples.* -import scala.Tuple2 import java.io.File -import java.io.Serializable import java.nio.charset.Charset import java.util.regex.Pattern -import kotlin.experimental.ExperimentalTypeInference import kotlin.system.exitProcess import kotlin.time.ExperimentalTime -import kotlin.time.measureTimedValue /** @@ -77,6 +73,8 @@ internal object KotlinDroppedWordsCounter { } /** + * Src: https://github.com/apache/spark/blob/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecoverableNetworkWordCount.java + * * Counts words in text encoded with UTF8 received from the network every second. This example also * shows how to use lazily instantiated singleton instances for Accumulator and Broadcast so that * they can be registered on driver failures. @@ -151,9 +149,6 @@ object KotlinRecoverableNetworkWordCount { } } - - @OptIn(ExperimentalTypeInference::class) - @Suppress("UnstableApiUsage") private fun KSparkStreamingSession.createContext( ip: String, port: Int, @@ -168,14 +163,11 @@ object KotlinRecoverableNetworkWordCount { createNewFile() } - // Create a socket stream on target ip:port and count the // words in input stream of \n delimited text (e.g. generated by 'nc') val lines = ssc.socketTextStream(ip, port) - val words = lines.flatMap { it.split(SPACE).iterator() } - - val wordCounts3 = words + val wordCounts = words .map { t(it, 1) } .reduceByKey { a, b -> a + b } @@ -189,7 +181,7 @@ object KotlinRecoverableNetworkWordCount { println("Context is created and started running!") } - wordCounts3.foreachRDD { rdd, time: Time -> + wordCounts.foreachRDD { rdd, time: Time -> // but in foreachRDD we must obtain this conf from the RDD // like `rdd.context().conf` withSpark(rdd) { @@ -228,6 +220,4 @@ object KotlinRecoverableNetworkWordCount { } } } - - } 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 new file mode 100644 index 00000000..e247a873 --- /dev/null +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/KotlinStatefulNetworkCount.kt @@ -0,0 +1,88 @@ +package org.jetbrains.kotlinx.spark.examples.streaming + +import org.apache.spark.SparkConf +import org.apache.spark.api.java.Optional +import org.apache.spark.api.java.StorageLevels +import org.apache.spark.api.java.function.Function3 +import org.apache.spark.streaming.Durations +import org.apache.spark.streaming.State +import org.apache.spark.streaming.StateSpec +import org.apache.spark.streaming.api.java.* +import org.jetbrains.kotlinx.spark.api.* +import org.jetbrains.kotlinx.spark.api.tuples.X +import org.jetbrains.kotlinx.spark.api.tuples.t +import scala.Tuple2 +import java.util.* +import java.util.regex.Pattern +import kotlin.system.exitProcess + + +/** + * Src: https://github.com/apache/spark/blob/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaStatefulNetworkWordCount.java + * + * Counts words cumulatively in UTF8 encoded, '\n' delimited text received from the network every + * second starting with initial value of word count. + * Usage: JavaStatefulNetworkWordCount + * and describe the TCP server that Spark Streaming would connect to receive + * data. + * + * + * To run this on your local machine, you need to first run a Netcat server + * `$ nc -lk 9999` + * and then run the example + * `$ bin/run-example + * org.apache.spark.examples.streaming.JavaStatefulNetworkWordCount localhost 9999` */ +object KotlinStatefulNetworkCount { + + private val SPACE = Pattern.compile(" ") + + private const val DEFAULT_HOSTNAME = "localhost" + private const val DEFAULT_PORT = "9999" + + @Throws(Exception::class) + @JvmStatic + fun main(args: Array) { + if (args.size < 2 && args.isNotEmpty()) { + System.err.println("Usage: JavaStatefulNetworkWordCount ") + exitProcess(1) + } + + // Create the context with a 1 second batch size + withSparkStreaming( + batchDuration = Durations.seconds(1), + checkpointPath = ".", + appName = "JavaStatefulNetworkWordCount", + ) { + + // Initial state RDD input to mapWithState + val tuples = listOf("hello" X 1, "world" X 1) + val initialRDD = ssc.sparkContext().parallelize(tuples) + + val lines = ssc.socketTextStream( + args.getOrElse(0) { DEFAULT_HOSTNAME }, + args.getOrElse(1) { DEFAULT_PORT }.toInt(), + StorageLevels.MEMORY_AND_DISK_SER_2, + ) + val words = lines.flatMap { it.split(SPACE).iterator() } + + val wordsDstream = words.map { it X 1 } + + // Update the cumulative count function + val mappingFunc = { word: String, one: Optional, state: State -> + val sum = one.getOrElse(0) + state.getOrElse(0) + val output = word X sum + state.update(sum) + output + } + + // DStream made of get cumulative counts that get updated in every batch + val stateDstream = wordsDstream.mapWithState( + StateSpec + .function(mappingFunc) + .initialState(initialRDD.toPairRDD()) + ) + + stateDstream.print() + } + } +} \ No newline at end of file diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Streaming.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/Streaming.kt similarity index 88% rename from examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Streaming.kt rename to examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/Streaming.kt index 07e06af9..e0befd04 100644 --- a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Streaming.kt +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/Streaming.kt @@ -17,7 +17,7 @@ * limitations under the License. * =LICENSEEND= */ -package org.jetbrains.kotlinx.spark.examples +package org.jetbrains.kotlinx.spark.examples.streaming import org.apache.spark.SparkConf import org.apache.spark.sql.Dataset @@ -30,13 +30,17 @@ data class TestRow( val word: String, ) +/** + * To run this on your local machine, you need to first run a Netcat server + * + * `$ nc -lk 9999` + */ fun main() = withSparkStreaming(Durations.seconds(1), timeout = 10_000) { val lines = ssc.socketTextStream("localhost", 9999) val words = lines.flatMap { it.split(" ").iterator() } - - words.foreachRDD { rdd, time -> + words.foreachRDD { rdd, _ -> withSpark(rdd) { val dataframe: Dataset = rdd.map { TestRow(it) }.toDS() diff --git a/kotlin-spark-api/3.2/pom_2.12.xml b/kotlin-spark-api/3.2/pom_2.12.xml index 586a89cc..5e421162 100644 --- a/kotlin-spark-api/3.2/pom_2.12.xml +++ b/kotlin-spark-api/3.2/pom_2.12.xml @@ -184,6 +184,14 @@ org.jacoco jacoco-maven-plugin + + org.apache.maven.plugins + maven-compiler-plugin + + 8 + 8 + + diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt index 135a75ec..35bb8ee8 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt @@ -44,19 +44,39 @@ import scala.collection.mutable.Buffer as ScalaMutableBuffer 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 + +/** Returns state value if it exists, else `null`. */ +fun State.getOrNull(): T? = if (exists()) get() else null + +/** Returns state value if it exists, else [other]. */ +fun State.getOrElse(other: T): T = if (exists()) get() else other /** Converts Scala [Option] to Kotlin nullable. */ fun Option.getOrNull(): T? = getOrElse(null) +/** Get if available else [other]. */ +fun Option.getOrElse(other: T): T = getOrElse { other } + /** Converts nullable value to Scala [Option]. */ -fun T?.asOption(): Option = Option.apply(this) +fun T?.toOption(): Option = Option.apply(this) + +/** Converts Scala [Option] to Java [Optional]. */ +fun Option.toOptional(): Optional = Optional.ofNullable(getOrNull()) + /** Converts [Optional] to Kotlin nullable. */ fun Optional.getOrNull(): T? = orNull() +/** Get if available else [other]. */ +fun Optional.getOrElse(other: T): T = orElse(other) + /** Converts nullable value to [Optional]. */ -fun T?.asOptional(): Optional = Optional.ofNullable(this) +fun T?.toOptional(): Optional = Optional.ofNullable(this) + +/** Converts Java [Optional] to Scala [Option]. */ +fun Optional.toOption(): Option = Option.apply(getOrNull()) /** * TODO test diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/DataStreamWriter.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/DataStreamWriter.kt new file mode 100644 index 00000000..5b6884ba --- /dev/null +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/DataStreamWriter.kt @@ -0,0 +1,23 @@ +package org.jetbrains.kotlinx.spark.api + +import org.apache.spark.api.java.function.VoidFunction2 +import org.apache.spark.sql.Dataset +import org.apache.spark.sql.streaming.DataStreamWriter + +/** + * :: Experimental :: + * + * (Scala-specific) Sets the output of the streaming query to be processed using the provided + * function. This is supported only in the micro-batch execution modes (that is, when the + * trigger is not continuous). In every micro-batch, the provided function will be called in + * every micro-batch with (i) the output rows as a Dataset and (ii) the batch identifier. + * The batchId can be used to deduplicate and transactionally write the output + * (that is, the provided Dataset) to external systems. The output Dataset is guaranteed + * to be exactly the same for the same batchId (assuming all operations are deterministic + * in the query). + * + * @since 2.4.0 + */ +fun DataStreamWriter.forEachBatch( + func: (batch: Dataset, batchId: Long) -> Unit, +): DataStreamWriter = foreachBatch(VoidFunction2(func)) \ No newline at end of file diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt index 1e6e5025..f089b6e5 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt @@ -318,7 +318,7 @@ fun withSparkStreaming( ssc } - val ssc: JavaStreamingContext = when { + val ssc = when { checkpointPath != null -> JavaStreamingContext.getOrCreate(checkpointPath, creatingFunc, hadoopConf, createOnError) diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Streaming.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/StreamingKeyValues.kt similarity index 93% rename from kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Streaming.kt rename to kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/StreamingKeyValues.kt index ea9b4df4..96f6dadf 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Streaming.kt +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/StreamingKeyValues.kt @@ -26,10 +26,8 @@ import org.apache.spark.api.java.Optional import org.apache.spark.streaming.Duration import org.apache.spark.streaming.StateSpec import org.apache.spark.streaming.api.java.JavaDStream -import org.apache.spark.streaming.api.java.JavaDStreamLike import org.apache.spark.streaming.api.java.JavaMapWithStateDStream import org.apache.spark.streaming.api.java.JavaPairDStream -import org.apache.spark.streaming.dstream.DStream import scala.Tuple2 @@ -50,7 +48,6 @@ fun JavaPairRDD.toTupleRDD(): JavaRDD> = * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to * generate the RDDs with `numPartitions` partitions. */ -@JvmName("groupByKeyTuple2") fun JavaDStream>.groupByKey( numPartitions: Int = dstream().ssc().sc().defaultParallelism(), ): JavaDStream>> = @@ -62,7 +59,6 @@ fun JavaDStream>.groupByKey( * Return a new DStream by applying `groupByKey` on each RDD. The supplied * org.apache.spark.Partitioner is used to control the partitioning of each RDD. */ -@JvmName("groupByKeyTuple2") fun JavaDStream>.groupByKey(partitioner: Partitioner): JavaDStream>> = toPairDStream() .groupByKey(partitioner) @@ -73,7 +69,6 @@ fun JavaDStream>.groupByKey(partitioner: Partitioner): JavaD * merged using the supplied reduce function. Hash partitioning is used to generate the RDDs * with `numPartitions` partitions. */ -@JvmName("reduceByKeyTuple2") fun JavaDStream>.reduceByKey( numPartitions: Int = dstream().ssc().sc().defaultParallelism(), reduceFunc: (V, V) -> V, @@ -87,7 +82,6 @@ fun JavaDStream>.reduceByKey( * merged using the supplied reduce function. org.apache.spark.Partitioner is used to control * the partitioning of each RDD. */ -@JvmName("reduceByKeyTuple2") fun JavaDStream>.reduceByKey( partitioner: Partitioner, reduceFunc: (V, V) -> V, @@ -101,7 +95,6 @@ fun JavaDStream>.reduceByKey( * combineByKey for RDDs. Please refer to combineByKey in * org.apache.spark.rdd.PairRDDFunctions in the Spark core documentation for more information. */ -@JvmName("combineByKeyTuple2") fun JavaDStream>.combineByKey( createCombiner: (V) -> C, mergeValue: (C, V) -> C, @@ -125,7 +118,6 @@ fun JavaDStream>.combineByKey( * @param numPartitions number of partitions of each RDD in the new DStream; if not specified * then Spark's default number of partitions will be used */ -@JvmName("groupByKeyAndWindowTuple2") fun JavaDStream>.groupByKeyAndWindow( windowDuration: Duration, slideDuration: Duration = dstream().slideDuration(), @@ -146,7 +138,6 @@ fun JavaDStream>.groupByKeyAndWindow( * @param partitioner partitioner for controlling the partitioning of each RDD in the new * DStream. */ -@JvmName("groupByKeyAndWindowTuple2") fun JavaDStream>.groupByKeyAndWindow( windowDuration: Duration, slideDuration: Duration = dstream().slideDuration(), @@ -168,7 +159,6 @@ fun JavaDStream>.groupByKeyAndWindow( * DStream's batching interval * @param numPartitions number of partitions of each RDD in the new DStream. */ -@JvmName("reduceByKeyAndWindowTuple2") fun JavaDStream>.reduceByKeyAndWindow( windowDuration: Duration, slideDuration: Duration = dstream().slideDuration(), @@ -191,7 +181,6 @@ fun JavaDStream>.reduceByKeyAndWindow( * @param partitioner partitioner for controlling the partitioning of each RDD * in the new DStream. */ -@JvmName("reduceByKeyAndWindowTuple2") fun JavaDStream>.reduceByKeyAndWindow( windowDuration: Duration, slideDuration: Duration = dstream().slideDuration(), @@ -223,7 +212,6 @@ fun JavaDStream>.reduceByKeyAndWindow( * @param filterFunc Optional function to filter expired key-value pairs; * only pairs that satisfy the function are retained */ -@JvmName("reduceByKeyAndWindowTuple2") fun JavaDStream>.reduceByKeyAndWindow( invReduceFunc: (V, V) -> V, windowDuration: Duration, @@ -266,7 +254,6 @@ fun JavaDStream>.reduceByKeyAndWindow( * @param filterFunc Optional function to filter expired key-value pairs; * only pairs that satisfy the function are retained */ -@JvmName("reduceByKeyAndWindowTuple2") fun JavaDStream>.reduceByKeyAndWindow( invReduceFunc: (V, V) -> V, windowDuration: Duration, @@ -298,23 +285,22 @@ fun JavaDStream>.reduceByKeyAndWindow( * as a parameter of type `State` in the mapping function. * * Example of using `mapWithState`: - * {{{ + * ```kotlin * // A mapping function that maintains an integer state and return a String - * def mappingFunction(key: String, value: Option[Int], state: State[Int]): Option[String] = { + * fun mappingFunction(key: String, value: Optional, state: State): Optional { * // Use state.exists(), state.get(), state.update() and state.remove() * // to manage state, and return the necessary string * } * - * val spec = StateSpec.function(mappingFunction).numPartitions(10) + * val spec = StateSpec.function(::mappingFunction).numPartitions(10) * - * val mapWithStateDStream = keyValueDStream.mapWithState[StateType, MappedType](spec) - * }}} + * val mapWithStateDStream = keyValueDStream.mapWithState(spec) + * ``` * * @param spec Specification of this transformation * @tparam StateType Class type of the state data * @tparam MappedType Class type of the mapped data */ -@JvmName("mapWithStateTuple2") fun JavaDStream>.mapWithState( spec: StateSpec, ): JavaMapWithStateDStream = @@ -329,7 +315,6 @@ fun JavaDStream>.mapWithState( * corresponding state key-value pair will be eliminated. * @tparam S State type */ -@JvmName("updateStateByKeyTuple2") fun JavaDStream>.updateStateByKey( numPartitions: Int = dstream().ssc().sc().defaultParallelism(), updateFunc: (List, S?) -> S?, @@ -337,7 +322,7 @@ fun JavaDStream>.updateStateByKey( toPairDStream() .updateStateByKey( { list: List, s: Optional -> - updateFunc(list, s.getOrNull()).asOptional() + updateFunc(list, s.getOrNull()).toOptional() }, numPartitions, ) @@ -356,7 +341,6 @@ fun JavaDStream>.updateStateByKey( * DStream * @tparam S State type */ -@JvmName("updateStateByKeyTuple2") fun JavaDStream>.updateStateByKey( partitioner: Partitioner, updateFunc: (List, S?) -> S?, @@ -364,7 +348,7 @@ fun JavaDStream>.updateStateByKey( toPairDStream() .updateStateByKey( { list: List, s: Optional -> - updateFunc(list, s.getOrNull()).asOptional() + updateFunc(list, s.getOrNull()).toOptional() }, partitioner, ) @@ -381,7 +365,6 @@ fun JavaDStream>.updateStateByKey( * @param initialRDD initial state value of each key. * @tparam S State type */ -@JvmName("updateStateByKeyTuple2") fun JavaDStream>.updateStateByKey( partitioner: Partitioner, initialRDD: JavaRDD>, @@ -390,7 +373,7 @@ fun JavaDStream>.updateStateByKey( toPairDStream() .updateStateByKey( { list: List, s: Optional -> - updateFunc(list, s.getOrNull()).asOptional() + updateFunc(list, s.getOrNull()).toOptional() }, partitioner, initialRDD.toPairRDD(), @@ -402,7 +385,6 @@ fun JavaDStream>.updateStateByKey( * Return a new DStream by applying a map function to the value of each key-value pairs in * 'this' DStream without changing the key. */ -@JvmName("mapValuesTuple2") fun JavaDStream>.mapValues( mapValuesFunc: (V) -> U, ): JavaDStream> = @@ -414,7 +396,6 @@ fun JavaDStream>.mapValues( * Return a new DStream by applying a flatmap function to the value of each key-value pairs in * 'this' DStream without changing the key. */ -@JvmName("flatMapValuesTuple2") fun JavaDStream>.flatMapValues( flatMapValuesFunc: (V) -> Iterator, ): JavaDStream> = @@ -426,7 +407,6 @@ fun JavaDStream>.flatMapValues( * Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream. * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. */ -@JvmName("cogroupTuple2") fun JavaDStream>.cogroup( other: JavaDStream>, numPartitions: Int = dstream().ssc().sc().defaultParallelism(), @@ -443,7 +423,6 @@ fun JavaDStream>.cogroup( * Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream. * The supplied org.apache.spark.Partitioner is used to partition the generated RDDs. */ -@JvmName("cogroupTuple2") fun JavaDStream>.cogroup( other: JavaDStream>, partitioner: Partitioner, @@ -459,7 +438,6 @@ fun JavaDStream>.cogroup( * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream. * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. */ -@JvmName("joinTuple2") fun JavaDStream>.join( other: JavaDStream>, numPartitions: Int = dstream().ssc().sc().defaultParallelism(), @@ -475,7 +453,6 @@ fun JavaDStream>.join( * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream. * The supplied org.apache.spark.Partitioner is used to control the partitioning of each RDD. */ -@JvmName("joinTuple2") fun JavaDStream>.join( other: JavaDStream>, partitioner: Partitioner, @@ -492,7 +469,6 @@ fun JavaDStream>.join( * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` * partitions. */ -@JvmName("leftOuterJoinTuple2") fun JavaDStream>.leftOuterJoin( other: JavaDStream>, numPartitions: Int = dstream().ssc().sc().defaultParallelism(), @@ -509,7 +485,6 @@ fun JavaDStream>.leftOuterJoin( * `other` DStream. The supplied org.apache.spark.Partitioner is used to control * the partitioning of each RDD. */ -@JvmName("leftOuterJoinTuple2") fun JavaDStream>.leftOuterJoin( other: JavaDStream>, partitioner: Partitioner, @@ -526,7 +501,6 @@ fun JavaDStream>.leftOuterJoin( * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` * partitions. */ -@JvmName("rightOuterJoinTuple2") fun JavaDStream>.rightOuterJoin( other: JavaDStream>, numPartitions: Int = dstream().ssc().sc().defaultParallelism(), @@ -543,7 +517,6 @@ fun JavaDStream>.rightOuterJoin( * `other` DStream. The supplied org.apache.spark.Partitioner is used to control * the partitioning of each RDD. */ -@JvmName("rightOuterJoinTuple2") fun JavaDStream>.rightOuterJoin( other: JavaDStream>, partitioner: Partitioner, @@ -560,7 +533,6 @@ fun JavaDStream>.rightOuterJoin( * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` * partitions. */ -@JvmName("fullOuterJoinTuple2") fun JavaDStream>.fullOuterJoin( other: JavaDStream>, numPartitions: Int = dstream().ssc().sc().defaultParallelism(), @@ -577,7 +549,6 @@ fun JavaDStream>.fullOuterJoin( * `other` DStream. The supplied org.apache.spark.Partitioner is used to control * the partitioning of each RDD. */ -@JvmName("fullOuterJoinTuple2") fun JavaDStream>.fullOuterJoin( other: JavaDStream>, partitioner: Partitioner, @@ -593,7 +564,6 @@ fun JavaDStream>.fullOuterJoin( * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix". */ -@JvmName("saveAsHadoopFilesTuple2") fun JavaDStream>.saveAsHadoopFiles( prefix: String, suffix: String, @@ -603,7 +573,6 @@ fun JavaDStream>.saveAsHadoopFiles( * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix". */ -@JvmName("saveAsNewAPIHadoopFilesTuple2") fun JavaDStream>.saveAsNewAPIHadoopFiles( prefix: String, suffix: String, diff --git a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt index 686f414f..d1639e85 100644 --- a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt +++ b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt @@ -20,31 +20,22 @@ package org.jetbrains.kotlinx.spark.api import io.kotest.assertions.throwables.shouldThrow -import io.kotest.assertions.timing.eventually import io.kotest.core.spec.style.ShouldSpec import io.kotest.matchers.collections.shouldBeIn import io.kotest.matchers.shouldBe -import kotlinx.coroutines.runBlocking import org.apache.commons.io.FileUtils import org.apache.hadoop.fs.FileSystem -import org.apache.spark.SparkConf import org.apache.spark.SparkException -import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.streaming.* -import org.apache.spark.streaming.api.java.JavaStreamingContext import org.apache.spark.util.Utils import org.jetbrains.kotlinx.spark.api.tuples.X import org.jetbrains.kotlinx.spark.api.tuples.component1 import org.jetbrains.kotlinx.spark.api.tuples.component2 -import scala.Option import java.io.File import java.io.Serializable -import java.net.ConnectException import java.nio.charset.StandardCharsets import java.util.* import java.util.concurrent.atomic.AtomicBoolean -import kotlin.time.Duration.Companion.seconds -import kotlin.time.ExperimentalTime class StreamingTest : ShouldSpec({ @@ -78,19 +69,15 @@ class StreamingTest : ShouldSpec({ should("Work with checkpointpath") { val emptyDir = createTempDir() - emptyDir.deleteOnExit() + val testDirectory = createTempDir() + val corruptedCheckpointDir = createCorruptedCheckpoint() val batchDuration = Durations.seconds(1) val timeout = Durations.seconds(1).milliseconds() - val testDirectory = createTempDir() - testDirectory.deleteOnExit() - - val corruptedCheckpointDir = createCorruptedCheckpoint() val newContextCreated = AtomicBoolean(false) - val creatingFun: KSparkStreamingSession.() -> Unit = { println("created new context") newContextCreated.set(true) @@ -151,12 +138,13 @@ class StreamingTest : ShouldSpec({ }) private fun createTempDir() = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark") + .apply { deleteOnExit() } private fun createCorruptedCheckpoint(): String { val checkpointDirectory = createTempDir().absolutePath val fakeCheckpointFile = Checkpoint.checkpointFile(checkpointDirectory, Time(1000)) FileUtils.write(File(fakeCheckpointFile.toString()), "blablabla", StandardCharsets.UTF_8) - assert(Checkpoint.getCheckpointFiles(checkpointDirectory, (null as FileSystem?).asOption()).nonEmpty()) + assert(Checkpoint.getCheckpointFiles(checkpointDirectory, (null as FileSystem?).toOption()).nonEmpty()) return checkpointDirectory } From 59f658e080d6d9ccfc98c3d7739d5cc65873904b Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Thu, 14 Apr 2022 13:45:06 +0200 Subject: [PATCH 22/44] added toDataFrame conversions for RDDs added KotlinSqlNetworkWordCount.kt --- .../streaming/KotlinSqlNetworkWordCount.kt | 80 +++++++++++++++++++ .../jetbrains/kotlinx/spark/api/Dataset.kt | 20 ++++- .../kotlinx/spark/api/SparkSession.kt | 13 +++ 3 files changed, 109 insertions(+), 4 deletions(-) create mode 100644 examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/KotlinSqlNetworkWordCount.kt diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/KotlinSqlNetworkWordCount.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/KotlinSqlNetworkWordCount.kt new file mode 100644 index 00000000..2f6aacf8 --- /dev/null +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/KotlinSqlNetworkWordCount.kt @@ -0,0 +1,80 @@ +package org.jetbrains.kotlinx.spark.examples.streaming + +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.api.java.StorageLevels +import org.apache.spark.streaming.Durations +import org.apache.spark.streaming.Time +import org.jetbrains.kotlinx.spark.api.withSparkStreaming +import java.io.Serializable +import java.util.regex.Pattern +import kotlin.system.exitProcess + + +/** + * Use DataFrames and SQL to count words in UTF8 encoded, '\n' delimited text received from the + * network every second. + * + * Usage: KotlinSqlNetworkWordCount + * and describe the TCP server that Spark Streaming would connect to receive data. + * + * To run this on your local machine, you need to first run a Netcat server + * `$ nc -lk 9999` + * and then run the example + * `$ bin/run-example org.apache.spark.examples.streaming.KotlinSqlNetworkWordCount localhost 9999` + */ +object KotlinSqlNetworkWordCount { + + private val SPACE = Pattern.compile(" ") + + private const val DEFAULT_IP = "localhost" + private const val DEFAULT_PORT = "9999" + + @Throws(Exception::class) + @JvmStatic + fun main(args: Array) { + if (args.size < 2 && args.isNotEmpty()) { + System.err.println("Usage: KotlinNetworkWordCount ") + exitProcess(1) + } + + // Create the context with a 1 second batch size + withSparkStreaming( + batchDuration = Durations.seconds(1), + appName = "KotlinSqlNetworkWordCount", + ) { + + + // Create a KotlinReceiverInputDStream on target ip:port and count the + // words in input stream of \n delimited text (e.g. generated by 'nc') + // Note that no duplication in storage level only for running locally. + // Replication necessary in distributed scenario for fault tolerance. + val lines = ssc.socketTextStream( + args.getOrElse(0) { DEFAULT_IP }, + args.getOrElse(1) { DEFAULT_PORT }.toInt(), + StorageLevels.MEMORY_AND_DISK_SER, + ) + val words = lines.flatMap { it.split(SPACE).iterator() } + + // Convert RDDs of the words DStream to DataFrame and run SQL query + words.foreachRDD { rdd: JavaRDD, time: Time -> + withSpark(rdd) { + + // Convert JavaRDD to JavaRDD to DataFrame (Dataset) + val rowRDD = rdd.map(::KotlinRecord) + val wordsDataFrame = rowRDD.toDF() + + // Creates a temporary view using the DataFrame + wordsDataFrame.createOrReplaceTempView("words") + + // Do word count on table using SQL and print it + val wordCountsDataFrame = + spark.sql("select word, count(*) as total from words group by word") + println("========= $time=========") + wordCountsDataFrame.show() + } + } + } + } +} + +data class KotlinRecord(val word: String): Serializable diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Dataset.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Dataset.kt index 9173bf8d..71abb1ee 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Dataset.kt +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Dataset.kt @@ -36,10 +36,7 @@ import org.apache.spark.api.java.function.ForeachPartitionFunction import org.apache.spark.api.java.function.MapFunction import org.apache.spark.api.java.function.ReduceFunction import org.apache.spark.rdd.RDD -import org.apache.spark.sql.Column -import org.apache.spark.sql.Dataset -import org.apache.spark.sql.KeyValueGroupedDataset -import org.apache.spark.sql.TypedColumn +import org.apache.spark.sql.* import org.jetbrains.kotlinx.spark.extensions.KSparkExtensions import scala.Tuple2 import scala.Tuple3 @@ -78,6 +75,21 @@ inline fun RDD.toDS(spark: SparkSession): Dataset = inline fun JavaRDDLike.toDS(spark: SparkSession): Dataset = spark.createDataset(this.rdd(), encoder()) +/** + * Utility method to create Dataset (Dataframe) from JavaRDD. + * NOTE: [T] must be [Serializable]. + */ +inline fun JavaRDDLike.toDF(spark: SparkSession): Dataset = + toDS(spark).toDF() + +/** + * Utility method to create Dataset (Dataframe) from RDD. + * NOTE: [T] must be [Serializable]. + */ +inline fun RDD.toDF(spark: SparkSession): Dataset = + toDS(spark).toDF() + + /** * (Kotlin-specific) * Returns a new Dataset that contains the result of applying [func] to each element. diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt index f089b6e5..27513ffc 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt @@ -36,6 +36,7 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.rdd.RDD import org.apache.spark.sql.Dataset +import org.apache.spark.sql.Row import org.apache.spark.sql.SparkSession.Builder import org.apache.spark.sql.UDFRegistration import org.apache.spark.streaming.Duration @@ -71,6 +72,18 @@ class KSparkSession(val spark: SparkSession) { /** Utility method to create dataset from [JavaRDDLike]. */ inline fun JavaRDDLike.toDS(): Dataset = toDS(spark) + /** + * Utility method to create Dataset (Dataframe) from RDD. + * NOTE: [T] must be [Serializable]. + */ + inline fun RDD.toDF(): Dataset = toDF(spark) + + /** + * Utility method to create Dataset (Dataframe) from JavaRDD. + * NOTE: [T] must be [Serializable]. + */ + inline fun JavaRDDLike.toDF(): Dataset = toDF(spark) + /** * A collection of methods for registering user-defined functions (UDF). * From 547ab1433caccfe34a7eb12a2446ea69f9def0cc Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Thu, 14 Apr 2022 13:52:28 +0200 Subject: [PATCH 23/44] added toDataFrame conversions for RDDs added KotlinSqlNetworkWordCount.kt --- .../examples/streaming/KotlinStatefulNetworkCount.kt | 11 ++++------- .../kotlinx/spark/examples/streaming/Streaming.kt | 3 --- 2 files changed, 4 insertions(+), 10 deletions(-) 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 e247a873..64ad0392 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 @@ -1,18 +1,15 @@ package org.jetbrains.kotlinx.spark.examples.streaming -import org.apache.spark.SparkConf import org.apache.spark.api.java.Optional import org.apache.spark.api.java.StorageLevels -import org.apache.spark.api.java.function.Function3 import org.apache.spark.streaming.Durations import org.apache.spark.streaming.State import org.apache.spark.streaming.StateSpec -import org.apache.spark.streaming.api.java.* -import org.jetbrains.kotlinx.spark.api.* +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.tuples.X -import org.jetbrains.kotlinx.spark.api.tuples.t -import scala.Tuple2 -import java.util.* +import org.jetbrains.kotlinx.spark.api.withSparkStreaming import java.util.regex.Pattern import kotlin.system.exitProcess diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/Streaming.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/Streaming.kt index e0befd04..fb576cbd 100644 --- a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/Streaming.kt +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/Streaming.kt @@ -19,11 +19,8 @@ */ package org.jetbrains.kotlinx.spark.examples.streaming -import org.apache.spark.SparkConf import org.apache.spark.sql.Dataset -import org.apache.spark.streaming.Duration import org.apache.spark.streaming.Durations -import org.apache.spark.streaming.api.java.JavaStreamingContext import org.jetbrains.kotlinx.spark.api.* data class TestRow( From 6a5fce452497e27f37d8b469d730517ed02c318d Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Thu, 14 Apr 2022 15:51:36 +0200 Subject: [PATCH 24/44] more tests --- .../kotlinx/spark/api/StreamingKeyValues.kt | 20 ++++++ .../kotlinx/spark/api/StreamingTest.kt | 63 ++++++++++++++++++- 2 files changed, 80 insertions(+), 3 deletions(-) diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/StreamingKeyValues.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/StreamingKeyValues.kt index 96f6dadf..75c2c23f 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/StreamingKeyValues.kt +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/StreamingKeyValues.kt @@ -19,6 +19,7 @@ */ 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 @@ -90,6 +91,22 @@ fun JavaDStream>.reduceByKey( .reduceByKey(reduceFunc, partitioner) .toTupleDStream() +/** + * Combine elements of each key in DStream's RDDs using custom functions. This is similar to the + * combineByKey for RDDs. Please refer to combineByKey in + * org.apache.spark.rdd.PairRDDFunctions in the Spark core documentation for more information. + */ +fun JavaDStream>.combineByKey( + createCombiner: (V) -> C, + mergeValue: (C, V) -> C, + mergeCombiner: (C, C) -> C, + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), + mapSideCombine: Boolean = true, +): JavaDStream> = + toPairDStream() + .combineByKey(createCombiner, mergeValue, mergeCombiner, HashPartitioner(numPartitions), mapSideCombine) + .toTupleDStream() + /** * Combine elements of each key in DStream's RDDs using custom functions. This is similar to the * combineByKey for RDDs. Please refer to combineByKey in @@ -311,6 +328,7 @@ fun JavaDStream>.mapWithState( * the given function on the previous state of the key and the new values of each key. * In every batch the updateFunc will be called for each state even if there are no new values. * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. + * Note: Needs checkpoint directory to be set. * @param updateFunc State update function. If `this` function returns `null`, then * corresponding state key-value pair will be eliminated. * @tparam S State type @@ -333,6 +351,7 @@ fun JavaDStream>.updateStateByKey( * the given function on the previous state of the key and the new values of each key. * In every batch the updateFunc will be called for each state even if there are no new values. * [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD. + * Note: Needs checkpoint directory to be set. * @param updateFunc State update function. Note, that this function may generate a different * tuple with a different key than the input key. Therefore keys may be removed * or added in this way. It is up to the developer to decide whether to @@ -358,6 +377,7 @@ fun JavaDStream>.updateStateByKey( * Return a new "state" DStream where the state for each key is updated by applying * the given function on the previous state of the key and the new values of the key. * org.apache.spark.Partitioner is used to control the partitioning of each RDD. + * Note: Needs checkpoint directory to be set. * @param updateFunc State update function. If `this` function returns `null`, then * corresponding state key-value pair will be eliminated. * @param partitioner Partitioner for controlling the partitioning of each RDD in the new diff --git a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt index d1639e85..0a88586d 100644 --- a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt +++ b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt @@ -22,15 +22,15 @@ package org.jetbrains.kotlinx.spark.api import io.kotest.assertions.throwables.shouldThrow import io.kotest.core.spec.style.ShouldSpec import io.kotest.matchers.collections.shouldBeIn +import io.kotest.matchers.collections.shouldContainAll import io.kotest.matchers.shouldBe import org.apache.commons.io.FileUtils import org.apache.hadoop.fs.FileSystem import org.apache.spark.SparkException import org.apache.spark.streaming.* import org.apache.spark.util.Utils -import org.jetbrains.kotlinx.spark.api.tuples.X -import org.jetbrains.kotlinx.spark.api.tuples.component1 -import org.jetbrains.kotlinx.spark.api.tuples.component2 +import org.jetbrains.kotlinx.spark.api.tuples.* +import scala.Tuple2 import java.io.File import java.io.Serializable import java.nio.charset.StandardCharsets @@ -134,6 +134,62 @@ class StreamingTest : ShouldSpec({ ) newContextCreated.get() shouldBe true } + + should("Have handy tuple2 functions") { + val input = listOf("aaa", "bbb", "aaa", "ccc") + val result = Result() + + withSparkStreaming(Duration(10), timeout = 1000, checkpointPath = createTempDir().absolutePath) { + + val (resultBroadcast, queue) = withSpark(ssc) { + spark.broadcast(result) X LinkedList(listOf(sc.parallelize(input))) + } + + val inputStream = ssc + + .queueStream(queue) // "aaa", "bbb", "aaa", "ccc" + + .map { it X 1 } // ("aaa", 1), ("bbb", 1), ("aaa", 1), ("ccc", 1) + + .reduceByKey(reduceFunc = Int::plus) // ("aaa", 2), ("bbb", 1), ("ccc", 1) + + .flatMapValues { iterator { yield(it); yield(it) } } // ("aaa", 2), ("aaa", 2), ("bbb", 1), ("bbb", 1), ("ccc", 1), ("ccc", 1) + + .groupByKey() // ("aaa", [2, 2]), ("bbb", [1, 1]), ("ccc", [1, 1]) + + .flatMap { (key, values) -> + values.mapIndexed { i, it -> key X it + i }.iterator() + } // ("aaa", 2), ("aaa", 3), ("bbb", 1), ("bbb", 2), ("ccc", 1), ("ccc", 2) + + .combineByKey( + createCombiner = { listOf(it) }, + mergeValue = { list, int -> + list + int + }, + mergeCombiner = { list1, list2 -> + list1 + list2 + }, + ) // ("aaa", [2, 3]), ("bbb", [1, 2]), ("ccc", [1, 2]) + + + // Note: this will update state inside the checkpoint, which we won't test here for now + .updateStateByKey(numPartitions = 3) { lists, s: Int? -> + (s ?: 0) + lists.sumOf { it.sum() } + } // ("aaa", 5), ("bbb", 3), ("ccc", 3) + + inputStream.foreachRDD { rdd, _ -> + withSpark(rdd) { + rdd.toDS().forEach { + it._1 shouldBeIn input + + resultBroadcast.value.list = resultBroadcast.value.list.plusElement(it) + } + } + } + } + + result.list.shouldContainAll(t("aaa", 5), t("bbb", 3), t("ccc", 3)) + } } }) @@ -151,3 +207,4 @@ private fun createCorruptedCheckpoint(): String { class Counter(@Volatile var value: Int) : Serializable +class Result(@Volatile var list: List> = listOf()) : Serializable \ No newline at end of file From 9449b9f0527584588f874e166e945d5db62dc401 Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Thu, 14 Apr 2022 17:44:33 +0200 Subject: [PATCH 25/44] added kafka test, but issue with kotest extension --- kotlin-spark-api/3.2/pom_2.12.xml | 6 ++ .../kotlinx/spark/api/StreamingTest.kt | 87 +++++++++++++++++++ pom.xml | 1 + 3 files changed, 94 insertions(+) diff --git a/kotlin-spark-api/3.2/pom_2.12.xml b/kotlin-spark-api/3.2/pom_2.12.xml index 5e421162..8a468e34 100644 --- a/kotlin-spark-api/3.2/pom_2.12.xml +++ b/kotlin-spark-api/3.2/pom_2.12.xml @@ -95,6 +95,12 @@ ${kotest-extension-allure.version} test + + io.kotest.extensions + kotest-extensions-embedded-kafka + ${kotest-extensions-embedded-kafka.version} + test + com.beust klaxon diff --git a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt index 0a88586d..70b5d8a9 100644 --- a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt +++ b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt @@ -20,14 +20,26 @@ package org.jetbrains.kotlinx.spark.api import io.kotest.assertions.throwables.shouldThrow +import io.kotest.assertions.timing.eventually import io.kotest.core.spec.style.ShouldSpec +import io.kotest.extensions.embedded.kafka.EmbeddedKafkaListener +import io.kotest.extensions.embedded.kafka.embeddedKafkaListener import io.kotest.matchers.collections.shouldBeIn import io.kotest.matchers.collections.shouldContainAll import io.kotest.matchers.shouldBe import org.apache.commons.io.FileUtils import org.apache.hadoop.fs.FileSystem +import org.apache.kafka.clients.consumer.ConsumerConfig +import org.apache.kafka.clients.consumer.ConsumerRecord +import org.apache.kafka.clients.producer.ProducerRecord +import org.apache.kafka.common.serialization.StringDeserializer import org.apache.spark.SparkException import org.apache.spark.streaming.* +import org.apache.spark.streaming.api.java.JavaDStream +import org.apache.spark.streaming.api.java.JavaInputDStream +import org.apache.spark.streaming.kafka010.ConsumerStrategies +import org.apache.spark.streaming.kafka010.KafkaUtils +import org.apache.spark.streaming.kafka010.LocationStrategies import org.apache.spark.util.Utils import org.jetbrains.kotlinx.spark.api.tuples.* import scala.Tuple2 @@ -36,10 +48,85 @@ import java.io.Serializable import java.nio.charset.StandardCharsets import java.util.* import java.util.concurrent.atomic.AtomicBoolean +import kotlin.time.Duration.Companion.seconds +import java.time.Duration class StreamingTest : ShouldSpec({ + context("streaming") { + + context("kafka") { + val port = 9092 + val broker = "localhost:$port" + val topic1 = "test1" + val topic2 = "test2" + val kafkaListener = EmbeddedKafkaListener(port) + + listener(kafkaListener) + + val producer = kafkaListener.stringStringProducer() + producer.send(ProducerRecord(topic1, "Hello this is a test test test")) + producer.send(ProducerRecord(topic2, "This is also also a test test something")) + producer.close() + + withSparkStreaming( + batchDuration = Durations.seconds(2), + appName = "KotlinDirectKafkaWordCount", + timeout = 1000L, + ) { + + val kafkaParams: Map = mapOf( + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG to broker, + ConsumerConfig.GROUP_ID_CONFIG to "consumer-group", + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG to StringDeserializer::class.java, + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG to StringDeserializer::class.java, + ) + + // Create direct kafka stream with brokers and topics + val messages: JavaInputDStream> = KafkaUtils.createDirectStream( + ssc, + LocationStrategies.PreferConsistent(), + ConsumerStrategies.Subscribe(setOf(topic1, topic2), kafkaParams), + ) + + // Get the lines, split them into words, count the words and print + val lines = messages.map { it.topic() X it.value() } + val words = lines.flatMapValues { it.split(" ").iterator() } + + val wordCounts = words + .map { t(it, 1) } + .reduceByKey { a: Int, b: Int -> a + b } + .map { (tup, counter) -> tup + counter } + + val resultLists = mapOf( + topic1 to listOf( + "Hello" X 1, + "this" X 1, + "is" X 1, + "a" X 1, + "test" X 3, + ), + topic2 to listOf( + "This" X 1, + "is" X 1, + "also" X 2, + "a" X 1, + "test" X 2, + "something" X 1, + ) + ) + + wordCounts.foreachRDD { rdd, _ -> + rdd.foreach { (topic, word, count) -> + t(word, count).shouldBeIn(collection = resultLists[topic]!!) + } + } + + wordCounts.print() + } + } + should("stream") { val input = listOf("aaa", "bbb", "aaa", "ccc") diff --git a/pom.xml b/pom.xml index acee0ac5..33ee181e 100644 --- a/pom.xml +++ b/pom.xml @@ -15,6 +15,7 @@ 0.16.0 4.6.0 1.0.1 + 1.0.6 3.2.1 3.3.1 From 70949ef3846b4fc9e40951870bea3c0f23f15108 Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Tue, 19 Apr 2022 13:40:53 +0200 Subject: [PATCH 26/44] added embedded kafka so kafka tests work --- .../streaming/KotlinSqlNetworkWordCount.kt | 19 +++ .../streaming/KotlinStatefulNetworkCount.kt | 21 ++- kotlin-spark-api/3.2/pom_2.12.xml | 34 ++--- .../kotlinx/spark/api/DataStreamWriter.kt | 21 ++- .../kotlinx/spark/api/StreamingTest.kt | 2 - .../kotlinx/spark/api/kafkaHelper.kt | 128 ++++++++++++++++++ pom.xml | 2 +- 7 files changed, 198 insertions(+), 29 deletions(-) create mode 100644 kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/kafkaHelper.kt diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/KotlinSqlNetworkWordCount.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/KotlinSqlNetworkWordCount.kt index 2f6aacf8..51060a20 100644 --- a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/KotlinSqlNetworkWordCount.kt +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/KotlinSqlNetworkWordCount.kt @@ -1,3 +1,22 @@ +/*- + * =LICENSE= + * Kotlin Spark API: Examples for Spark 3.2+ (Scala 2.12) + * ---------- + * Copyright (C) 2019 - 2022 JetBrains + * ---------- + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * =LICENSEEND= + */ package org.jetbrains.kotlinx.spark.examples.streaming import org.apache.spark.api.java.JavaRDD 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 64ad0392..2c938ead 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 @@ -1,3 +1,22 @@ +/*- + * =LICENSE= + * Kotlin Spark API: Examples for Spark 3.2+ (Scala 2.12) + * ---------- + * Copyright (C) 2019 - 2022 JetBrains + * ---------- + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * =LICENSEEND= + */ package org.jetbrains.kotlinx.spark.examples.streaming import org.apache.spark.api.java.Optional @@ -82,4 +101,4 @@ object KotlinStatefulNetworkCount { stateDstream.print() } } -} \ No newline at end of file +} diff --git a/kotlin-spark-api/3.2/pom_2.12.xml b/kotlin-spark-api/3.2/pom_2.12.xml index 8a468e34..eeea1bff 100644 --- a/kotlin-spark-api/3.2/pom_2.12.xml +++ b/kotlin-spark-api/3.2/pom_2.12.xml @@ -31,7 +31,6 @@ org.jetbrains.kotlinx.spark scala-tuples-in-kotlin - org.apache.spark spark-sql-kafka-0-10_${scala.compat.version} @@ -42,12 +41,7 @@ spark-streaming-kafka-0-10_${scala.compat.version} ${spark3.version} - - org.apache.kafka - kafka-streams-test-utils - 3.1.0 - test - + @@ -62,19 +56,6 @@ ${spark3.version} provided - - - - - - - - - - - - - org.apache.hadoop hadoop-client @@ -96,9 +77,9 @@ test - io.kotest.extensions - kotest-extensions-embedded-kafka - ${kotest-extensions-embedded-kafka.version} + io.github.embeddedkafka + embedded-kafka_${scala.compat.version} + ${embedded-kafka.version} test @@ -119,7 +100,12 @@ ${spark3.version} tests test - + + + org.apache.kafka + kafka-streams-test-utils + 3.1.0 + test diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/DataStreamWriter.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/DataStreamWriter.kt index 5b6884ba..d0c1ece1 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/DataStreamWriter.kt +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/DataStreamWriter.kt @@ -1,3 +1,22 @@ +/*- + * =LICENSE= + * Kotlin Spark API: API for Spark 3.2+ (Scala 2.12) + * ---------- + * Copyright (C) 2019 - 2022 JetBrains + * ---------- + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * =LICENSEEND= + */ package org.jetbrains.kotlinx.spark.api import org.apache.spark.api.java.function.VoidFunction2 @@ -20,4 +39,4 @@ import org.apache.spark.sql.streaming.DataStreamWriter */ fun DataStreamWriter.forEachBatch( func: (batch: Dataset, batchId: Long) -> Unit, -): DataStreamWriter = foreachBatch(VoidFunction2(func)) \ No newline at end of file +): DataStreamWriter = foreachBatch(VoidFunction2(func)) diff --git a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt index 70b5d8a9..8d9bb818 100644 --- a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt +++ b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt @@ -22,8 +22,6 @@ package org.jetbrains.kotlinx.spark.api import io.kotest.assertions.throwables.shouldThrow import io.kotest.assertions.timing.eventually import io.kotest.core.spec.style.ShouldSpec -import io.kotest.extensions.embedded.kafka.EmbeddedKafkaListener -import io.kotest.extensions.embedded.kafka.embeddedKafkaListener import io.kotest.matchers.collections.shouldBeIn import io.kotest.matchers.collections.shouldContainAll import io.kotest.matchers.shouldBe diff --git a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/kafkaHelper.kt b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/kafkaHelper.kt new file mode 100644 index 00000000..a9e32aaf --- /dev/null +++ b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/kafkaHelper.kt @@ -0,0 +1,128 @@ +@file:Suppress("MemberVisibilityCanBePrivate", "BlockingMethodInNonBlockingContext") + +package org.jetbrains.kotlinx.spark.api + +/** + * Source: https://github.com/kotest/kotest-extensions-embedded-kafka + * + */ + +import io.github.embeddedkafka.EmbeddedKafka +import io.github.embeddedkafka.EmbeddedKafkaConfig +import io.kotest.core.listeners.TestListener +import io.kotest.core.spec.Spec +import org.apache.kafka.clients.CommonClientConfigs +import org.apache.kafka.clients.consumer.ConsumerConfig +import org.apache.kafka.clients.consumer.KafkaConsumer +import org.apache.kafka.clients.producer.KafkaProducer +import org.apache.kafka.common.serialization.BytesDeserializer +import org.apache.kafka.common.serialization.BytesSerializer +import org.apache.kafka.common.serialization.StringDeserializer +import org.apache.kafka.common.serialization.StringSerializer +import org.apache.kafka.common.utils.Bytes +import scala.Predef +import java.util.Properties + +val embeddedKafkaListener: EmbeddedKafkaListener = EmbeddedKafkaListener(EmbeddedKafkaConfig.defaultConfig()) + +class EmbeddedKafkaListener( + private val config: EmbeddedKafkaConfig, +) : TestListener { + + constructor(port: Int) : this( + EmbeddedKafkaConfig.apply( + port, + EmbeddedKafkaConfig.defaultZookeeperPort(), + Predef.Map().empty(), + Predef.Map().empty(), + Predef.Map().empty(), + ) + ) + + constructor(kafkaPort: Int, zookeeperPort: Int) : this( + EmbeddedKafkaConfig.apply( + kafkaPort, + zookeeperPort, + Predef.Map().empty(), + Predef.Map().empty(), + Predef.Map().empty(), + ) + ) + + val port: Int = config.kafkaPort() + + val host: String = "127.0.0.1" + + val bootstrapServer = "$host:$port" + + override suspend fun beforeSpec(spec: Spec) { + EmbeddedKafka.start(config) + while (!EmbeddedKafka.isRunning()) { + Thread.sleep(100) + } + } + + override suspend fun afterSpec(spec: Spec) { + EmbeddedKafka.stop() + while (EmbeddedKafka.isRunning()) { + Thread.sleep(100) + } + } + + /** + * Returns a kafka consumer configured with the details of the embedded broker. + */ + fun stringStringConsumer(configure: Properties.() -> Unit = {}): KafkaConsumer { + val props = Properties() + props[CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG] = "$host:$port" + props[ConsumerConfig.GROUP_ID_CONFIG] = "test_consumer_group_" + System.currentTimeMillis() + props[ConsumerConfig.AUTO_OFFSET_RESET_CONFIG] = "earliest" + props.configure() + return KafkaConsumer(props, StringDeserializer(), StringDeserializer()) + } + + /** + * Returns a kafka consumer subscribed to the given topic on the embedded broker. + */ + fun stringStringConsumer(topic: String, configure: Properties.() -> Unit = {}): KafkaConsumer { + val consumer = stringStringConsumer(configure) + consumer.subscribe(listOf(topic)) + return consumer + } + + /** + * Returns a kafka consumer configured with the details of the embedded broker. + */ + fun bytesBytesConsumer(configure: Properties.() -> Unit = {}): KafkaConsumer { + val props = Properties() + props[CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG] = "$host:$port" + props[ConsumerConfig.GROUP_ID_CONFIG] = "test_consumer_group_" + System.currentTimeMillis() + props[ConsumerConfig.AUTO_OFFSET_RESET_CONFIG] = "earliest" + props.configure() + return KafkaConsumer(props, BytesDeserializer(), BytesDeserializer()) + } + + /** + * Returns a kafka consumer subscribed to the given topic on the embedded broker. + */ + fun bytesBytesConsumer(topic: String, configure: Properties.() -> Unit = {}): KafkaConsumer { + val consumer = bytesBytesConsumer(configure) + consumer.subscribe(listOf(topic)) + return consumer + } + + fun bytesBytesProducer(configure: Properties.() -> Unit = {}): KafkaProducer { + val props = Properties() + props[CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG] = "$host:$port" + props.configure() + return KafkaProducer(props, BytesSerializer(), BytesSerializer()) + } + + fun stringStringProducer(configure: Properties.() -> Unit = {}): KafkaProducer { + val props = Properties() + props[CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG] = "$host:$port" + props.configure() + return KafkaProducer(props, StringSerializer(), StringSerializer()) + } +} + diff --git a/pom.xml b/pom.xml index 33ee181e..dd6f3844 100644 --- a/pom.xml +++ b/pom.xml @@ -15,7 +15,7 @@ 0.16.0 4.6.0 1.0.1 - 1.0.6 + 3.1.0 3.2.1 3.3.1 From eaf13cea02b184d18cd3aaa3ff45948fdac3525e Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Tue, 19 Apr 2022 16:26:37 +0200 Subject: [PATCH 27/44] Optional and nullable options for updateStateByKey, changing port for kafka test --- .../kotlinx/spark/api/StreamingKeyValues.kt | 75 +++++++++++++++++++ .../kotlinx/spark/api/StreamingTest.kt | 2 +- .../kotlinx/spark/api/kafkaHelper.kt | 19 +++++ 3 files changed, 95 insertions(+), 1 deletion(-) diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/StreamingKeyValues.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/StreamingKeyValues.kt index 75c2c23f..4a0c6195 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/StreamingKeyValues.kt +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/StreamingKeyValues.kt @@ -333,6 +333,7 @@ fun JavaDStream>.mapWithState( * corresponding state key-value pair will be eliminated. * @tparam S State type */ +@JvmName("updateStateByKeyNullable") fun JavaDStream>.updateStateByKey( numPartitions: Int = dstream().ssc().sc().defaultParallelism(), updateFunc: (List, S?) -> S?, @@ -346,6 +347,28 @@ fun JavaDStream>.updateStateByKey( ) .toTupleDStream() +/** + * Return a new "state" DStream where the state for each key is updated by applying + * the given function on the previous state of the key and the new values of each key. + * In every batch the updateFunc will be called for each state even if there are no new values. + * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. + * Note: Needs checkpoint directory to be set. + * @param updateFunc State update function. If `this` function returns `null`, then + * corresponding state key-value pair will be eliminated. + * @tparam S State type + */ +@JvmName("updateStateByKey") +fun JavaDStream>.updateStateByKey( + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), + updateFunc: (List, Optional) -> Optional, +): JavaDStream> = + toPairDStream() + .updateStateByKey( + updateFunc, + numPartitions, + ) + .toTupleDStream() + /** * Return a new "state" DStream where the state for each key is updated by applying * the given function on the previous state of the key and the new values of each key. @@ -360,6 +383,7 @@ fun JavaDStream>.updateStateByKey( * DStream * @tparam S State type */ +@JvmName("updateStateByKeyNullable") fun JavaDStream>.updateStateByKey( partitioner: Partitioner, updateFunc: (List, S?) -> S?, @@ -373,6 +397,31 @@ fun JavaDStream>.updateStateByKey( ) .toTupleDStream() +/** + * Return a new "state" DStream where the state for each key is updated by applying + * the given function on the previous state of the key and the new values of each key. + * In every batch the updateFunc will be called for each state even if there are no new values. + * [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD. + * Note: Needs checkpoint directory to be set. + * @param updateFunc State update function. Note, that this function may generate a different + * tuple with a different key than the input key. Therefore keys may be removed + * or added in this way. It is up to the developer to decide whether to + * remember the partitioner despite the key being changed. + * @param partitioner Partitioner for controlling the partitioning of each RDD in the new + * DStream + * @tparam S State type + */ +fun JavaDStream>.updateStateByKey( + partitioner: Partitioner, + updateFunc: (List, Optional) -> Optional, +): JavaDStream> = + toPairDStream() + .updateStateByKey( + updateFunc, + partitioner, + ) + .toTupleDStream() + /** * Return a new "state" DStream where the state for each key is updated by applying * the given function on the previous state of the key and the new values of the key. @@ -385,6 +434,7 @@ fun JavaDStream>.updateStateByKey( * @param initialRDD initial state value of each key. * @tparam S State type */ +@JvmName("updateStateByKeyNullable") fun JavaDStream>.updateStateByKey( partitioner: Partitioner, initialRDD: JavaRDD>, @@ -400,6 +450,31 @@ fun JavaDStream>.updateStateByKey( ) .toTupleDStream() +/** + * Return a new "state" DStream where the state for each key is updated by applying + * the given function on the previous state of the key and the new values of the key. + * org.apache.spark.Partitioner is used to control the partitioning of each RDD. + * Note: Needs checkpoint directory to be set. + * @param updateFunc State update function. If `this` function returns `null`, then + * corresponding state key-value pair will be eliminated. + * @param partitioner Partitioner for controlling the partitioning of each RDD in the new + * DStream. + * @param initialRDD initial state value of each key. + * @tparam S State type + */ +fun JavaDStream>.updateStateByKey( + partitioner: Partitioner, + initialRDD: JavaRDD>, + updateFunc: (List, Optional) -> Optional, +): JavaDStream> = + toPairDStream() + .updateStateByKey( + updateFunc, + partitioner, + initialRDD.toPairRDD(), + ) + .toTupleDStream() + /** * Return a new DStream by applying a map function to the value of each key-value pairs in diff --git a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt index 8d9bb818..054628d2 100644 --- a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt +++ b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt @@ -55,7 +55,7 @@ class StreamingTest : ShouldSpec({ context("streaming") { context("kafka") { - val port = 9092 + val port = 6001 val broker = "localhost:$port" val topic1 = "test1" val topic2 = "test2" diff --git a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/kafkaHelper.kt b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/kafkaHelper.kt index a9e32aaf..a9e5c9f9 100644 --- a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/kafkaHelper.kt +++ b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/kafkaHelper.kt @@ -1,3 +1,22 @@ +/*- + * =LICENSE= + * Kotlin Spark API: API for Spark 3.2+ (Scala 2.12) + * ---------- + * Copyright (C) 2019 - 2022 JetBrains + * ---------- + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * =LICENSEEND= + */ @file:Suppress("MemberVisibilityCanBePrivate", "BlockingMethodInNonBlockingContext") package org.jetbrains.kotlinx.spark.api From 8ccf5ea5d9b84c7b64bcc9b7c5834308f27c51af Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Tue, 19 Apr 2022 16:44:22 +0200 Subject: [PATCH 28/44] qodana suggestions --- .../examples/streaming/KotlinRecoverableNetworkWordCount.kt | 2 ++ .../org/jetbrains/kotlinx/spark/api/StreamingKeyValues.kt | 4 +++- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/KotlinRecoverableNetworkWordCount.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/KotlinRecoverableNetworkWordCount.kt index f25571b0..32db58f7 100644 --- a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/KotlinRecoverableNetworkWordCount.kt +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/KotlinRecoverableNetworkWordCount.kt @@ -209,6 +209,8 @@ object KotlinRecoverableNetworkWordCount { println(output) println("Dropped ${droppedWordsCounter.value()} word(s) totally") println("Appending to " + outputFile.absolutePath) + + @Suppress("UnstableApiUsage") Files.append( """ $output diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/StreamingKeyValues.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/StreamingKeyValues.kt index 4a0c6195..8664081b 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/StreamingKeyValues.kt +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/StreamingKeyValues.kt @@ -17,6 +17,8 @@ * limitations under the License. * =LICENSEEND= */ +@file:Suppress("unused") + package org.jetbrains.kotlinx.spark.api import org.apache.spark.HashPartitioner @@ -295,7 +297,7 @@ fun JavaDStream>.reduceByKeyAndWindow( .toTupleDStream() /** - * Return a [MapWithStateDStream] by applying a function to every key-value element of + * Return a [JavaMapWithStateDStream] by applying a function to every key-value element of * `this` stream, while maintaining some state data for each unique key. The mapping function * and other specification (e.g. partitioners, timeouts, initial state data, etc.) of this * transformation can be specified using `StateSpec` class. The state data is accessible in From 8e3b952d063f40fdfa17f59929543e1ab6f1b880 Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Tue, 19 Apr 2022 17:34:24 +0200 Subject: [PATCH 29/44] qodana suggestions --- kotlin-spark-api/3.2/pom_2.12.xml | 5 ----- .../kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt | 2 +- 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/kotlin-spark-api/3.2/pom_2.12.xml b/kotlin-spark-api/3.2/pom_2.12.xml index eeea1bff..f9dc7332 100644 --- a/kotlin-spark-api/3.2/pom_2.12.xml +++ b/kotlin-spark-api/3.2/pom_2.12.xml @@ -31,11 +31,6 @@ org.jetbrains.kotlinx.spark scala-tuples-in-kotlin - - org.apache.spark - spark-sql-kafka-0-10_${scala.compat.version} - ${spark3.version} - org.apache.spark spark-streaming-kafka-0-10_${scala.compat.version} diff --git a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt index 054628d2..8d9bb818 100644 --- a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt +++ b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt @@ -55,7 +55,7 @@ class StreamingTest : ShouldSpec({ context("streaming") { context("kafka") { - val port = 6001 + val port = 9092 val broker = "localhost:$port" val topic1 = "test1" val topic2 = "test2" From 1af4c044a4d6f1e5f415dc40adfae5a546e223f3 Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Tue, 19 Apr 2022 21:41:53 +0200 Subject: [PATCH 30/44] let's see if adding a container does anything --- .github/workflows/build.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index e8024b45..d0d201d4 100644 --- a/.github/workflows/build.yml +++ b/.github/workflows/build.yml @@ -8,6 +8,7 @@ on: jobs: build-scala-12: + container: node:14.16 runs-on: ubuntu-latest steps: From e4d6e2aea155e46c295ed1862a8f8a38489b78aa Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Wed, 20 Apr 2022 14:44:59 +0200 Subject: [PATCH 31/44] changing to ip 0.0.0.0 --- .github/workflows/build.yml | 1 - kotlin-spark-api/3.2/pom_2.12.xml | 8 +- .../kotlinx/spark/api/StreamingTest.kt | 150 +++++++++++------- .../kotlinx/spark/api/kafkaHelper.kt | 3 +- pom.xml | 7 +- scala-tuples-in-kotlin/pom_2.12.xml | 2 +- 6 files changed, 111 insertions(+), 60 deletions(-) diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index d0d201d4..e8024b45 100644 --- a/.github/workflows/build.yml +++ b/.github/workflows/build.yml @@ -8,7 +8,6 @@ on: jobs: build-scala-12: - container: node:14.16 runs-on: ubuntu-latest steps: diff --git a/kotlin-spark-api/3.2/pom_2.12.xml b/kotlin-spark-api/3.2/pom_2.12.xml index f9dc7332..99172895 100644 --- a/kotlin-spark-api/3.2/pom_2.12.xml +++ b/kotlin-spark-api/3.2/pom_2.12.xml @@ -68,7 +68,7 @@ io.kotest.extensions kotest-extensions-allure - ${kotest-extension-allure.version} + ${kotest-extensions-allure.version} test @@ -77,6 +77,12 @@ ${embedded-kafka.version} test + + io.kotest.extensions + kotest-extensions-testcontainers + ${kotest-extensions-testcontainers.version} + test + com.beust klaxon diff --git a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt index 8d9bb818..9b293efa 100644 --- a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt +++ b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt @@ -19,16 +19,24 @@ */ package org.jetbrains.kotlinx.spark.api +import io.kotest.assertions.print.print import io.kotest.assertions.throwables.shouldThrow import io.kotest.assertions.timing.eventually +import io.kotest.core.extensions.install import io.kotest.core.spec.style.ShouldSpec +import io.kotest.extensions.testcontainers.TestContainerExtension +import io.kotest.extensions.testcontainers.kafka.createStringStringConsumer +import io.kotest.extensions.testcontainers.kafka.createStringStringProducer +import io.kotest.extensions.testcontainers.perTest import io.kotest.matchers.collections.shouldBeIn import io.kotest.matchers.collections.shouldContainAll +import io.kotest.matchers.collections.shouldHaveSize import io.kotest.matchers.shouldBe import org.apache.commons.io.FileUtils import org.apache.hadoop.fs.FileSystem import org.apache.kafka.clients.consumer.ConsumerConfig import org.apache.kafka.clients.consumer.ConsumerRecord +import org.apache.kafka.clients.producer.ProducerConfig import org.apache.kafka.clients.producer.ProducerRecord import org.apache.kafka.common.serialization.StringDeserializer import org.apache.spark.SparkException @@ -40,6 +48,8 @@ import org.apache.spark.streaming.kafka010.KafkaUtils import org.apache.spark.streaming.kafka010.LocationStrategies import org.apache.spark.util.Utils import org.jetbrains.kotlinx.spark.api.tuples.* +import org.testcontainers.containers.KafkaContainer +import org.testcontainers.utility.DockerImageName import scala.Tuple2 import java.io.File import java.io.Serializable @@ -55,76 +65,110 @@ class StreamingTest : ShouldSpec({ context("streaming") { context("kafka") { + +// val kafka = install( +// TestContainerExtension(KafkaContainer(DockerImageName.parse("confluentinc/cp-kafka:6.2.1"))) +// ) { +// withEmbeddedZookeeper() +// } +// +// should("support kafka streams") { +// val topic1 = "test1" +// val topic2 = "test2" +// +// val producer = kafka.createStringStringProducer() +// producer.send(ProducerRecord(topic1, "Hello this is a test test test")) +// producer.send(ProducerRecord(topic1, "Hello this is a test test test")) +// producer.send(ProducerRecord(topic1, "Hello this is a test test test")) +// producer.send(ProducerRecord(topic1, "Hello this is a test test test")) +// producer.send(ProducerRecord(topic2, "This is also also a test test something")) +// producer.send(ProducerRecord(topic2, "This is also also a test test something")) +// producer.send(ProducerRecord(topic2, "This is also also a test test something")) +// producer.send(ProducerRecord(topic2, "This is also also a test test something")) +// producer.close() +// +// val consumer = kafka.createStringStringConsumer { +// this[ConsumerConfig.MAX_POLL_RECORDS_CONFIG] = 1 +// } +// +// consumer.subscribe(listOf(topic1)) +// val records = consumer.poll(Duration.ofSeconds(100)) +// records.shouldHaveSize(4) +// records.print() + val port = 9092 val broker = "localhost:$port" val topic1 = "test1" val topic2 = "test2" val kafkaListener = EmbeddedKafkaListener(port) - listener(kafkaListener) - val producer = kafkaListener.stringStringProducer() - producer.send(ProducerRecord(topic1, "Hello this is a test test test")) - producer.send(ProducerRecord(topic2, "This is also also a test test something")) - producer.close() + should("support kafka streams") { + val producer = kafkaListener.stringStringProducer() + producer.send(ProducerRecord(topic1, "Hello this is a test test test")) + producer.send(ProducerRecord(topic2, "This is also also a test test something")) + producer.close() - withSparkStreaming( - batchDuration = Durations.seconds(2), - appName = "KotlinDirectKafkaWordCount", - timeout = 1000L, - ) { - - val kafkaParams: Map = mapOf( - ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG to broker, - ConsumerConfig.GROUP_ID_CONFIG to "consumer-group", - ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG to StringDeserializer::class.java, - ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG to StringDeserializer::class.java, - ) + withSparkStreaming( + batchDuration = Durations.seconds(2), + appName = "KotlinDirectKafkaWordCount", + timeout = 1000L, + ) { + + val kafkaParams: Map = mapOf( + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG to broker, + ConsumerConfig.GROUP_ID_CONFIG to "consumer-group", + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG to StringDeserializer::class.java, + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG to StringDeserializer::class.java, + ) - // Create direct kafka stream with brokers and topics - val messages: JavaInputDStream> = KafkaUtils.createDirectStream( - ssc, - LocationStrategies.PreferConsistent(), - ConsumerStrategies.Subscribe(setOf(topic1, topic2), kafkaParams), - ) + // Create direct kafka stream with brokers and topics + val messages: JavaInputDStream> = KafkaUtils.createDirectStream( + ssc, + LocationStrategies.PreferConsistent(), + ConsumerStrategies.Subscribe(setOf(topic1, topic2), kafkaParams), + ) - // Get the lines, split them into words, count the words and print - val lines = messages.map { it.topic() X it.value() } - val words = lines.flatMapValues { it.split(" ").iterator() } - - val wordCounts = words - .map { t(it, 1) } - .reduceByKey { a: Int, b: Int -> a + b } - .map { (tup, counter) -> tup + counter } - - val resultLists = mapOf( - topic1 to listOf( - "Hello" X 1, - "this" X 1, - "is" X 1, - "a" X 1, - "test" X 3, - ), - topic2 to listOf( - "This" X 1, - "is" X 1, - "also" X 2, - "a" X 1, - "test" X 2, - "something" X 1, + // Get the lines, split them into words, count the words and print + val lines = messages.map { it.topic() X it.value() } + val words = lines.flatMapValues { it.split(" ").iterator() } + + val wordCounts = words + .map { t(it, 1) } + .reduceByKey { a: Int, b: Int -> a + b } + .map { (tup, counter) -> tup + counter } + + val resultLists = mapOf( + topic1 to listOf( + "Hello" X 1, + "this" X 1, + "is" X 1, + "a" X 1, + "test" X 3, + ), + topic2 to listOf( + "This" X 1, + "is" X 1, + "also" X 2, + "a" X 1, + "test" X 2, + "something" X 1, + ) ) - ) - wordCounts.foreachRDD { rdd, _ -> - rdd.foreach { (topic, word, count) -> - t(word, count).shouldBeIn(collection = resultLists[topic]!!) + wordCounts.foreachRDD { rdd, _ -> + rdd.foreach { (topic, word, count) -> + t(word, count).shouldBeIn(collection = resultLists[topic]!!) + } } - } - wordCounts.print() + wordCounts.print() + } } + } + should("stream") { val input = listOf("aaa", "bbb", "aaa", "ccc") diff --git a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/kafkaHelper.kt b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/kafkaHelper.kt index a9e5c9f9..e6e14228 100644 --- a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/kafkaHelper.kt +++ b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/kafkaHelper.kt @@ -70,7 +70,8 @@ class EmbeddedKafkaListener( val port: Int = config.kafkaPort() - val host: String = "127.0.0.1" +// val host: String = "127.0.0.1" + val host: String = "0.0.0.0" val bootstrapServer = "$host:$port" diff --git a/pom.xml b/pom.xml index dd6f3844..8c99ec41 100644 --- a/pom.xml +++ b/pom.xml @@ -12,9 +12,10 @@ 1.6.20 1.6.10 - 0.16.0 - 4.6.0 - 1.0.1 + 0.17.0 + 5.2.3 + 1.1.0 + 1.3.1 3.1.0 3.2.1 3.3.1 diff --git a/scala-tuples-in-kotlin/pom_2.12.xml b/scala-tuples-in-kotlin/pom_2.12.xml index c06aa7ee..cf67af41 100644 --- a/scala-tuples-in-kotlin/pom_2.12.xml +++ b/scala-tuples-in-kotlin/pom_2.12.xml @@ -34,7 +34,7 @@ io.kotest.extensions kotest-extensions-allure - ${kotest-extension-allure.version} + ${kotest-extensions-allure.version} test From 25464d8a9935f489e3723c13da3d7b4d9b8e58b0 Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Wed, 20 Apr 2022 15:13:13 +0200 Subject: [PATCH 32/44] changing to ip localhost --- .../test/kotlin/org/jetbrains/kotlinx/spark/api/kafkaHelper.kt | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/kafkaHelper.kt b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/kafkaHelper.kt index e6e14228..20b7eca6 100644 --- a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/kafkaHelper.kt +++ b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/kafkaHelper.kt @@ -71,7 +71,8 @@ class EmbeddedKafkaListener( val port: Int = config.kafkaPort() // val host: String = "127.0.0.1" - val host: String = "0.0.0.0" +// val host: String = "0.0.0.0" + val host: String = "localhost" val bootstrapServer = "$host:$port" From 9901c5bbed372d690dd36334b04e77225fc0776d Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Wed, 20 Apr 2022 16:35:55 +0200 Subject: [PATCH 33/44] attempt to add exclusion for kafka streaming test for github --- .github/workflows/build.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index e8024b45..fd1587ac 100644 --- a/.github/workflows/build.yml +++ b/.github/workflows/build.yml @@ -25,5 +25,5 @@ jobs: key: ${{ runner.os }}-m2-${{ hashFiles('**/pom.xml') }} restore-keys: ${{ runner.os }}-m2 - name: Build with Maven - run: ./mvnw -B package --file pom.xml -Pscala-2.12 + run: ./mvnw -B package --file pom.xml -Pscala-2.12 -Dkotlin-spark-api-3.2.test.excludes="**/*support kafka streams" # vim: ts=2:sts=2:sw=2:expandtab From 1090e2b2d58be6de9fb6eee8f681ad960bc30aae Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Wed, 20 Apr 2022 16:49:23 +0200 Subject: [PATCH 34/44] attempting to exclude entire file --- .github/workflows/build.yml | 2 +- .../kotlinx/spark/api/KafkaStreamingTest.kt | 140 ++++++++++++++++++ .../kotlinx/spark/api/StreamingTest.kt | 105 ------------- 3 files changed, 141 insertions(+), 106 deletions(-) create mode 100644 kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/KafkaStreamingTest.kt diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index fd1587ac..909c7d16 100644 --- a/.github/workflows/build.yml +++ b/.github/workflows/build.yml @@ -25,5 +25,5 @@ jobs: key: ${{ runner.os }}-m2-${{ hashFiles('**/pom.xml') }} restore-keys: ${{ runner.os }}-m2 - name: Build with Maven - run: ./mvnw -B package --file pom.xml -Pscala-2.12 -Dkotlin-spark-api-3.2.test.excludes="**/*support kafka streams" + run: ./mvnw -B package --file pom.xml -Pscala-2.12 -Dkotlin-spark-api-3.2.test.excludes="**/*KafkaStreamingTest.*" # vim: ts=2:sts=2:sw=2:expandtab diff --git a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/KafkaStreamingTest.kt b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/KafkaStreamingTest.kt new file mode 100644 index 00000000..19bfc17b --- /dev/null +++ b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/KafkaStreamingTest.kt @@ -0,0 +1,140 @@ +/*- + * =LICENSE= + * Kotlin Spark API: API for Spark 3.2+ (Scala 2.12) + * ---------- + * Copyright (C) 2019 - 2022 JetBrains + * ---------- + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * =LICENSEEND= + */ +package org.jetbrains.kotlinx.spark.api + +import io.kotest.core.spec.style.ShouldSpec +import io.kotest.matchers.collections.shouldBeIn +import org.apache.kafka.clients.consumer.ConsumerConfig +import org.apache.kafka.clients.consumer.ConsumerRecord +import org.apache.kafka.clients.producer.ProducerRecord +import org.apache.kafka.common.serialization.StringDeserializer +import org.apache.spark.streaming.Durations +import org.apache.spark.streaming.api.java.JavaInputDStream +import org.apache.spark.streaming.kafka010.ConsumerStrategies +import org.apache.spark.streaming.kafka010.KafkaUtils +import org.apache.spark.streaming.kafka010.LocationStrategies +import org.jetbrains.kotlinx.spark.api.tuples.* +import java.io.Serializable + +class KafkaStreamingTest : ShouldSpec({ + context("kafka") { + +// val kafka = install( +// TestContainerExtension(KafkaContainer(DockerImageName.parse("confluentinc/cp-kafka:6.2.1"))) +// ) { +// withEmbeddedZookeeper() +// } +// +// should("support kafka streams") { +// val topic1 = "test1" +// val topic2 = "test2" +// +// val producer = kafka.createStringStringProducer() +// producer.send(ProducerRecord(topic1, "Hello this is a test test test")) +// producer.send(ProducerRecord(topic1, "Hello this is a test test test")) +// producer.send(ProducerRecord(topic1, "Hello this is a test test test")) +// producer.send(ProducerRecord(topic1, "Hello this is a test test test")) +// producer.send(ProducerRecord(topic2, "This is also also a test test something")) +// producer.send(ProducerRecord(topic2, "This is also also a test test something")) +// producer.send(ProducerRecord(topic2, "This is also also a test test something")) +// producer.send(ProducerRecord(topic2, "This is also also a test test something")) +// producer.close() +// +// val consumer = kafka.createStringStringConsumer { +// this[ConsumerConfig.MAX_POLL_RECORDS_CONFIG] = 1 +// } +// +// consumer.subscribe(listOf(topic1)) +// val records = consumer.poll(Duration.ofSeconds(100)) +// records.shouldHaveSize(4) +// records.print() + + val port = 9092 + val broker = "localhost:$port" + val topic1 = "test1" + val topic2 = "test2" + val kafkaListener = EmbeddedKafkaListener(port) + listener(kafkaListener) + + should("support kafka streams") { + val producer = kafkaListener.stringStringProducer() + producer.send(ProducerRecord(topic1, "Hello this is a test test test")) + producer.send(ProducerRecord(topic2, "This is also also a test test something")) + producer.close() + + withSparkStreaming( + batchDuration = Durations.seconds(2), + appName = "KotlinDirectKafkaWordCount", + timeout = 1000L, + ) { + + val kafkaParams: Map = mapOf( + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG to broker, + ConsumerConfig.GROUP_ID_CONFIG to "consumer-group", + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG to StringDeserializer::class.java, + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG to StringDeserializer::class.java, + ) + + // Create direct kafka stream with brokers and topics + val messages: JavaInputDStream> = KafkaUtils.createDirectStream( + ssc, + LocationStrategies.PreferConsistent(), + ConsumerStrategies.Subscribe(setOf(topic1, topic2), kafkaParams), + ) + + // Get the lines, split them into words, count the words and print + val lines = messages.map { it.topic() X it.value() } + val words = lines.flatMapValues { it.split(" ").iterator() } + + val wordCounts = words + .map { t(it, 1) } + .reduceByKey { a: Int, b: Int -> a + b } + .map { (tup, counter) -> tup + counter } + + val resultLists = mapOf( + topic1 to listOf( + "Hello" X 1, + "this" X 1, + "is" X 1, + "a" X 1, + "test" X 3, + ), + topic2 to listOf( + "This" X 1, + "is" X 1, + "also" X 2, + "a" X 1, + "test" X 2, + "something" X 1, + ) + ) + + wordCounts.foreachRDD { rdd, _ -> + rdd.foreach { (topic, word, count) -> + t(word, count).shouldBeIn(collection = resultLists[topic]!!) + } + } + + wordCounts.print() + } + } + + } +}) diff --git a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt index 9b293efa..64f82365 100644 --- a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt +++ b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt @@ -64,111 +64,6 @@ class StreamingTest : ShouldSpec({ context("streaming") { - context("kafka") { - -// val kafka = install( -// TestContainerExtension(KafkaContainer(DockerImageName.parse("confluentinc/cp-kafka:6.2.1"))) -// ) { -// withEmbeddedZookeeper() -// } -// -// should("support kafka streams") { -// val topic1 = "test1" -// val topic2 = "test2" -// -// val producer = kafka.createStringStringProducer() -// producer.send(ProducerRecord(topic1, "Hello this is a test test test")) -// producer.send(ProducerRecord(topic1, "Hello this is a test test test")) -// producer.send(ProducerRecord(topic1, "Hello this is a test test test")) -// producer.send(ProducerRecord(topic1, "Hello this is a test test test")) -// producer.send(ProducerRecord(topic2, "This is also also a test test something")) -// producer.send(ProducerRecord(topic2, "This is also also a test test something")) -// producer.send(ProducerRecord(topic2, "This is also also a test test something")) -// producer.send(ProducerRecord(topic2, "This is also also a test test something")) -// producer.close() -// -// val consumer = kafka.createStringStringConsumer { -// this[ConsumerConfig.MAX_POLL_RECORDS_CONFIG] = 1 -// } -// -// consumer.subscribe(listOf(topic1)) -// val records = consumer.poll(Duration.ofSeconds(100)) -// records.shouldHaveSize(4) -// records.print() - - val port = 9092 - val broker = "localhost:$port" - val topic1 = "test1" - val topic2 = "test2" - val kafkaListener = EmbeddedKafkaListener(port) - listener(kafkaListener) - - should("support kafka streams") { - val producer = kafkaListener.stringStringProducer() - producer.send(ProducerRecord(topic1, "Hello this is a test test test")) - producer.send(ProducerRecord(topic2, "This is also also a test test something")) - producer.close() - - withSparkStreaming( - batchDuration = Durations.seconds(2), - appName = "KotlinDirectKafkaWordCount", - timeout = 1000L, - ) { - - val kafkaParams: Map = mapOf( - ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG to broker, - ConsumerConfig.GROUP_ID_CONFIG to "consumer-group", - ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG to StringDeserializer::class.java, - ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG to StringDeserializer::class.java, - ) - - // Create direct kafka stream with brokers and topics - val messages: JavaInputDStream> = KafkaUtils.createDirectStream( - ssc, - LocationStrategies.PreferConsistent(), - ConsumerStrategies.Subscribe(setOf(topic1, topic2), kafkaParams), - ) - - // Get the lines, split them into words, count the words and print - val lines = messages.map { it.topic() X it.value() } - val words = lines.flatMapValues { it.split(" ").iterator() } - - val wordCounts = words - .map { t(it, 1) } - .reduceByKey { a: Int, b: Int -> a + b } - .map { (tup, counter) -> tup + counter } - - val resultLists = mapOf( - topic1 to listOf( - "Hello" X 1, - "this" X 1, - "is" X 1, - "a" X 1, - "test" X 3, - ), - topic2 to listOf( - "This" X 1, - "is" X 1, - "also" X 2, - "a" X 1, - "test" X 2, - "something" X 1, - ) - ) - - wordCounts.foreachRDD { rdd, _ -> - rdd.foreach { (topic, word, count) -> - t(word, count).shouldBeIn(collection = resultLists[topic]!!) - } - } - - wordCounts.print() - } - } - - } - - should("stream") { val input = listOf("aaa", "bbb", "aaa", "ccc") From 729279e4e70fdb39c3ec860aaaead49bef908b09 Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Wed, 20 Apr 2022 17:02:29 +0200 Subject: [PATCH 35/44] attempting to exclude entire file --- .github/workflows/build.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index 909c7d16..6b3cbe49 100644 --- a/.github/workflows/build.yml +++ b/.github/workflows/build.yml @@ -25,5 +25,5 @@ jobs: key: ${{ runner.os }}-m2-${{ hashFiles('**/pom.xml') }} restore-keys: ${{ runner.os }}-m2 - name: Build with Maven - run: ./mvnw -B package --file pom.xml -Pscala-2.12 -Dkotlin-spark-api-3.2.test.excludes="**/*KafkaStreamingTest.*" + run: ./mvnw -B package --file pom.xml -Pscala-2.12 -Dkotlin-spark-api-3.2.test.excludes="**/*KafkaStreamingTest*" # vim: ts=2:sts=2:sw=2:expandtab From 9b5c1fb6c2a79e93d9d953183b61c11d6c7224ff Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Wed, 20 Apr 2022 17:14:38 +0200 Subject: [PATCH 36/44] attempting to exclude entire file --- .github/workflows/build.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index 6b3cbe49..db88061f 100644 --- a/.github/workflows/build.yml +++ b/.github/workflows/build.yml @@ -25,5 +25,5 @@ jobs: key: ${{ runner.os }}-m2-${{ hashFiles('**/pom.xml') }} restore-keys: ${{ runner.os }}-m2 - name: Build with Maven - run: ./mvnw -B package --file pom.xml -Pscala-2.12 -Dkotlin-spark-api-3.2.test.excludes="**/*KafkaStreamingTest*" + run: ./mvnw -B package --file pom.xml -Pscala-2.12 -Dkotlin-spark-api-3.2.test.excludes="**KafkaStreamingTest" # vim: ts=2:sts=2:sw=2:expandtab From 290fc9e1842c46932858dde03c14d6485151b02a Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Wed, 20 Apr 2022 17:42:59 +0200 Subject: [PATCH 37/44] attempting to exclude entire file --- .github/workflows/build.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index db88061f..25cc1bb1 100644 --- a/.github/workflows/build.yml +++ b/.github/workflows/build.yml @@ -25,5 +25,5 @@ jobs: key: ${{ runner.os }}-m2-${{ hashFiles('**/pom.xml') }} restore-keys: ${{ runner.os }}-m2 - name: Build with Maven - run: ./mvnw -B package --file pom.xml -Pscala-2.12 -Dkotlin-spark-api-3.2.test.excludes="**KafkaStreamingTest" + run: ./mvnw -B package --file pom.xml -Pscala-2.12 -Dkotlin-spark-api-3.2.test.excludes="org.jetbrains.kotlinx.spark.api.KafkaStreamingTest" # vim: ts=2:sts=2:sw=2:expandtab From fdee3a2e0cd39446205054a703fe4e6ba6a755d7 Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Thu, 21 Apr 2022 11:56:46 +0200 Subject: [PATCH 38/44] exclusion kafka works! --- .github/workflows/build.yml | 2 +- .../kotlinx/spark/api/KafkaStreamingTest.kt | 36 ++++--------------- 2 files changed, 7 insertions(+), 31 deletions(-) diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index 25cc1bb1..5288b2c3 100644 --- a/.github/workflows/build.yml +++ b/.github/workflows/build.yml @@ -25,5 +25,5 @@ jobs: key: ${{ runner.os }}-m2-${{ hashFiles('**/pom.xml') }} restore-keys: ${{ runner.os }}-m2 - name: Build with Maven - run: ./mvnw -B package --file pom.xml -Pscala-2.12 -Dkotlin-spark-api-3.2.test.excludes="org.jetbrains.kotlinx.spark.api.KafkaStreamingTest" + run: ./mvnw -B package --file pom.xml -Pscala-2.12 -Dkotest.tags="!Kafka" # vim: ts=2:sts=2:sw=2:expandtab diff --git a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/KafkaStreamingTest.kt b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/KafkaStreamingTest.kt index 19bfc17b..fe414cc8 100644 --- a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/KafkaStreamingTest.kt +++ b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/KafkaStreamingTest.kt @@ -19,6 +19,7 @@ */ package org.jetbrains.kotlinx.spark.api +import io.kotest.core.Tag import io.kotest.core.spec.style.ShouldSpec import io.kotest.matchers.collections.shouldBeIn import org.apache.kafka.clients.consumer.ConsumerConfig @@ -33,39 +34,14 @@ import org.apache.spark.streaming.kafka010.LocationStrategies import org.jetbrains.kotlinx.spark.api.tuples.* import java.io.Serializable +object Kafka : Tag() + class KafkaStreamingTest : ShouldSpec({ - context("kafka") { -// val kafka = install( -// TestContainerExtension(KafkaContainer(DockerImageName.parse("confluentinc/cp-kafka:6.2.1"))) -// ) { -// withEmbeddedZookeeper() -// } -// -// should("support kafka streams") { -// val topic1 = "test1" -// val topic2 = "test2" -// -// val producer = kafka.createStringStringProducer() -// producer.send(ProducerRecord(topic1, "Hello this is a test test test")) -// producer.send(ProducerRecord(topic1, "Hello this is a test test test")) -// producer.send(ProducerRecord(topic1, "Hello this is a test test test")) -// producer.send(ProducerRecord(topic1, "Hello this is a test test test")) -// producer.send(ProducerRecord(topic2, "This is also also a test test something")) -// producer.send(ProducerRecord(topic2, "This is also also a test test something")) -// producer.send(ProducerRecord(topic2, "This is also also a test test something")) -// producer.send(ProducerRecord(topic2, "This is also also a test test something")) -// producer.close() -// -// val consumer = kafka.createStringStringConsumer { -// this[ConsumerConfig.MAX_POLL_RECORDS_CONFIG] = 1 -// } -// -// consumer.subscribe(listOf(topic1)) -// val records = consumer.poll(Duration.ofSeconds(100)) -// records.shouldHaveSize(4) -// records.print() + // making sure it can be skipped on github actions since it times out + tags(Kafka) + context("kafka") { val port = 9092 val broker = "localhost:$port" val topic1 = "test1" From 4dfb747764622ccc661add0acae17d0008642686 Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Thu, 21 Apr 2022 12:11:39 +0200 Subject: [PATCH 39/44] exclusion kafka works! --- examples/pom-3.2_2.12.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/examples/pom-3.2_2.12.xml b/examples/pom-3.2_2.12.xml index 58d9856c..c3b95b0e 100644 --- a/examples/pom-3.2_2.12.xml +++ b/examples/pom-3.2_2.12.xml @@ -29,6 +29,11 @@ spark-streaming_${scala.compat.version} ${spark3.version} + + org.apache.spark + spark-streaming-kafka-0-10_${scala.compat.version} + ${spark3.version} + From 54b9d1066e5ad75819a52d56ddd73b6c790d8032 Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Thu, 21 Apr 2022 13:18:51 +0200 Subject: [PATCH 40/44] updating readme and example --- README.md | 43 +++++++++++++++++++ .../spark/examples/streaming/Streaming.kt | 18 ++++---- 2 files changed, 52 insertions(+), 9 deletions(-) diff --git a/README.md b/README.md index bd86267f..bd227403 100644 --- a/README.md +++ b/README.md @@ -23,6 +23,7 @@ We have opened a Spark Project Improvement Proposal: [Kotlin support for Apache - [Column infix/operator functions](#column-infixoperator-functions) - [Overload Resolution Ambiguity](#overload-resolution-ambiguity) - [Tuples](#tuples) + - [Streaming](#streaming) - [Examples](#examples) - [Reporting issues/Support](#reporting-issuessupport) - [Code of Conduct](#code-of-conduct) @@ -267,6 +268,48 @@ Finally, all these tuple helper functions are also baked in: - `map` - `cast` +### Streaming + +A popular Spark extension is [Spark Streaming](https://spark.apache.org/docs/latest/streaming-programming-guide.html). +Of course the Kotlin Spark API also introduces a more Kotlin-esque approach to write your streaming programs. +There are examples for use with a checkpoint, Kafka and SQL in the [examples module](examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming). + +We shall also provide a quick example below: +```kotlin +// Automatically provides ssc: JavaStreamingContext which starts and awaits termination or timeout +withSparkStreaming(batchDuration = Durations.seconds(1), timeout = 10_000) { // this: KSparkStreamingSession + + // create input stream for, for instance, Netcat: `$ nc -lk 9999` + val lines: JavaReceiverInputDStream = ssc.socketTextStream("localhost", 9999) + + // split input stream on space + val words: JavaDStream = lines.flatMap { it.split(" ").iterator() } + + // perform action on each formed RDD in the stream + words.foreachRDD { rdd: JavaRDD, _: Time -> + + // to convert the JavaRDD to a Dataset, we need a spark session using the RDD context + withSpark(rdd) { // this: KSparkSession + val dataframe: Dataset = rdd.map { TestRow(word = it) }.toDS() + dataframe + .groupByKey { it.word } + .count() + .show() + // +-----+--------+ + // | key|count(1)| + // +-----+--------+ + // |hello| 1| + // | is| 1| + // | a| 1| + // | this| 1| + // | test| 3| + // +-----+--------+ + } + } +} +``` + + ## Examples For more, check out [examples](https://github.com/JetBrains/kotlin-spark-api/tree/master/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples) module. diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/Streaming.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/Streaming.kt index fb576cbd..85db9775 100644 --- a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/Streaming.kt +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/Streaming.kt @@ -19,8 +19,12 @@ */ package org.jetbrains.kotlinx.spark.examples.streaming +import org.apache.spark.api.java.JavaRDD import org.apache.spark.sql.Dataset import org.apache.spark.streaming.Durations +import org.apache.spark.streaming.Time +import org.apache.spark.streaming.api.java.JavaDStream +import org.apache.spark.streaming.api.java.JavaReceiverInputDStream import org.jetbrains.kotlinx.spark.api.* data class TestRow( @@ -32,22 +36,18 @@ data class TestRow( * * `$ nc -lk 9999` */ -fun main() = withSparkStreaming(Durations.seconds(1), timeout = 10_000) { +fun main() = withSparkStreaming(batchDuration = Durations.seconds(1), timeout = 10_000) { // this: KSparkStreamingSession - val lines = ssc.socketTextStream("localhost", 9999) - val words = lines.flatMap { it.split(" ").iterator() } - - words.foreachRDD { rdd, _ -> - withSpark(rdd) { + val lines: JavaReceiverInputDStream = ssc.socketTextStream("localhost", 9999) + val words: JavaDStream = lines.flatMap { it.split(" ").iterator() } + words.foreachRDD { rdd: JavaRDD, _: Time -> + withSpark(rdd) { // this: KSparkSession val dataframe: Dataset = rdd.map { TestRow(it) }.toDS() - dataframe .groupByKey { it.word } .count() .show() } - } - } \ No newline at end of file From f694d07a0f185cf553c3993e192bce426cb09c02 Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Thu, 21 Apr 2022 16:28:38 +0200 Subject: [PATCH 41/44] attempt to add qodana scan action to github actions --- .github/workflows/build.yml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index 5288b2c3..6e86848d 100644 --- a/.github/workflows/build.yml +++ b/.github/workflows/build.yml @@ -26,4 +26,10 @@ jobs: restore-keys: ${{ runner.os }}-m2 - name: Build with Maven run: ./mvnw -B package --file pom.xml -Pscala-2.12 -Dkotest.tags="!Kafka" + qodana: + runs-on: ubuntu-latest + steps: + - uses: actions/checkout@v3 + - name: 'Qodana Scan' + uses: JetBrains/qodana-action@v5.0.2 # vim: ts=2:sts=2:sw=2:expandtab From 299fb7509c6195665f983c59171898d32df74010 Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Thu, 21 Apr 2022 17:07:02 +0200 Subject: [PATCH 42/44] removed qodana app, let's try --- .github/workflows/build.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index 6e86848d..63d49c25 100644 --- a/.github/workflows/build.yml +++ b/.github/workflows/build.yml @@ -32,4 +32,5 @@ jobs: - uses: actions/checkout@v3 - name: 'Qodana Scan' uses: JetBrains/qodana-action@v5.0.2 + # vim: ts=2:sts=2:sw=2:expandtab From f83727d9a66b171a5ee1a58bd8f5d8c0e95e7f2d Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Thu, 21 Apr 2022 17:33:23 +0200 Subject: [PATCH 43/44] removed todo test and updated unused imports --- .../kotlinx/spark/examples/Broadcasting.kt | 1 - .../org/jetbrains/kotlinx/spark/examples/Main.kt | 5 +++-- .../org/jetbrains/kotlinx/spark/api/Conversions.kt | 14 -------------- 3 files changed, 3 insertions(+), 17 deletions(-) diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Broadcasting.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Broadcasting.kt index 2e5914e3..9612b350 100644 --- a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Broadcasting.kt +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Broadcasting.kt @@ -21,7 +21,6 @@ package org.jetbrains.kotlinx.spark.examples import org.jetbrains.kotlinx.spark.api.broadcast import org.jetbrains.kotlinx.spark.api.map -import org.jetbrains.kotlinx.spark.api.sparkContext import org.jetbrains.kotlinx.spark.api.withSpark import java.io.Serializable diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Main.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Main.kt index 0fc2517f..fc0a2888 100644 --- a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Main.kt +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Main.kt @@ -19,13 +19,14 @@ */ package org.jetbrains.kotlinx.spark.examples -import org.apache.spark.api.java.function.ReduceFunction import org.apache.spark.sql.Dataset import org.jetbrains.kotlinx.spark.api.* import org.jetbrains.kotlinx.spark.api.tuples.* -import scala.* +import scala.Tuple2 +import scala.Tuple3 data class Q(val id: Int, val text: T) + @Suppress("RedundantLambdaArrow", "UsePropertyAccessSyntax") object Main { diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt index 35bb8ee8..034109f2 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt @@ -78,20 +78,6 @@ fun T?.toOptional(): Optional = Optional.ofNullable(this) /** Converts Java [Optional] to Scala [Option]. */ fun Optional.toOption(): Option = Option.apply(getOrNull()) -/** - * TODO test - * 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. - */ -fun fakeClassTag(): ClassTag = ClassTag.AnyRef() as ClassTag - /** * @see JavaConverters.asScalaIterator for more information. */ From 4ece47eafc2aa74a4051afb8fdaecd575d3f0f3f Mon Sep 17 00:00:00 2001 From: Jolanrensen Date: Thu, 21 Apr 2022 17:58:53 +0200 Subject: [PATCH 44/44] last cleanups --- .../kotlinx/spark/api/Conversions.kt | 1 - .../api/{kafkaHelper.kt => KafkaHelper.kt} | 3 +- .../kotlinx/spark/api/KafkaStreamingTest.kt | 2 +- .../kotlinx/spark/api/ProjectConfig.kt | 2 ++ .../kotlinx/spark/api/StreamingTest.kt | 32 +++++-------------- 5 files changed, 12 insertions(+), 28 deletions(-) rename kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/{kafkaHelper.kt => KafkaHelper.kt} (99%) diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt index 034109f2..f8d90fa3 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt +++ b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt @@ -30,7 +30,6 @@ package org.jetbrains.kotlinx.spark.api import org.apache.spark.api.java.Optional import scala.* import scala.collection.JavaConverters -import scala.reflect.ClassTag import java.util.* import java.util.Enumeration import java.util.concurrent.ConcurrentMap diff --git a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/kafkaHelper.kt b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/KafkaHelper.kt similarity index 99% rename from kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/kafkaHelper.kt rename to kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/KafkaHelper.kt index 20b7eca6..6ec5924c 100644 --- a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/kafkaHelper.kt +++ b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/KafkaHelper.kt @@ -23,7 +23,6 @@ package org.jetbrains.kotlinx.spark.api /** * Source: https://github.com/kotest/kotest-extensions-embedded-kafka - * */ import io.github.embeddedkafka.EmbeddedKafka @@ -40,7 +39,7 @@ import org.apache.kafka.common.serialization.StringDeserializer import org.apache.kafka.common.serialization.StringSerializer import org.apache.kafka.common.utils.Bytes import scala.Predef -import java.util.Properties +import java.util.* val embeddedKafkaListener: EmbeddedKafkaListener = EmbeddedKafkaListener(EmbeddedKafkaConfig.defaultConfig()) diff --git a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/KafkaStreamingTest.kt b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/KafkaStreamingTest.kt index fe414cc8..b755b1e6 100644 --- a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/KafkaStreamingTest.kt +++ b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/KafkaStreamingTest.kt @@ -38,7 +38,7 @@ object Kafka : Tag() class KafkaStreamingTest : ShouldSpec({ - // making sure it can be skipped on github actions since it times out + // making sure it can be skipped on Github actions since it times out tags(Kafka) context("kafka") { diff --git a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/ProjectConfig.kt b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/ProjectConfig.kt index 8516ae62..4238cd78 100644 --- a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/ProjectConfig.kt +++ b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/ProjectConfig.kt @@ -25,4 +25,6 @@ import io.kotest.extensions.allure.AllureTestReporter @Suppress("unused") object ProjectConfig : AbstractProjectConfig() { override fun listeners() = super.listeners() + AllureTestReporter(true) + + override fun extensions() = super.extensions() + AllureTestReporter(true) } diff --git a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt index 64f82365..8ae7f5c2 100644 --- a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt +++ b/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt @@ -19,45 +19,29 @@ */ package org.jetbrains.kotlinx.spark.api -import io.kotest.assertions.print.print import io.kotest.assertions.throwables.shouldThrow -import io.kotest.assertions.timing.eventually -import io.kotest.core.extensions.install import io.kotest.core.spec.style.ShouldSpec -import io.kotest.extensions.testcontainers.TestContainerExtension -import io.kotest.extensions.testcontainers.kafka.createStringStringConsumer -import io.kotest.extensions.testcontainers.kafka.createStringStringProducer -import io.kotest.extensions.testcontainers.perTest import io.kotest.matchers.collections.shouldBeIn import io.kotest.matchers.collections.shouldContainAll -import io.kotest.matchers.collections.shouldHaveSize import io.kotest.matchers.shouldBe import org.apache.commons.io.FileUtils import org.apache.hadoop.fs.FileSystem -import org.apache.kafka.clients.consumer.ConsumerConfig -import org.apache.kafka.clients.consumer.ConsumerRecord -import org.apache.kafka.clients.producer.ProducerConfig -import org.apache.kafka.clients.producer.ProducerRecord -import org.apache.kafka.common.serialization.StringDeserializer import org.apache.spark.SparkException -import org.apache.spark.streaming.* -import org.apache.spark.streaming.api.java.JavaDStream -import org.apache.spark.streaming.api.java.JavaInputDStream -import org.apache.spark.streaming.kafka010.ConsumerStrategies -import org.apache.spark.streaming.kafka010.KafkaUtils -import org.apache.spark.streaming.kafka010.LocationStrategies +import org.apache.spark.streaming.Checkpoint +import org.apache.spark.streaming.Duration +import org.apache.spark.streaming.Durations +import org.apache.spark.streaming.Time import org.apache.spark.util.Utils -import org.jetbrains.kotlinx.spark.api.tuples.* -import org.testcontainers.containers.KafkaContainer -import org.testcontainers.utility.DockerImageName +import org.jetbrains.kotlinx.spark.api.tuples.X +import org.jetbrains.kotlinx.spark.api.tuples.component1 +import org.jetbrains.kotlinx.spark.api.tuples.component2 +import org.jetbrains.kotlinx.spark.api.tuples.t import scala.Tuple2 import java.io.File import java.io.Serializable import java.nio.charset.StandardCharsets import java.util.* import java.util.concurrent.atomic.AtomicBoolean -import kotlin.time.Duration.Companion.seconds -import java.time.Duration class StreamingTest : ShouldSpec({