Skip to content

Commit e234f40

Browse files
committed
updating all references to the old encoder<>() function in favor of the new kotlinEncoderFor<>()
1 parent 4896354 commit e234f40

File tree

10 files changed

+1821
-1845
lines changed

10 files changed

+1821
-1845
lines changed

examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/UDFs.kt

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -270,10 +270,10 @@ private object MyAverage : Aggregator<Employee, Average, Double>() {
270270
override fun finish(reduction: Average): Double = reduction.sum.toDouble() / reduction.count
271271

272272
// Specifies the Encoder for the intermediate value type
273-
override fun bufferEncoder(): Encoder<Average> = encoder()
273+
override fun bufferEncoder(): Encoder<Average> = kotlinEncoderFor()
274274

275275
// Specifies the Encoder for the final output value type
276-
override fun outputEncoder(): Encoder<Double> = encoder()
276+
override fun outputEncoder(): Encoder<Double> = kotlinEncoderFor()
277277

278278
}
279279

kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Column.kt

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -443,7 +443,7 @@ operator fun Column.get(key: Any): Column = getItem(key)
443443
* @see typed
444444
*/
445445
@Suppress("UNCHECKED_CAST")
446-
inline fun <DsType, reified U> Column.`as`(): TypedColumn<DsType, U> = `as`(encoder<U>()) as TypedColumn<DsType, U>
446+
inline fun <DsType, reified U> Column.`as`(): TypedColumn<DsType, U> = `as`(kotlinEncoderFor<U>()) as TypedColumn<DsType, U>
447447

448448
/**
449449
* Provides a type hint about the expected return value of this column. This information can
@@ -458,7 +458,7 @@ inline fun <DsType, reified U> Column.`as`(): TypedColumn<DsType, U> = `as`(enco
458458
* @see typed
459459
*/
460460
@Suppress("UNCHECKED_CAST")
461-
inline fun <DsType, reified U> TypedColumn<DsType, *>.`as`(): TypedColumn<DsType, U> = `as`(encoder<U>()) as TypedColumn<DsType, U>
461+
inline fun <DsType, reified U> TypedColumn<DsType, *>.`as`(): TypedColumn<DsType, U> = `as`(kotlinEncoderFor<U>()) as TypedColumn<DsType, U>
462462

463463
/**
464464
* Provides a type hint about the expected return value of this column. This information can

kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Dataset.kt

Lines changed: 21 additions & 48 deletions
Original file line numberDiff line numberDiff line change
@@ -37,7 +37,6 @@ import org.apache.spark.api.java.function.MapFunction
3737
import org.apache.spark.api.java.function.ReduceFunction
3838
import org.apache.spark.rdd.RDD
3939
import org.apache.spark.sql.*
40-
import org.jetbrains.kotlinx.spark.extensions.KSparkExtensions
4140
import scala.Tuple2
4241
import scala.Tuple3
4342
import scala.Tuple4
@@ -49,7 +48,7 @@ import kotlin.reflect.KProperty1
4948
* Utility method to create dataset from list
5049
*/
5150
inline fun <reified T> SparkSession.toDS(list: List<T>): Dataset<T> =
52-
createDataset(list, encoder<T>())
51+
createDataset(list, kotlinEncoderFor<T>())
5352

5453
/**
5554
* Utility method to create dataframe from list
@@ -61,26 +60,26 @@ inline fun <reified T> SparkSession.toDF(list: List<T>, vararg colNames: String)
6160
* Utility method to create dataset from *array or vararg arguments
6261
*/
6362
inline fun <reified T> SparkSession.dsOf(vararg t: T): Dataset<T> =
64-
createDataset(t.toList(), encoder<T>())
63+
createDataset(t.toList(), kotlinEncoderFor<T>())
6564

6665
/**
6766
* Utility method to create dataframe from *array or vararg arguments
6867
*/
6968
inline fun <reified T> SparkSession.dfOf(vararg t: T): Dataset<Row> =
70-
createDataset(t.toList(), encoder<T>()).toDF()
69+
createDataset(t.toList(), kotlinEncoderFor<T>()).toDF()
7170

7271
/**
7372
* Utility method to create dataframe from *array or vararg arguments with given column names
7473
*/
7574
inline fun <reified T> SparkSession.dfOf(colNames: Array<String>, vararg t: T): Dataset<Row> =
76-
createDataset(t.toList(), encoder<T>())
75+
createDataset(t.toList(), kotlinEncoderFor<T>())
7776
.run { if (colNames.isEmpty()) toDF() else toDF(*colNames) }
7877

7978
/**
8079
* Utility method to create dataset from list
8180
*/
8281
inline fun <reified T> List<T>.toDS(spark: SparkSession): Dataset<T> =
83-
spark.createDataset(this, encoder<T>())
82+
spark.createDataset(this, kotlinEncoderFor<T>())
8483

8584
/**
8685
* Utility method to create dataframe from list
@@ -104,13 +103,13 @@ inline fun <reified T> Array<T>.toDF(spark: SparkSession, vararg colNames: Strin
104103
* Utility method to create dataset from RDD
105104
*/
106105
inline fun <reified T> RDD<T>.toDS(spark: SparkSession): Dataset<T> =
107-
spark.createDataset(this, encoder<T>())
106+
spark.createDataset(this, kotlinEncoderFor<T>())
108107

109108
/**
110109
* Utility method to create dataset from JavaRDD
111110
*/
112111
inline fun <reified T> JavaRDDLike<T, *>.toDS(spark: SparkSession): Dataset<T> =
113-
spark.createDataset(this.rdd(), encoder<T>())
112+
spark.createDataset(this.rdd(), kotlinEncoderFor<T>())
114113

115114
/**
116115
* Utility method to create Dataset<Row> (Dataframe) from JavaRDD.
@@ -132,37 +131,37 @@ inline fun <reified T> RDD<T>.toDF(spark: SparkSession, vararg colNames: String)
132131
* Returns a new Dataset that contains the result of applying [func] to each element.
133132
*/
134133
inline fun <reified T, reified R> Dataset<T>.map(noinline func: (T) -> R): Dataset<R> =
135-
map(MapFunction(func), encoder<R>())
134+
map(MapFunction(func), kotlinEncoderFor<R>())
136135

137136
/**
138137
* (Kotlin-specific)
139138
* Returns a new Dataset by first applying a function to all elements of this Dataset,
140139
* and then flattening the results.
141140
*/
142141
inline fun <T, reified R> Dataset<T>.flatMap(noinline func: (T) -> Iterator<R>): Dataset<R> =
143-
flatMap(func, encoder<R>())
142+
flatMap(func, kotlinEncoderFor<R>())
144143

145144
/**
146145
* (Kotlin-specific)
147146
* Returns a new Dataset by flattening. This means that a Dataset of an iterable such as
148147
* `listOf(listOf(1, 2, 3), listOf(4, 5, 6))` will be flattened to a Dataset of `listOf(1, 2, 3, 4, 5, 6)`.
149148
*/
150149
inline fun <reified T, I : Iterable<T>> Dataset<I>.flatten(): Dataset<T> =
151-
flatMap(FlatMapFunction { it.iterator() }, encoder<T>())
150+
flatMap(FlatMapFunction { it.iterator() }, kotlinEncoderFor<T>())
152151

153152
/**
154153
* (Kotlin-specific)
155154
* Returns a [KeyValueGroupedDataset] where the data is grouped by the given key [func].
156155
*/
157156
inline fun <T, reified R> Dataset<T>.groupByKey(noinline func: (T) -> R): KeyValueGroupedDataset<R, T> =
158-
groupByKey(MapFunction(func), encoder<R>())
157+
groupByKey(MapFunction(func), kotlinEncoderFor<R>())
159158

160159
/**
161160
* (Kotlin-specific)
162161
* Returns a new Dataset that contains the result of applying [func] to each partition.
163162
*/
164163
inline fun <T, reified R> Dataset<T>.mapPartitions(noinline func: (Iterator<T>) -> Iterator<R>): Dataset<R> =
165-
mapPartitions(func, encoder<R>())
164+
mapPartitions(func, kotlinEncoderFor<R>())
166165

167166
/**
168167
* (Kotlin-specific)
@@ -193,15 +192,6 @@ inline fun <reified T1, T2> Dataset<Tuple2<T1, T2>>.takeKeys(): Dataset<T1> = ma
193192
*/
194193
inline fun <reified T1, T2> Dataset<Pair<T1, T2>>.takeKeys(): Dataset<T1> = map { it.first }
195194

196-
/**
197-
* (Kotlin-specific)
198-
* Maps the Dataset to only retain the "keys" or [Arity2._1] values.
199-
*/
200-
@Suppress("DEPRECATION")
201-
@JvmName("takeKeysArity2")
202-
@Deprecated("Use Scala tuples instead.", ReplaceWith(""))
203-
inline fun <reified T1, T2> Dataset<Arity2<T1, T2>>.takeKeys(): Dataset<T1> = map { it._1 }
204-
205195
/**
206196
* (Kotlin-specific)
207197
* Maps the Dataset to only retain the "values" or [Tuple2._2] values.
@@ -215,22 +205,13 @@ inline fun <T1, reified T2> Dataset<Tuple2<T1, T2>>.takeValues(): Dataset<T2> =
215205
*/
216206
inline fun <T1, reified T2> Dataset<Pair<T1, T2>>.takeValues(): Dataset<T2> = map { it.second }
217207

218-
/**
219-
* (Kotlin-specific)
220-
* Maps the Dataset to only retain the "values" or [Arity2._2] values.
221-
*/
222-
@Suppress("DEPRECATION")
223-
@JvmName("takeValuesArity2")
224-
@Deprecated("Use Scala tuples instead.", ReplaceWith(""))
225-
inline fun <T1, reified T2> Dataset<Arity2<T1, T2>>.takeValues(): Dataset<T2> = map { it._2 }
226-
227208
/** DEPRECATED: Use [as] or [to] for this. */
228209
@Deprecated(
229210
message = "Deprecated, since we already have `as`() and to().",
230211
replaceWith = ReplaceWith("this.to<R>()"),
231212
level = DeprecationLevel.ERROR,
232213
)
233-
inline fun <T, reified R> Dataset<T>.downcast(): Dataset<R> = `as`(encoder<R>())
214+
inline fun <T, reified R> Dataset<T>.downcast(): Dataset<R> = `as`(kotlinEncoderFor<R>())
234215

235216
/**
236217
* (Kotlin-specific)
@@ -252,7 +233,7 @@ inline fun <T, reified R> Dataset<T>.downcast(): Dataset<R> = `as`(encoder<R>())
252233
*
253234
* @see to as alias for [as]
254235
*/
255-
inline fun <reified R> Dataset<*>.`as`(): Dataset<R> = `as`(encoder<R>())
236+
inline fun <reified R> Dataset<*>.`as`(): Dataset<R> = `as`(kotlinEncoderFor<R>())
256237

257238
/**
258239
* (Kotlin-specific)
@@ -274,7 +255,7 @@ inline fun <reified R> Dataset<*>.`as`(): Dataset<R> = `as`(encoder<R>())
274255
*
275256
* @see as as alias for [to]
276257
*/
277-
inline fun <reified R> Dataset<*>.to(): Dataset<R> = `as`(encoder<R>())
258+
inline fun <reified R> Dataset<*>.to(): Dataset<R> = `as`(kotlinEncoderFor<R>())
278259

279260
/**
280261
* (Kotlin-specific)
@@ -292,12 +273,16 @@ inline fun <reified T> Dataset<T>.forEachPartition(noinline func: (Iterator<T>)
292273
/**
293274
* It's hard to call `Dataset.debugCodegen` from kotlin, so here is utility for that
294275
*/
295-
fun <T> Dataset<T>.debugCodegen(): Dataset<T> = also { KSparkExtensions.debugCodegen(it) }
276+
fun <T> Dataset<T>.debugCodegen(): Dataset<T> = also {
277+
org.apache.spark.sql.execution.debug.`package$`.`MODULE$`.DebugQuery(it).debugCodegen()
278+
}
296279

297280
/**
298281
* It's hard to call `Dataset.debug` from kotlin, so here is utility for that
299282
*/
300-
fun <T> Dataset<T>.debug(): Dataset<T> = also { KSparkExtensions.debug(it) }
283+
fun <T> Dataset<T>.debug(): Dataset<T> = also {
284+
org.apache.spark.sql.execution.debug.`package$`.`MODULE$`.DebugQuery(it).debug()
285+
}
301286

302287

303288
/**
@@ -370,18 +355,6 @@ fun <T1, T2> Dataset<Tuple2<T1, T2>>.sortByKey(): Dataset<Tuple2<T1, T2>> = sort
370355
@JvmName("sortByTuple2Value")
371356
fun <T1, T2> Dataset<Tuple2<T1, T2>>.sortByValue(): Dataset<Tuple2<T1, T2>> = sort("_2")
372357

373-
/** Returns a dataset sorted by the first (`_1`) value of each [Arity2] inside. */
374-
@Suppress("DEPRECATION")
375-
@Deprecated("Use Scala tuples instead.", ReplaceWith(""))
376-
@JvmName("sortByArity2Key")
377-
fun <T1, T2> Dataset<Arity2<T1, T2>>.sortByKey(): Dataset<Arity2<T1, T2>> = sort("_1")
378-
379-
/** Returns a dataset sorted by the second (`_2`) value of each [Arity2] inside. */
380-
@Suppress("DEPRECATION")
381-
@Deprecated("Use Scala tuples instead.", ReplaceWith(""))
382-
@JvmName("sortByArity2Value")
383-
fun <T1, T2> Dataset<Arity2<T1, T2>>.sortByValue(): Dataset<Arity2<T1, T2>> = sort("_2")
384-
385358
/** Returns a dataset sorted by the first (`first`) value of each [Pair] inside. */
386359
@JvmName("sortByPairKey")
387360
fun <T1, T2> Dataset<Pair<T1, T2>>.sortByKey(): Dataset<Pair<T1, T2>> = sort("first")

kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/KeyValueGroupedDataset.kt

Lines changed: 10 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -51,7 +51,7 @@ import scala.Tuple2
5151
* ```
5252
*/
5353
inline fun <KEY, VALUE, reified R> KeyValueGroupedDataset<KEY, VALUE>.mapValues(noinline func: (VALUE) -> R): KeyValueGroupedDataset<KEY, R> =
54-
mapValues(MapFunction(func), encoder<R>())
54+
mapValues(MapFunction(func), kotlinEncoderFor<R>())
5555

5656
/**
5757
* (Kotlin-specific)
@@ -70,7 +70,7 @@ inline fun <KEY, VALUE, reified R> KeyValueGroupedDataset<KEY, VALUE>.mapValues(
7070
* constraints of their cluster.
7171
*/
7272
inline fun <KEY, VALUE, reified R> KeyValueGroupedDataset<KEY, VALUE>.mapGroups(noinline func: (KEY, Iterator<VALUE>) -> R): Dataset<R> =
73-
mapGroups(MapGroupsFunction(func), encoder<R>())
73+
mapGroups(MapGroupsFunction(func), kotlinEncoderFor<R>())
7474

7575
/**
7676
* (Kotlin-specific)
@@ -104,7 +104,7 @@ inline fun <K, V, reified U> KeyValueGroupedDataset<K, V>.flatMapGroups(
104104
noinline func: (key: K, values: Iterator<V>) -> Iterator<U>,
105105
): Dataset<U> = flatMapGroups(
106106
FlatMapGroupsFunction(func),
107-
encoder<U>(),
107+
kotlinEncoderFor<U>(),
108108
)
109109

110110

@@ -127,8 +127,8 @@ inline fun <K, V, reified S, reified U> KeyValueGroupedDataset<K, V>.mapGroupsWi
127127
noinline func: (key: K, values: Iterator<V>, state: GroupState<S>) -> U,
128128
): Dataset<U> = mapGroupsWithState(
129129
MapGroupsWithStateFunction(func),
130-
encoder<S>(),
131-
encoder<U>(),
130+
kotlinEncoderFor<S>(),
131+
kotlinEncoderFor<U>(),
132132
)
133133

134134
/**
@@ -152,8 +152,8 @@ inline fun <K, V, reified S, reified U> KeyValueGroupedDataset<K, V>.mapGroupsWi
152152
noinline func: (key: K, values: Iterator<V>, state: GroupState<S>) -> U,
153153
): Dataset<U> = mapGroupsWithState(
154154
MapGroupsWithStateFunction(func),
155-
encoder<S>(),
156-
encoder<U>(),
155+
kotlinEncoderFor<S>(),
156+
kotlinEncoderFor<U>(),
157157
timeoutConf,
158158
)
159159

@@ -181,8 +181,8 @@ inline fun <K, V, reified S, reified U> KeyValueGroupedDataset<K, V>.flatMapGrou
181181
): Dataset<U> = flatMapGroupsWithState(
182182
FlatMapGroupsWithStateFunction(func),
183183
outputMode,
184-
encoder<S>(),
185-
encoder<U>(),
184+
kotlinEncoderFor<S>(),
185+
kotlinEncoderFor<U>(),
186186
timeoutConf,
187187
)
188188

@@ -199,5 +199,5 @@ inline fun <K, V, U, reified R> KeyValueGroupedDataset<K, V>.cogroup(
199199
): Dataset<R> = cogroup(
200200
other,
201201
CoGroupFunction(func),
202-
encoder<R>(),
202+
kotlinEncoderFor<R>(),
203203
)

kotlin-spark-api/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt

Lines changed: 17 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -34,7 +34,6 @@ import org.apache.spark.api.java.JavaRDD
3434
import org.apache.spark.api.java.JavaRDDLike
3535
import org.apache.spark.api.java.JavaSparkContext
3636
import org.apache.spark.broadcast.Broadcast
37-
import org.apache.spark.deploy.SparkHadoopUtil
3837
import org.apache.spark.rdd.RDD
3938
import org.apache.spark.sql.Dataset
4039
import org.apache.spark.sql.Row
@@ -45,7 +44,6 @@ import org.apache.spark.streaming.Durations
4544
import org.apache.spark.streaming.api.java.JavaStreamingContext
4645
import org.jetbrains.kotlinx.spark.api.SparkLogLevel.ERROR
4746
import org.jetbrains.kotlinx.spark.api.tuples.*
48-
import org.jetbrains.kotlinx.spark.extensions.KSparkExtensions
4947
import java.io.Serializable
5048

5149
/**
@@ -76,7 +74,7 @@ class KSparkSession(val spark: SparkSession) {
7674
inline fun <reified T> dsOf(vararg arg: T): Dataset<T> = spark.dsOf(*arg)
7775

7876
/** Creates new empty dataset of type [T]. */
79-
inline fun <reified T> emptyDataset(): Dataset<T> = spark.emptyDataset(encoder<T>())
77+
inline fun <reified T> emptyDataset(): Dataset<T> = spark.emptyDataset(kotlinEncoderFor<T>())
8078

8179
/** Utility method to create dataframe from *array or vararg arguments */
8280
inline fun <reified T> dfOf(vararg arg: T): Dataset<Row> = spark.dfOf(*arg)
@@ -227,7 +225,7 @@ enum class SparkLogLevel {
227225
* Returns the Spark context associated with this Spark session.
228226
*/
229227
val SparkSession.sparkContext: SparkContext
230-
get() = KSparkExtensions.sparkContext(this)
228+
get() = sparkContext()
231229

232230
/**
233231
* Wrapper for spark creation which allows setting different spark params.
@@ -339,7 +337,7 @@ inline fun withSpark(sparkConf: SparkConf, logLevel: SparkLogLevel = ERROR, func
339337
fun withSparkStreaming(
340338
batchDuration: Duration = Durations.seconds(1L),
341339
checkpointPath: String? = null,
342-
hadoopConf: Configuration = SparkHadoopUtil.get().conf(),
340+
hadoopConf: Configuration = getDefaultHadoopConf(),
343341
createOnError: Boolean = false,
344342
props: Map<String, Any> = emptyMap(),
345343
master: String = SparkConf().get("spark.master", "local[*]"),
@@ -386,6 +384,18 @@ fun withSparkStreaming(
386384
ssc.stop()
387385
}
388386

387+
// calling org.apache.spark.deploy.`SparkHadoopUtil$`.`MODULE$`.get().conf()
388+
private fun getDefaultHadoopConf(): Configuration {
389+
val klass = Class.forName("org.apache.spark.deploy.SparkHadoopUtil$")
390+
val moduleField = klass.getField("MODULE$").also { it.isAccessible = true }
391+
val module = moduleField.get(null)
392+
val getMethod = klass.getMethod("get").also { it.isAccessible = true }
393+
val sparkHadoopUtil = getMethod.invoke(module)
394+
val confMethod = sparkHadoopUtil.javaClass.getMethod("conf").also { it.isAccessible = true }
395+
val conf = confMethod.invoke(sparkHadoopUtil) as Configuration
396+
397+
return conf
398+
}
389399

390400
/**
391401
* Broadcast a read-only variable to the cluster, returning a
@@ -396,7 +406,7 @@ fun withSparkStreaming(
396406
* @return `Broadcast` object, a read-only variable cached on each machine
397407
*/
398408
inline fun <reified T> SparkSession.broadcast(value: T): Broadcast<T> = try {
399-
sparkContext.broadcast(value, encoder<T>().clsTag())
409+
sparkContext.broadcast(value, kotlinEncoderFor<T>().clsTag())
400410
} catch (e: ClassNotFoundException) {
401411
JavaSparkContext(sparkContext).broadcast(value)
402412
}
@@ -416,7 +426,7 @@ inline fun <reified T> SparkSession.broadcast(value: T): Broadcast<T> = try {
416426
DeprecationLevel.WARNING
417427
)
418428
inline fun <reified T> SparkContext.broadcast(value: T): Broadcast<T> = try {
419-
broadcast(value, encoder<T>().clsTag())
429+
broadcast(value, kotlinEncoderFor<T>().clsTag())
420430
} catch (e: ClassNotFoundException) {
421431
JavaSparkContext(this).broadcast(value)
422432
}

0 commit comments

Comments
 (0)