diff --git a/README.md b/README.md index c6b11f71..3ab69c55 100644 --- a/README.md +++ b/README.md @@ -192,6 +192,9 @@ to create `TypedColumn`s and with those a new Dataset from pieces of another usi ```kotlin val dataset: Dataset = ... val newDataset: Dataset> = dataset.selectTyped(col(YourClass::colA), col(YourClass::colB)) + +// Alternatively, for instance when working with a Dataset +val typedDataset: Dataset> = otherDataset.selectTyped(col("a").`as`(), col("b").`as`()) ``` ### Overload resolution ambiguity diff --git a/core/3.0/src/main/scala/org/jetbrains/kotlinx/spark/extensions/KSparkExtensions.scala b/core/3.0/src/main/scala/org/jetbrains/kotlinx/spark/extensions/KSparkExtensions.scala index 6b4935f1..3ee379fc 100644 --- a/core/3.0/src/main/scala/org/jetbrains/kotlinx/spark/extensions/KSparkExtensions.scala +++ b/core/3.0/src/main/scala/org/jetbrains/kotlinx/spark/extensions/KSparkExtensions.scala @@ -34,6 +34,7 @@ object KSparkExtensions { def collectAsList[T](ds: Dataset[T]): util.List[T] = JavaConverters.seqAsJavaList(ds.collect()) + def tailAsList[T](ds: Dataset[T], n: Int): util.List[T] = util.Arrays.asList(ds.tail(n) : _*) def debugCodegen(df: Dataset[_]): Unit = { import org.apache.spark.sql.execution.debug._ diff --git a/kotlin-spark-api/2.4/src/main/kotlin/org/jetbrains/kotlinx/spark/api/ApiV1.kt b/kotlin-spark-api/2.4/src/main/kotlin/org/jetbrains/kotlinx/spark/api/ApiV1.kt index 34152494..122e1747 100644 --- a/kotlin-spark-api/2.4/src/main/kotlin/org/jetbrains/kotlinx/spark/api/ApiV1.kt +++ b/kotlin-spark-api/2.4/src/main/kotlin/org/jetbrains/kotlinx/spark/api/ApiV1.kt @@ -647,12 +647,19 @@ operator fun Column.get(key: Any): Column = getItem(key) fun lit(a: Any) = functions.lit(a) /** - * Provides a type hint about the expected return value of this column. This information can + * Provides a type hint about the expected return value of this column. This information can * be used by operations such as `select` on a [Dataset] to automatically convert the * results into the correct JVM types. + * + * ``` + * val df: Dataset = ... + * val typedColumn: Dataset = df.selectTyped( col("a").`as`() ) + * ``` */ +@Suppress("UNCHECKED_CAST") inline fun Column.`as`(): TypedColumn = `as`(encoder()) + /** * Alias for [Dataset.joinWith] which passes "left" argument * and respects the fact that in result of left join right relation is nullable @@ -809,45 +816,74 @@ fun Dataset.showDS(numRows: Int = 20, truncate: Boolean = true) = apply { /** * Returns a new Dataset by computing the given [Column] expressions for each element. */ +@Suppress("UNCHECKED_CAST") +inline fun Dataset.selectTyped( + c1: TypedColumn, +): Dataset = select(c1 as TypedColumn) + +/** + * Returns a new Dataset by computing the given [Column] expressions for each element. + */ +@Suppress("UNCHECKED_CAST") inline fun Dataset.selectTyped( - c1: TypedColumn, - c2: TypedColumn, + c1: TypedColumn, + c2: TypedColumn, ): Dataset> = - select(c1, c2).map { Pair(it._1(), it._2()) } + select( + c1 as TypedColumn, + c2 as TypedColumn, + ).map { Pair(it._1(), it._2()) } /** * Returns a new Dataset by computing the given [Column] expressions for each element. */ +@Suppress("UNCHECKED_CAST") inline fun Dataset.selectTyped( - c1: TypedColumn, - c2: TypedColumn, - c3: TypedColumn, + c1: TypedColumn, + c2: TypedColumn, + c3: TypedColumn, ): Dataset> = - select(c1, c2, c3).map { Triple(it._1(), it._2(), it._3()) } + select( + c1 as TypedColumn, + c2 as TypedColumn, + c3 as TypedColumn, + ).map { Triple(it._1(), it._2(), it._3()) } /** * Returns a new Dataset by computing the given [Column] expressions for each element. */ +@Suppress("UNCHECKED_CAST") inline fun Dataset.selectTyped( - c1: TypedColumn, - c2: TypedColumn, - c3: TypedColumn, - c4: TypedColumn, + c1: TypedColumn, + c2: TypedColumn, + c3: TypedColumn, + c4: TypedColumn, ): Dataset> = - select(c1, c2, c3, c4).map { Arity4(it._1(), it._2(), it._3(), it._4()) } + select( + c1 as TypedColumn, + c2 as TypedColumn, + c3 as TypedColumn, + c4 as TypedColumn, + ).map { Arity4(it._1(), it._2(), it._3(), it._4()) } /** * Returns a new Dataset by computing the given [Column] expressions for each element. */ +@Suppress("UNCHECKED_CAST") inline fun Dataset.selectTyped( - c1: TypedColumn, - c2: TypedColumn, - c3: TypedColumn, - c4: TypedColumn, - c5: TypedColumn, + c1: TypedColumn, + c2: TypedColumn, + c3: TypedColumn, + c4: TypedColumn, + c5: TypedColumn, ): Dataset> = - select(c1, c2, c3, c4, c5).map { Arity5(it._1(), it._2(), it._3(), it._4(), it._5()) } - + select( + c1 as TypedColumn, + c2 as TypedColumn, + c3 as TypedColumn, + c4 as TypedColumn, + c5 as TypedColumn, + ).map { Arity5(it._1(), it._2(), it._3(), it._4(), it._5()) } @OptIn(ExperimentalStdlibApi::class) inline fun schema(map: Map = mapOf()) = schema(typeOf(), map) diff --git a/kotlin-spark-api/2.4/src/test/kotlin/org/jetbrains/kotlinx/spark/api/ApiTest.kt b/kotlin-spark-api/2.4/src/test/kotlin/org/jetbrains/kotlinx/spark/api/ApiTest.kt index bae27b2e..ec8a6e14 100644 --- a/kotlin-spark-api/2.4/src/test/kotlin/org/jetbrains/kotlinx/spark/api/ApiTest.kt +++ b/kotlin-spark-api/2.4/src/test/kotlin/org/jetbrains/kotlinx/spark/api/ApiTest.kt @@ -339,23 +339,26 @@ class ApiTest : ShouldSpec({ SomeClass(intArrayOf(1, 2, 4), 5), ) - val typedColumnA: TypedColumn = dataset.col("a").`as`(encoder()) + val newDS1WithAs: Dataset = dataset.selectTyped( + col("b").`as`(), + ) + newDS1WithAs.show() - val newDS2 = dataset.selectTyped( + val newDS2: Dataset> = dataset.selectTyped( // col(SomeClass::a), NOTE that this doesn't work on 2.4, returnting a data class with an array in it col(SomeClass::b), col(SomeClass::b), ) newDS2.show() - val newDS3 = dataset.selectTyped( + val newDS3: Dataset> = dataset.selectTyped( col(SomeClass::b), col(SomeClass::b), col(SomeClass::b), ) newDS3.show() - val newDS4 = dataset.selectTyped( + val newDS4: Dataset> = dataset.selectTyped( col(SomeClass::b), col(SomeClass::b), col(SomeClass::b), @@ -363,7 +366,7 @@ class ApiTest : ShouldSpec({ ) newDS4.show() - val newDS5 = dataset.selectTyped( + val newDS5: Dataset> = dataset.selectTyped( col(SomeClass::b), col(SomeClass::b), col(SomeClass::b), diff --git a/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/ApiV1.kt b/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/ApiV1.kt index 2dde48cb..695d9c32 100644 --- a/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/ApiV1.kt +++ b/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/ApiV1.kt @@ -28,6 +28,7 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.* import org.apache.spark.sql.Encoders.* import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +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.streaming.OutputMode @@ -74,6 +75,7 @@ import kotlin.Unit import kotlin.also import kotlin.apply import kotlin.invoke +import kotlin.random.Random import kotlin.reflect.* import kotlin.reflect.full.findAnnotation import kotlin.reflect.full.isSubclassOf @@ -185,6 +187,327 @@ private fun kotlinClassEncoder(schema: DataType, kClass: KClass<*>): Encoder ) } +/** + * Allows `for (element in dataset)`. + * + * Note that this creates an iterator which can consume lots of memory. `.forEach {}` might be more efficient. + * TODO: Add plugin inspection hint + */ +operator fun Dataset.iterator(): Iterator = toLocalIterator() + +fun Dataset.toIterable(): Iterable = Iterable { toLocalIterator() } + +/** + * Returns `true` if [element] is found in the collection. + * + * Note: Converting the dataset to an [Iterable] first might be a faster but potentially more memory + * intensive solution. See [toIterable]. + * TODO: Add plugin inspection hint + */ +inline operator fun Dataset.contains(element: T): Boolean = !filter { it == element }.isEmpty + +/** + * Returns the first element matching the given [predicate], or `null` if no such element was found. + * + * Note: Converting the dataset to an [Iterable] first might be a faster but potentially more memory + * intensive solution. See [toIterable]. + * TODO: Add plugin inspection hint + */ +fun Dataset.find(predicate: (T) -> Boolean): T? { + return firstOrNull(predicate) +} + +/** + * Returns the last element matching the given [predicate], or `null` if no such element was found. + * TODO: Add plugin inspection hint + */ +fun Dataset.findLast(predicate: (T) -> Boolean): T? { + return lastOrNull(predicate) +} + +/** + * Returns the first element matching the given [predicate]. + * @throws [NoSuchElementException] if no such element is found. + * TODO: Add plugin inspection hint + */ +fun Dataset.first(predicate: (T) -> Boolean): T = + filter(predicate).first() + +/** + * Returns the first non-null value produced by [transform] function being applied to elements of this collection in iteration order, + * or throws [NoSuchElementException] if no non-null value was produced. + * TODO: Add plugin inspection hint + */ +inline fun Dataset.firstNotNullOf(noinline transform: (T) -> R?): R = + map(transform) + .filterNotNull() + .first() + +/** + * Returns the first non-null value produced by [transform] function being applied to elements of this collection in iteration order, + * or `null` if no non-null value was produced. + * TODO: Add plugin inspection hint + */ +inline fun Dataset.firstNotNullOfOrNull(noinline transform: (T) -> R?): R? = + map(transform) + .filterNotNull() + .firstOrNull() + +/** + * Returns the first element, or `null` if the collection is empty. + */ +fun Dataset.firstOrNull(): T? = if (isEmpty) null else first() + +/** + * Returns the first element matching the given [predicate], or `null` if element was not found. + * TODO: Add plugin inspection hint + */ +fun Dataset.firstOrNull(predicate: (T) -> Boolean): T? = filter(predicate).firstOrNull() + +/** + * Returns the last element. + * + * @throws NoSuchElementException if the collection is empty. + */ +fun Dataset.last(): T = tailAsList(1).first() + +/** + * Returns the last element matching the given [predicate]. + * + * @throws NoSuchElementException if no such element is found. + * TODO: Add plugin inspection hint + */ +fun Dataset.last(predicate: (T) -> Boolean): T = filter(predicate).last() + +/** + * Returns the last element, or `null` if the collection is empty. + * TODO: Add plugin inspection hint + */ +fun Dataset.lastOrNull(): T? = if (isEmpty) null else last() + +/** + * Returns the last element matching the given [predicate], or `null` if no such element was found. + * TODO: Add plugin inspection hint + */ +fun Dataset.lastOrNull(predicate: (T) -> Boolean): T? = filter(predicate).lastOrNull() + +/** + * Returns the last `n` rows in the Dataset as a list. + * + * Running tail requires moving data into the application's driver process, and doing so with + * a very large `n` can crash the driver process with OutOfMemoryError. + */ +fun Dataset.tailAsList(n: Int): List = KSparkExtensions.tailAsList(this, n) + +/** + * Returns a random element from this Dataset using the specified source of randomness. + * + * @param seed seed for the random number generator + * + * @throws NoSuchElementException if this collection is empty. + */ +fun Dataset.random(seed: Long = Random.nextLong()): T = + randomOrNull(seed) ?: throw NoSuchElementException("Collection is empty.") + +/** + * Returns a random element from this collection using the specified source of randomness, or `null` if this collection is empty. + * @param seed seed for the random number generator + */ +fun Dataset.randomOrNull(seed: Long = Random.nextLong()): T? { + if (isEmpty) + return null + + return toJavaRDD() + .takeSample(false, 1, seed) + .first() +} + +/** + * Returns the single element, or throws an exception if the Dataset is empty or has more than one element. + */ +fun Dataset.single(): T { + if (isEmpty) + throw NoSuchElementException("Dataset is empty.") + + val firstTwo: List = takeAsList(2) // less heavy than count() + return when (firstTwo.size) { + 1 -> firstTwo.first() + else -> throw IllegalArgumentException("Dataset has more than one element.") + } +} + +/** + * Returns single element, or `null` if the Dataset is empty or has more than one element. + */ +fun Dataset.singleOrNull(): T? { + if (isEmpty) + return null + + val firstTwo: List = takeAsList(2) // less heavy than count() + return when (firstTwo.size) { + 1 -> firstTwo.first() + else -> null + } +} + + +fun Dataset<*>.getUniqueNewColumnName(): String { + val rowKeys = columns() + val alphabet = 'a'..'z' + var colName = alphabet.random().toString() + while (colName in rowKeys) colName += alphabet.random() + + return colName +} + +/** + * Returns a Dataset containing all elements except first [n] elements. + * + * @throws IllegalArgumentException if [n] is negative. + * + * TODO make more efficient + */ +inline fun Dataset.drop(n: Int): Dataset { + require(n >= 0) { "Requested element count $n is less than zero." } + return when { + isEmpty -> this + n >= count() -> limit(0) + else -> { + val index = getUniqueNewColumnName() + withColumn(index, monotonicallyIncreasingId()) + .orderBy(desc(index)) + .dropLast(n) + .orderBy(index) + .drop(index) + .`as`() + } + } + +} + +/** + * Returns a Dataset containing all elements except last [n] elements. + * + * @throws IllegalArgumentException if [n] is negative. + */ +fun Dataset.dropLast(n: Int): Dataset { + require(n >= 0) { "Requested element count $n is less than zero." } + return when { + isEmpty -> this + n >= count() -> limit(0) + else -> limit( + (count() - n).toInt().coerceAtLeast(0) + ) + } + +} + +/** + * Returns a Dataset containing all elements except last elements that satisfy the given [predicate]. + * + * TODO Add plugin toIterable warning + */ +inline fun Dataset.dropLastWhile(noinline predicate: (T) -> Boolean): Dataset { + if (isEmpty) return this + + val filterApplied = map(predicate) + .withColumn( + getUniqueNewColumnName(), + monotonicallyIncreasingId(), + ) + + if (filterApplied.all { it.getBoolean(0) }) + return limit(0) + + if (filterApplied.all { !it.getBoolean(0) }) + return this + + val dropFrom = filterApplied + .lastOrNull { !it.getBoolean(0) } + ?.getLong(1) + ?: -1L + + return dropLast(count().toInt() - (dropFrom.toInt() + 1)) +} + +/** + * Returns a Dataset containing all elements except first elements that satisfy the given [predicate]. + * + * TODO Can definitely be made more efficient + * TODO Add plugin toIterable warning + */ +inline fun Dataset.dropWhile(noinline predicate: (T) -> Boolean): Dataset { + if (isEmpty) return this + + val filterApplied = map(predicate) + .withColumn( + getUniqueNewColumnName(), + monotonicallyIncreasingId(), + ) + + if (filterApplied.all { it.getBoolean(0) }) + return limit(0) + + if (filterApplied.all { !it.getBoolean(0) }) + return this + + val dropUntil = filterApplied + .firstOrNull { it.getBoolean(0) } + ?.getLong(1) + ?: -1L + + return drop(dropUntil.toInt() + 1) +} + +/** + * Returns a list containing only elements matching the given [predicate]. + * @param [predicate] function that takes the index of an element and the element itself + * and returns the result of predicate evaluation on the element. + * + */ +inline fun Dataset.filterIndexed(crossinline predicate: (index: Long, T) -> Boolean): Dataset { + TODO() + val indices = selectTyped(monotonicallyIncreasingId().`as`()) + // TODO this needs to zip, not join + val joined = indices.leftJoin(this, col(indices.columns().first()) neq -1L) + val filterResults = joined.map { (index, value) -> predicate(index, value!!) } + val filtered = selectTyped(col(filterResults.columns().first()).`as`()) + + return filtered +} + + + +/** + * Returns `true` if collection has at least one element. + */ +fun Dataset<*>.any(): Boolean = !isEmpty + +/** + * Returns `true` if all elements match the given [predicate]. + * + * TODO plugin (!any) + */ +inline fun Dataset.all(noinline predicate: (T) -> Boolean): Boolean { + if (isEmpty) return true + + return map(predicate) + .reduceK { a, b -> a && b } +} + + +/** + * Returns `true` if at least one element matches the given [predicate]. + * + * TODO plugin note to make it faster + */ +inline fun Dataset.any(noinline predicate: (T) -> Boolean): Boolean { + if (isEmpty) return false + + return map(predicate) + .reduceK { a, b -> a || b } +} + inline fun Dataset.map(noinline func: (T) -> R): Dataset = map(MapFunction(func), encoder()) @@ -200,7 +523,8 @@ inline fun Dataset.groupByKey(noinline func: (T) -> R): KeyVal inline fun Dataset.mapPartitions(noinline func: (Iterator) -> Iterator): Dataset = mapPartitions(func, encoder()) -fun Dataset.filterNotNull() = filter { it != null } +@Suppress("UNCHECKED_CAST") +fun Dataset.filterNotNull(): Dataset = filter { it != null } as Dataset inline fun KeyValueGroupedDataset.mapValues(noinline func: (VALUE) -> R): KeyValueGroupedDataset = mapValues(MapFunction(func), encoder()) @@ -643,10 +967,16 @@ operator fun Column.get(key: Any): Column = getItem(key) fun lit(a: Any) = functions.lit(a) /** - * Provides a type hint about the expected return value of this column. This information can + * Provides a type hint about the expected return value of this column. This information can * be used by operations such as `select` on a [Dataset] to automatically convert the * results into the correct JVM types. + * + * ``` + * val df: Dataset = ... + * val typedColumn: Dataset = df.selectTyped( col("a").`as`() ) + * ``` */ +@Suppress("UNCHECKED_CAST") inline fun Column.`as`(): TypedColumn = `as`(encoder()) /** @@ -768,9 +1098,8 @@ inline fun Dataset.col(column: KProperty1): Type * Returns a [Column] based on the given class attribute, not connected to a dataset. * ```kotlin * val dataset: Dataset = ... - * val new: Dataset> = dataset.select( col(YourClass::a), col(YourClass::b) ) + * val new: Dataset> = dataset.select( col(YourClass::a), col(YourClass::b) ) * ``` - * TODO: change example to [Pair]s when merged */ @Suppress("UNCHECKED_CAST") inline fun col(column: KProperty1): TypedColumn = @@ -805,44 +1134,74 @@ fun Dataset.showDS(numRows: Int = 20, truncate: Boolean = true) = apply { /** * Returns a new Dataset by computing the given [Column] expressions for each element. */ +@Suppress("UNCHECKED_CAST") +inline fun Dataset.selectTyped( + c1: TypedColumn, +): Dataset = select(c1 as TypedColumn) + +/** + * Returns a new Dataset by computing the given [Column] expressions for each element. + */ +@Suppress("UNCHECKED_CAST") inline fun Dataset.selectTyped( - c1: TypedColumn, - c2: TypedColumn, + c1: TypedColumn, + c2: TypedColumn, ): Dataset> = - select(c1, c2).map { Pair(it._1(), it._2()) } + select( + c1 as TypedColumn, + c2 as TypedColumn, + ).map { Pair(it._1(), it._2()) } /** * Returns a new Dataset by computing the given [Column] expressions for each element. */ +@Suppress("UNCHECKED_CAST") inline fun Dataset.selectTyped( - c1: TypedColumn, - c2: TypedColumn, - c3: TypedColumn, + c1: TypedColumn, + c2: TypedColumn, + c3: TypedColumn, ): Dataset> = - select(c1, c2, c3).map { Triple(it._1(), it._2(), it._3()) } + select( + c1 as TypedColumn, + c2 as TypedColumn, + c3 as TypedColumn, + ).map { Triple(it._1(), it._2(), it._3()) } /** * Returns a new Dataset by computing the given [Column] expressions for each element. */ +@Suppress("UNCHECKED_CAST") inline fun Dataset.selectTyped( - c1: TypedColumn, - c2: TypedColumn, - c3: TypedColumn, - c4: TypedColumn, + c1: TypedColumn, + c2: TypedColumn, + c3: TypedColumn, + c4: TypedColumn, ): Dataset> = - select(c1, c2, c3, c4).map { Arity4(it._1(), it._2(), it._3(), it._4()) } + select( + c1 as TypedColumn, + c2 as TypedColumn, + c3 as TypedColumn, + c4 as TypedColumn, + ).map { Arity4(it._1(), it._2(), it._3(), it._4()) } /** * Returns a new Dataset by computing the given [Column] expressions for each element. */ +@Suppress("UNCHECKED_CAST") inline fun Dataset.selectTyped( - c1: TypedColumn, - c2: TypedColumn, - c3: TypedColumn, - c4: TypedColumn, - c5: TypedColumn, + c1: TypedColumn, + c2: TypedColumn, + c3: TypedColumn, + c4: TypedColumn, + c5: TypedColumn, ): Dataset> = - select(c1, c2, c3, c4, c5).map { Arity5(it._1(), it._2(), it._3(), it._4(), it._5()) } + select( + c1 as TypedColumn, + c2 as TypedColumn, + c3 as TypedColumn, + c4 as TypedColumn, + c5 as TypedColumn, + ).map { Arity5(it._1(), it._2(), it._3(), it._4(), it._5()) } @OptIn(ExperimentalStdlibApi::class) diff --git a/kotlin-spark-api/3.0/src/test/kotlin/org/jetbrains/kotlinx/spark/api/ApiTest.kt b/kotlin-spark-api/3.0/src/test/kotlin/org/jetbrains/kotlinx/spark/api/ApiTest.kt index bff38ac1..c4fb0d3e 100644 --- a/kotlin-spark-api/3.0/src/test/kotlin/org/jetbrains/kotlinx/spark/api/ApiTest.kt +++ b/kotlin-spark-api/3.0/src/test/kotlin/org/jetbrains/kotlinx/spark/api/ApiTest.kt @@ -362,22 +362,25 @@ class ApiTest : ShouldSpec({ SomeClass(intArrayOf(1, 2, 4), 5), ) - val typedColumnA: TypedColumn = dataset.col("a").`as`(encoder()) + val newDS1WithAs: Dataset = dataset.selectTyped( + col("a").`as`(), + ) + newDS1WithAs.show() - val newDS2 = dataset.selectTyped( + val newDS2: Dataset> = dataset.selectTyped( col(SomeClass::a), // NOTE: this only works on 3.0, returning a data class with an array in it col(SomeClass::b), ) newDS2.show() - val newDS3 = dataset.selectTyped( + val newDS3: Dataset> = dataset.selectTyped( col(SomeClass::a), col(SomeClass::b), col(SomeClass::b), ) newDS3.show() - val newDS4 = dataset.selectTyped( + val newDS4: Dataset> = dataset.selectTyped( col(SomeClass::a), col(SomeClass::b), col(SomeClass::b), @@ -385,7 +388,7 @@ class ApiTest : ShouldSpec({ ) newDS4.show() - val newDS5 = dataset.selectTyped( + val newDS5: Dataset> = dataset.selectTyped( col(SomeClass::a), col(SomeClass::b), col(SomeClass::b), @@ -558,6 +561,12 @@ class ApiTest : ShouldSpec({ first.someOtherArray shouldBe arrayOf(SomeOtherEnum.C, SomeOtherEnum.D) first.enumMap shouldBe mapOf(SomeEnum.A to SomeOtherEnum.C) } + should("Have more stdlib functions for Datasets") { + val dataset = listOf(1, 2, 3).toDS().drop(2) + dataset.count() shouldBe 1L + (3 in dataset) shouldBe true + + } } } })