diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index e8024b45..98a7ac04 100644 --- a/.github/workflows/build.yml +++ b/.github/workflows/build.yml @@ -26,4 +26,11 @@ jobs: restore-keys: ${{ runner.os }}-m2 - name: Build with Maven run: ./mvnw -B package --file pom.xml -Pscala-2.12 +# 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 diff --git a/.github/workflows/generate_docs.yml b/.github/workflows/generate_docs.yml new file mode 100644 index 00000000..cdbd1949 --- /dev/null +++ b/.github/workflows/generate_docs.yml @@ -0,0 +1,30 @@ +name: Generate and publish docs + +on: + push: + branches: + - "spark-3.2" + +jobs: + generate-and-publish-docs: + runs-on: ubuntu-latest + + steps: + - uses: actions/checkout@v2 + - name: Set up JDK 11 + uses: actions/setup-java@v1 + with: + distributions: adopt + java-version: 11 + check-latest: true + - name: Generate docs + run: ./mvnw clean package site -Dmaven.test.skip=true + - name: Copy docs to "docs" branch + uses: peaceiris/actions-gh-pages@v3 + with: + github_token: ${{ secrets.GITHUB_TOKEN }} + publish_branch: docs + publish_dir: ./kotlin-spark-api/3.2/target/dokka + force_orphan: true + + diff --git a/.github/workflows/publish_dev_version.yml b/.github/workflows/publish_dev_version.yml new file mode 100644 index 00000000..a0f2a374 --- /dev/null +++ b/.github/workflows/publish_dev_version.yml @@ -0,0 +1,31 @@ +name: Generate and publish docs + +on: + push: + branches: + - "spark-3.2" + +jobs: + build-and-deploy: + runs-on: ubuntu-latest + + steps: + - uses: actions/checkout@v2 + - name: Set up JDK 11 + uses: actions/setup-java@v1 + with: + distributions: adopt + java-version: 11 + check-latest: true + - name: Cache Maven packages + uses: actions/cache@v2 + with: + path: ~/.m2 + key: ${{ runner.os }}-m2-${{ hashFiles('**/pom.xml') }} + restore-keys: ${{ runner.os }}-m2 + - name: Deploy to GH Packages + run: ./mvnw --batch-mode deploy + env: + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + + diff --git a/README.md b/README.md index 3f518316..095fbb79 100644 --- a/README.md +++ b/README.md @@ -1,4 +1,4 @@ -# Kotlin for Apache® Spark™ [![Maven Central](https://img.shields.io/maven-central/v/org.jetbrains.kotlinx.spark/kotlin-spark-api-parent.svg?label=Maven%20Central)](https://search.maven.org/search?q=g:org.jetbrains.kotlinx.spark%20AND%20v:1.0.2) [![official JetBrains project](http://jb.gg/badges/incubator.svg)](https://confluence.jetbrains.com/display/ALL/JetBrains+on+GitHub) +# Kotlin for Apache® Spark™ [![Maven Central](https://img.shields.io/maven-central/v/org.jetbrains.kotlinx.spark/kotlin-spark-api-parent-3.2.svg?label=Maven%20Central)](https://search.maven.org/search?q=g:org.jetbrains.kotlinx.spark%20AND%20v:1.1.0) [![official JetBrains project](http://jb.gg/badges/official.svg)](https://confluence.jetbrains.com/display/ALL/JetBrains+on+GitHub) [![Join the chat at https://gitter.im/JetBrains/kotlin-spark-api](https://badges.gitter.im/JetBrains/kotlin-spark-api.svg)](https://gitter.im/JetBrains/kotlin-spark-api?utm_source=badge&utm_medium=badge&utm_campaign=pr-badge&utm_content=badge) Your next API to work with [Apache Spark](https://spark.apache.org/). @@ -22,18 +22,22 @@ We have opened a Spark Project Improvement Proposal: [Kotlin support for Apache - [toList and toArray](#tolist-and-toarray-methods) - [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) - [License](#license) -## Supported versions of Apache Spark #TODO +## Supported versions of Apache Spark -| Apache Spark | Scala | Kotlin for Apache Spark | +| Apache Spark | Scala | Kotlin for Apache Spark | |:------------:|:-----:|:-------------------------------:| -| 3.0.0+ | 2.12 | kotlin-spark-api-3.0:1.0.2 | -| 2.4.1+ | 2.12 | kotlin-spark-api-2.4_2.12:1.0.2 | -| 2.4.1+ | 2.11 | kotlin-spark-api-2.4_2.11:1.0.2 | +| 3.2.1+ | 2.12 | kotlin-spark-api-3.2:1.1.0 | +| 3.1.3+ | 2.12 | kotlin-spark-api-3.1:1.1.0 | +| 3.0.3+ | 2.12 | kotlin-spark-api-3.0:1.1.0 | +| 2.4.1+ | 2.12 | kotlin-spark-api-2.4_2.12:1.0.2 | +| 2.4.1+ | 2.11 | kotlin-spark-api-2.4_2.11:1.0.2 | ## Releases @@ -41,7 +45,7 @@ The list of Kotlin for Apache Spark releases is available [here](https://github. The Kotlin for Spark artifacts adhere to the following convention: `[Apache Spark version]_[Scala core version]:[Kotlin for Apache Spark API version]` -[![Maven Central](https://img.shields.io/maven-central/v/org.jetbrains.kotlinx.spark/kotlin-spark-api-parent.svg?label=Maven%20Central)](https://search.maven.org/search?q=g:"org.jetbrains.kotlinx.spark"%20AND%20a:"kotlin-spark-api-3.0") +[![Maven Central](https://img.shields.io/maven-central/v/org.jetbrains.kotlinx.spark/kotlin-spark-api-parent-3.2.svg?label=Maven%20Central)](https://search.maven.org/search?q=g:"org.jetbrains.kotlinx.spark"%20AND%20a:"kotlin-spark-api-3.2") ## How to configure Kotlin for Apache Spark in your project @@ -70,6 +74,41 @@ Once you have configured the dependency, you only need to add the following impo import org.jetbrains.kotlinx.spark.api.* ``` +### Jupyter + +The Kotlin Spark API also supports Kotlin Jupyter notebooks. +To it, simply add + +```jupyterpython +%use spark +``` +to the top of your notebook. This will get the latest version of the API, together with the latest version of Spark. +To define a certain version of Spark or the API itself, simply add it like this: +```jupyterpython +%use spark(spark=3.2, v=1.1.0) +``` + +Inside the notebook a Spark session will be initiated automatically. This can be accessed via the `spark` value. +`sc: JavaSparkContext` can also be accessed directly. The API operates pretty similarly. + +There is also support for HTML rendering of Datasets and simple (Java)RDDs. +Check out the [example](examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/JupyterExample.ipynb) as well. + + +To use Spark Streaming abilities, instead use +```jupyterpython +%use spark-streaming +``` +This does not start a Spark session right away, meaning you can call `withSparkStreaming(batchDuration) {}` +in whichever cell you want. +Check out the [example](examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/JupyterStreamingExample.ipynb). + +NOTE: You need `kotlin-jupyter-kernel` to be at least version 0.11.0.83 for the Kotlin Spark API to work. Also, if the +`%use spark` magic does not output "Spark session has been started...", and `%use spark-streaming` doesn't work at all, +add `%useLatestDescriptors` above it. + +For more information, check the [wiki](https://github.com/JetBrains/kotlin-spark-api/wiki/Jupyter). + ## Kotlin for Apache Spark features ### Creating a SparkSession in Kotlin @@ -78,32 +117,40 @@ val spark = SparkSession .builder() .master("local[2]") .appName("Simple Application").orCreate - ``` +This is not needed when running the Kotlin Spark API from a Jupyter notebook. + ### Creating a Dataset in Kotlin ```kotlin -spark.toDS("a" to 1, "b" to 2) +spark.dsOf("a" to 1, "b" to 2) ``` -The example above produces `Dataset>`. +The example above produces `Dataset>`. While Kotlin Pairs and Triples are supported, Scala Tuples are +recommended for better support. ### Null safety There are several aliases in API, like `leftJoin`, `rightJoin` etc. These are null-safe by design. For example, `leftJoin` is aware of nullability and returns `Dataset>`. Note that we are forcing `RIGHT` to be nullable for you as a developer to be able to handle this situation. -`NullPointerException`s are hard to debug in Spark, and we doing our best to make them as rare as possible. +`NullPointerException`s are hard to debug in Spark, and we're doing our best to make them as rare as possible. +In Spark, you might also come across Scala-native `Option<*>` or Java-compatible `Optional<*>` classes. +We provide `getOrNull()` and `getOrElse()` functions for these to use Kotlin's null safety for good. + +Similarly, you can also create `Option<*>`s and `Optional<*>`s like `T?.toOptional()` if a Spark function requires it. ### withSpark function We provide you with useful function `withSpark`, which accepts everything that may be needed to run Spark — properties, name, master location and so on. It also accepts a block of code to execute inside Spark context. After work block ends, `spark.stop()` is called automatically. +Do not use this when running the Kotlin Spark API from a Jupyter notebook. + ```kotlin withSpark { dsOf(1, 2) - .map { it to it } - .show() + .map { it X it } // creates Tuple2 + .show() } ``` @@ -120,14 +167,14 @@ To solve these problems we've added `withCached` function ```kotlin withSpark { dsOf(1, 2, 3, 4, 5) - .map { it to (it + 2) } - .withCached { - showDS() - - filter { it.first % 2 == 0 }.showDS() - } - .map { c(it.first, it.second, (it.first + it.second) * 2) } - .show() + .map { tupleOf(it, it + 2) } + .withCached { + showDS() + + filter { it._1 % 2 == 0 }.showDS() + } + .map { tupleOf(it._1, it._2, (it._1 + it._2) * 2) } + .show() } ``` @@ -153,60 +200,80 @@ dataset.where( col("colA") `===` 6 ) dataset.where( col("colA") eq 6) ``` -In short, all supported operators are: - -- `==`, -- `!=`, -- `eq` / `` `===` ``, -- `neq` / `` `=!=` ``, -- `-col(...)`, -- `!col(...)`, -- `gt`, -- `lt`, -- `geq`, -- `leq`, -- `or`, -- `and` / `` `&&` ``, -- `+`, -- `-`, -- `*`, -- `/`, -- `%` - -Secondly, there are some quality of life additions as well: - -In Kotlin, Ranges are often -used to solve inclusive/exclusive situations for a range. So, you can now do: -```kotlin -dataset.where( col("colA") inRangeOf 0..2 ) -``` +To read more, check the [wiki](https://github.com/JetBrains/kotlin-spark-api/wiki/Column-functions). + +### Overload resolution ambiguity + +We had to implement the functions `reduceGroups` and `reduce` for Kotlin separately as `reduceGroupsK` and `reduceK` respectively, because otherwise it caused resolution ambiguity between Kotlin, Scala and Java APIs, which was quite hard to solve. + +We have a special example of work with this function in the [Groups example](https://github.com/JetBrains/kotlin-spark-api/blob/main/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Group.kt). -Also, for columns containing map- or array like types: +### Tuples +Inspired by [ScalaTuplesInKotlin](https://github.com/Jolanrensen/ScalaTuplesInKotlin), the API introduces a lot of helper- extension functions +to make working with Scala Tuples a breeze in your Kotlin Spark projects. While working with data classes is encouraged, +for pair-like Datasets / RDDs / DStreams Scala Tuples are recommended, both for the useful helper functions, as well as Spark performance. +To enable these features +simply add ```kotlin -dataset.where( col("colB")[0] geq 5 ) +import org.jetbrains.kotlinx.spark.api.tuples.* ``` +to the start of your file. -Finally, thanks to Kotlin reflection, we can provide a type- and refactor safe way -to create `TypedColumn`s and with those a new Dataset from pieces of another using the `selectTyped()` function, added to the API: +Tuple creation can be done in the following manners: ```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`()) +val a: Tuple2 = tupleOf(1, 2L) +val b: Tuple3 = t("test", 1.0, 2) +val c: Tuple3 = 5f X "aaa" X 1 ``` +To read more about tuples and all the added functions, refer to the [wiki](https://github.com/JetBrains/kotlin-spark-api/wiki/Tuples). -### Overload resolution ambiguity +### Streaming -We had to implement the functions `reduceGroups` and `reduce` for Kotlin separately as `reduceGroupsK` and `reduceK` respectively, because otherwise it caused resolution ambiguity between Kotlin, Scala and Java APIs, which was quite hard to solve. +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 have a special example of work with this function in the [Groups example](https://github.com/JetBrains/kotlin-spark-api/blob/main/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Group.kt). +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| + // +-----+--------+ + } + } +} +``` +For more information, check the [wiki](https://github.com/JetBrains/kotlin-spark-api/wiki/Streaming). ## 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. +For more, check out [examples](examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples) module. To get up and running quickly, check out this [tutorial](https://github.com/JetBrains/kotlin-spark-api/wiki/Quick-Start-Guide). ## Reporting issues/Support diff --git a/core/3.0/pom_2.12.xml b/core/3.0/pom_2.12.xml new file mode 100644 index 00000000..5038f4e1 --- /dev/null +++ b/core/3.0/pom_2.12.xml @@ -0,0 +1,80 @@ + + + 4.0.0 + + Kotlin Spark API: Scala core for Spark 3.0+ (Scala 2.12) + Scala-Spark 3.0+ compatibility layer for Kotlin for Apache Spark + core-3.0_2.12 + + org.jetbrains.kotlinx.spark + kotlin-spark-api-parent-3.0_2.12 + 1.1.0 + ../../pom_2.12.xml + + + + + org.scala-lang + scala-library + ${scala.version} + + + org.jetbrains.kotlin + kotlin-reflect + + + + + + org.apache.spark + spark-sql_${scala.compat.version} + ${spark3.version} + provided + + + + + + src/main/scala + src/test/scala + target/${scala.compat.version} + + + net.alchim31.maven + scala-maven-plugin + ${scala-maven-plugin.version} + + + compile + + compile + testCompile + + + + -dependencyfile + ${project.build.directory}/.scala_dependencies + + + + + docjar + + doc-jar + + pre-integration-test + + + + + org.apache.maven.plugins + maven-site-plugin + + true + + + + + + + diff --git a/core/3.0/src/main/scala/org/apache/spark/sql/KotlinReflection.scala b/core/3.0/src/main/scala/org/apache/spark/sql/KotlinReflection.scala new file mode 100644 index 00000000..7f1190d8 --- /dev/null +++ b/core/3.0/src/main/scala/org/apache/spark/sql/KotlinReflection.scala @@ -0,0 +1,1410 @@ +/*- + * =LICENSE= + * Kotlin Spark API: Examples + * ---------- + * Copyright (C) 2019 - 2020 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.apache.spark.sql + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.DeserializerBuildHelper._ +import org.apache.spark.sql.catalyst.ScalaReflection.{Schema, dataTypeFor, getClassFromType, isSubtype, javaBoxedType, localTypeOf} +import org.apache.spark.sql.catalyst.SerializerBuildHelper._ +import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal +import org.apache.spark.sql.catalyst.expressions.objects._ +import org.apache.spark.sql.catalyst.expressions.{Expression, _} +import org.apache.spark.sql.catalyst.util.ArrayBasedMapData +import org.apache.spark.sql.catalyst.{DefinedByConstructorParams, InternalRow, ScalaReflection, WalkedTypePath} +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.util.Utils + +import java.beans.{Introspector, PropertyDescriptor} +import java.lang.Exception + + +/** + * A helper trait to create [[org.apache.spark.sql.catalyst.encoders.ExpressionEncoder]]s + * for classes whose fields are entirely defined by constructor params but should not be + * case classes. + */ +//trait DefinedByConstructorParams + +/** + * KotlinReflection is heavily inspired by ScalaReflection and even extends it just to add several methods + */ +//noinspection RedundantBlock +object KotlinReflection extends KotlinReflection { + ScalaReflection + /** + * Returns the Spark SQL DataType for a given java class. Where this is not an exact mapping + * to a native type, an ObjectType is returned. + * + * Unlike `inferDataType`, this function doesn't do any massaging of types into the Spark SQL type + * system. As a result, ObjectType will be returned for things like boxed Integers. + */ + private def inferExternalType(cls: Class[_]): DataType = cls match { + case c if c == java.lang.Boolean.TYPE => BooleanType + case c if c == java.lang.Byte.TYPE => ByteType + case c if c == java.lang.Short.TYPE => ShortType + case c if c == java.lang.Integer.TYPE => IntegerType + case c if c == java.lang.Long.TYPE => LongType + case c if c == java.lang.Float.TYPE => FloatType + case c if c == java.lang.Double.TYPE => DoubleType + case c if c == classOf[Array[Byte]] => BinaryType + case c if c == classOf[Decimal] => DecimalType.SYSTEM_DEFAULT + case c if c == classOf[CalendarInterval] => CalendarIntervalType + case _ => ObjectType(cls) + } + + val universe: scala.reflect.runtime.universe.type = scala.reflect.runtime.universe + + // Since we are creating a runtime mirror using the class loader of current thread, + // we need to use def at here. So, every time we call mirror, it is using the + // class loader of the current thread. + override def mirror: universe.Mirror = { + universe.runtimeMirror(Thread.currentThread().getContextClassLoader) + } + + import universe._ + + // The Predef.Map is scala.collection.immutable.Map. + // Since the map values can be mutable, we explicitly import scala.collection.Map at here. + import scala.collection.Map + + + def isSubtype(t: universe.Type, t2: universe.Type): Boolean = t <:< t2 + + /** + * Synchronize to prevent concurrent usage of `<:<` operator. + * This operator is not thread safe in any current version of scala; i.e. + * (2.11.12, 2.12.10, 2.13.0-M5). + * + * See https://github.com/scala/bug/issues/10766 + */ + /* + private[catalyst] def isSubtype(tpe1: `Type`, tpe2: `Type`): Boolean = { + ScalaReflection.ScalaSubtypeLock.synchronized { + tpe1 <:< tpe2 + } + } + */ + + private def dataTypeFor(tpe: `Type`): DataType = cleanUpReflectionObjects { + tpe.dealias match { + case t if isSubtype(t, definitions.NullTpe) => NullType + case t if isSubtype(t, definitions.IntTpe) => IntegerType + case t if isSubtype(t, definitions.LongTpe) => LongType + case t if isSubtype(t, definitions.DoubleTpe) => DoubleType + case t if isSubtype(t, definitions.FloatTpe) => FloatType + case t if isSubtype(t, definitions.ShortTpe) => ShortType + case t if isSubtype(t, definitions.ByteTpe) => ByteType + case t if isSubtype(t, definitions.BooleanTpe) => BooleanType + case t if isSubtype(t, localTypeOf[Array[Byte]]) => BinaryType + case t if isSubtype(t, localTypeOf[CalendarInterval]) => CalendarIntervalType + case t if isSubtype(t, localTypeOf[Decimal]) => DecimalType.SYSTEM_DEFAULT + case _ => { + val className = getClassNameFromType(tpe) + className match { + case "scala.Array" => { + val TypeRef(_, _, Seq(elementType)) = tpe + arrayClassFor(elementType) + } + case _ => { + val clazz = getClassFromType(tpe) + ObjectType(clazz) + } + } + } + } + } + + /** + * Given a type `T` this function constructs `ObjectType` that holds a class of type + * `Array[T]`. + * + * Special handling is performed for primitive types to map them back to their raw + * JVM form instead of the Scala Array that handles auto boxing. + */ + private def arrayClassFor(tpe: `Type`): ObjectType = cleanUpReflectionObjects { + val cls = tpe.dealias match { + case t if isSubtype(t, definitions.IntTpe) => classOf[Array[Int]] + case t if isSubtype(t, definitions.LongTpe) => classOf[Array[Long]] + case t if isSubtype(t, definitions.DoubleTpe) => classOf[Array[Double]] + case t if isSubtype(t, definitions.FloatTpe) => classOf[Array[Float]] + case t if isSubtype(t, definitions.ShortTpe) => classOf[Array[Short]] + case t if isSubtype(t, definitions.ByteTpe) => classOf[Array[Byte]] + case t if isSubtype(t, definitions.BooleanTpe) => classOf[Array[Boolean]] + case t if isSubtype(t, localTypeOf[Array[Byte]]) => classOf[Array[Array[Byte]]] + case t if isSubtype(t, localTypeOf[CalendarInterval]) => classOf[Array[CalendarInterval]] + case t if isSubtype(t, localTypeOf[Decimal]) => classOf[Array[Decimal]] + case other => { + // There is probably a better way to do this, but I couldn't find it... + val elementType = dataTypeFor(other).asInstanceOf[ObjectType].cls + java.lang.reflect.Array.newInstance(elementType, 0).getClass + } + + } + ObjectType(cls) + } + + /** + * Returns true if the value of this data type is same between internal and external. + */ + def isNativeType(dt: DataType): Boolean = dt match { + case NullType | BooleanType | ByteType | ShortType | IntegerType | LongType | + FloatType | DoubleType | BinaryType | CalendarIntervalType => { + true + } + case _ => false + } + + private def baseType(tpe: `Type`): `Type` = { + tpe.dealias match { + case annotatedType: AnnotatedType => annotatedType.underlying + case other => other + } + } + + /** + * Returns an expression that can be used to deserialize a Spark SQL representation to an object + * of type `T` with a compatible schema. The Spark SQL representation is located at ordinal 0 of + * a row, i.e., `GetColumnByOrdinal(0, _)`. Nested classes will have their fields accessed using + * `UnresolvedExtractValue`. + * + * The returned expression is used by `ExpressionEncoder`. The encoder will resolve and bind this + * deserializer expression when using it. + */ + def deserializerForType(tpe: `Type`): Expression = { + val clsName = getClassNameFromType(tpe) + val walkedTypePath = WalkedTypePath().recordRoot(clsName) + val Schema(dataType, nullable) = schemaFor(tpe) + + // Assumes we are deserializing the first column of a row. + deserializerForWithNullSafetyAndUpcast( + GetColumnByOrdinal(0, dataType), dataType, + nullable = nullable, walkedTypePath, + (casted, typePath) => deserializerFor(tpe, casted, typePath) + ) + } + + + /** + * Returns an expression that can be used to deserialize an input expression to an object of type + * `T` with a compatible schema. + * + * @param tpe The `Type` of deserialized object. + * @param path The expression which can be used to extract serialized value. + * @param walkedTypePath The paths from top to bottom to access current field when deserializing. + */ + private def deserializerFor( + tpe: `Type`, + path: Expression, + walkedTypePath: WalkedTypePath, + predefinedDt: Option[DataTypeWithClass] = None + ): Expression = cleanUpReflectionObjects { + baseType(tpe) match { + + // + case t if ( + try { + !dataTypeFor(t).isInstanceOf[ObjectType] + } catch { + case _: Throwable => false + }) && !predefinedDt.exists(_.isInstanceOf[ComplexWrapper]) => { + path + } + + case t if isSubtype(t, localTypeOf[java.lang.Integer]) => { + createDeserializerForTypesSupportValueOf(path, classOf[java.lang.Integer]) + } + case t if isSubtype(t, localTypeOf[Int]) => { + createDeserializerForTypesSupportValueOf(path, classOf[java.lang.Integer]) + } + case t if isSubtype(t, localTypeOf[java.lang.Long]) => { + createDeserializerForTypesSupportValueOf(path, classOf[java.lang.Long]) + } + case t if isSubtype(t, localTypeOf[Long]) => { + createDeserializerForTypesSupportValueOf(path, classOf[java.lang.Long]) + } + case t if isSubtype(t, localTypeOf[java.lang.Double]) => { + createDeserializerForTypesSupportValueOf(path, classOf[java.lang.Double]) + } + case t if isSubtype(t, localTypeOf[Double]) => { + createDeserializerForTypesSupportValueOf(path, classOf[java.lang.Double]) + } + case t if isSubtype(t, localTypeOf[java.lang.Float]) => { + createDeserializerForTypesSupportValueOf(path, classOf[java.lang.Float]) + } + case t if isSubtype(t, localTypeOf[Float]) => { + createDeserializerForTypesSupportValueOf(path, classOf[java.lang.Float]) + } + case t if isSubtype(t, localTypeOf[java.lang.Short]) => { + createDeserializerForTypesSupportValueOf(path, classOf[java.lang.Short]) + } + case t if isSubtype(t, localTypeOf[Short]) => { + createDeserializerForTypesSupportValueOf(path, classOf[java.lang.Short]) + } + case t if isSubtype(t, localTypeOf[java.lang.Byte]) => { + createDeserializerForTypesSupportValueOf(path, classOf[java.lang.Byte]) + } + case t if isSubtype(t, localTypeOf[Byte]) => { + createDeserializerForTypesSupportValueOf(path, classOf[java.lang.Byte]) + } + case t if isSubtype(t, localTypeOf[java.lang.Boolean]) => { + createDeserializerForTypesSupportValueOf(path, classOf[java.lang.Boolean]) + } + case t if isSubtype(t, localTypeOf[Boolean]) => { + createDeserializerForTypesSupportValueOf(path, classOf[java.lang.Boolean]) + } + case t if isSubtype(t, localTypeOf[java.time.LocalDate]) => { + createDeserializerForLocalDate(path) + } + case t if isSubtype(t, localTypeOf[java.sql.Date]) => { + createDeserializerForSqlDate(path) + } // + + case t if isSubtype(t, localTypeOf[java.time.Instant]) => { + createDeserializerForInstant(path) + } + case t if isSubtype(t, localTypeOf[java.lang.Enum[_]]) => { + createDeserializerForTypesSupportValueOf( + Invoke(path, "toString", ObjectType(classOf[String]), returnNullable = false), + getClassFromType(t), + ) + } + case t if isSubtype(t, localTypeOf[java.sql.Timestamp]) => { + createDeserializerForSqlTimestamp(path) + } + case t if isSubtype(t, localTypeOf[java.time.LocalDateTime]) => { + throw new IllegalArgumentException("java.time.LocalDateTime is supported in Spark 3.2+") + } + case t if isSubtype(t, localTypeOf[java.time.Duration]) => { + throw new IllegalArgumentException("java.time.Duration is supported in Spark 3.2+") + } + case t if isSubtype(t, localTypeOf[java.time.Period]) => { + throw new IllegalArgumentException("java.time.Period is supported in Spark 3.2+") + } + case t if isSubtype(t, localTypeOf[java.lang.String]) => { + createDeserializerForString(path, returnNullable = false) + } + case t if isSubtype(t, localTypeOf[java.math.BigDecimal]) => { + createDeserializerForJavaBigDecimal(path, returnNullable = false) + } + case t if isSubtype(t, localTypeOf[BigDecimal]) => { + createDeserializerForScalaBigDecimal(path, returnNullable = false) + } + case t if isSubtype(t, localTypeOf[java.math.BigInteger]) => { + createDeserializerForJavaBigInteger(path, returnNullable = false) + } + case t if isSubtype(t, localTypeOf[scala.math.BigInt]) => { + createDeserializerForScalaBigInt(path) + } + + case t if isSubtype(t, localTypeOf[Array[_]]) => { + var TypeRef(_, _, Seq(elementType)) = t + if (predefinedDt.isDefined && !elementType.dealias.typeSymbol.isClass) + elementType = getType(predefinedDt.get.asInstanceOf[KComplexTypeWrapper].dt.asInstanceOf[ArrayType] + .elementType.asInstanceOf[DataTypeWithClass].cls + ) + val Schema(dataType, elementNullable) = predefinedDt.map { it => + val elementInfo = it.asInstanceOf[KComplexTypeWrapper].dt.asInstanceOf[ArrayType].elementType + .asInstanceOf[DataTypeWithClass] + Schema(elementInfo.dt, elementInfo.nullable) + }.getOrElse(schemaFor(elementType)) + val className = getClassNameFromType(elementType) + val newTypePath = walkedTypePath.recordArray(className) + + val mapFunction: Expression => Expression = element => { + // upcast the array element to the data type the encoder expected. + deserializerForWithNullSafetyAndUpcast( + element, + dataType, + nullable = elementNullable, + newTypePath, + (casted, typePath) => deserializerFor( + tpe = elementType, + path = casted, + walkedTypePath = typePath, + predefinedDt = predefinedDt + .map(_.asInstanceOf[KComplexTypeWrapper].dt.asInstanceOf[ArrayType].elementType) + .filter(_.isInstanceOf[ComplexWrapper]) + .map(_.asInstanceOf[ComplexWrapper]) + ) + ) + } + + val arrayData = UnresolvedMapObjects(mapFunction, path) + val arrayCls = arrayClassFor(elementType) + + val methodName = elementType match { + case t if isSubtype(t, definitions.IntTpe) => "toIntArray" + case t if isSubtype(t, definitions.LongTpe) => "toLongArray" + case t if isSubtype(t, definitions.DoubleTpe) => "toDoubleArray" + case t if isSubtype(t, definitions.FloatTpe) => "toFloatArray" + case t if isSubtype(t, definitions.ShortTpe) => "toShortArray" + case t if isSubtype(t, definitions.ByteTpe) => "toByteArray" + case t if isSubtype(t, definitions.BooleanTpe) => "toBooleanArray" + // non-primitive + case _ => "array" + } + Invoke(arrayData, methodName, arrayCls, returnNullable = false) + } + + // We serialize a `Set` to Catalyst array. When we deserialize a Catalyst array + // to a `Set`, if there are duplicated elements, the elements will be de-duplicated. + + case t if isSubtype(t, localTypeOf[Map[_, _]]) => { + val TypeRef(_, _, Seq(keyType, valueType)) = t + + val classNameForKey = getClassNameFromType(keyType) + val classNameForValue = getClassNameFromType(valueType) + + val newTypePath = walkedTypePath.recordMap(classNameForKey, classNameForValue) + + UnresolvedCatalystToExternalMap( + path, + p => deserializerFor(keyType, p, newTypePath), + p => deserializerFor(valueType, p, newTypePath), + mirror.runtimeClass(t.typeSymbol.asClass) + ) + } + + case t if isSubtype(t, localTypeOf[java.lang.Enum[_]]) => { + createDeserializerForTypesSupportValueOf( + createDeserializerForString(path, returnNullable = false), + Class.forName(t.toString), + ) + } + case t if t.typeSymbol.annotations.exists(_.tree.tpe =:= typeOf[SQLUserDefinedType]) => { + val udt = getClassFromType(t).getAnnotation(classOf[SQLUserDefinedType]).udt(). + getConstructor().newInstance() + val obj = NewInstance( + udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt(), + Nil, + dataType = ObjectType(udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt()) + ) + Invoke(obj, "deserialize", ObjectType(udt.userClass), path :: Nil) + } + + case t if UDTRegistration.exists(getClassNameFromType(t)) => { + val udt = UDTRegistration.getUDTFor(getClassNameFromType(t)).get.getConstructor(). + newInstance().asInstanceOf[UserDefinedType[_]] + val obj = NewInstance( + udt.getClass, + Nil, + dataType = ObjectType(udt.getClass) + ) + Invoke(obj, "deserialize", ObjectType(udt.userClass), path :: Nil) + } + + case _ if predefinedDt.isDefined => { + predefinedDt.get match { + + case wrapper: KDataTypeWrapper => { + val structType = wrapper.dt + val cls = wrapper.cls + val arguments = structType + .fields + .map { field => + val dataType = field.dataType.asInstanceOf[DataTypeWithClass] + val nullable = dataType.nullable + val clsName = getClassNameFromType(getType(dataType.cls)) + val newTypePath = walkedTypePath.recordField(clsName, field.name) + + // For tuples, we based grab the inner fields by ordinal instead of name. + val newPath = deserializerFor( + tpe = getType(dataType.cls), + path = addToPath(path, field.name, dataType.dt, newTypePath), + walkedTypePath = newTypePath, + predefinedDt = Some(dataType).filter(_.isInstanceOf[ComplexWrapper]) + ) + expressionWithNullSafety( + newPath, + nullable = nullable, + newTypePath + ) + } + val newInstance = NewInstance(cls, arguments, ObjectType(cls), propagateNull = false) + + org.apache.spark.sql.catalyst.expressions.If( + IsNull(path), + org.apache.spark.sql.catalyst.expressions.Literal.create(null, ObjectType(cls)), + newInstance + ) + } + + case t: ComplexWrapper => { + + t.dt match { + case MapType(kt, vt, _) => { + val Seq(keyType, valueType) = Seq(kt, vt).map(_.asInstanceOf[DataTypeWithClass].cls) + .map(getType(_)) + val Seq(keyDT, valueDT) = Seq(kt, vt).map(_.asInstanceOf[DataTypeWithClass]) + val classNameForKey = getClassNameFromType(keyType) + val classNameForValue = getClassNameFromType(valueType) + + val newTypePath = walkedTypePath.recordMap(classNameForKey, classNameForValue) + + val keyData = + Invoke( + UnresolvedMapObjects( + p => deserializerFor( + keyType, p, newTypePath, Some(keyDT) + .filter(_.isInstanceOf[ComplexWrapper]) + ), + MapKeys(path) + ), + "array", + ObjectType(classOf[Array[Any]]) + ) + + val valueData = + Invoke( + UnresolvedMapObjects( + p => deserializerFor( + valueType, p, newTypePath, Some(valueDT) + .filter(_.isInstanceOf[ComplexWrapper]) + ), + MapValues(path) + ), + "array", + ObjectType(classOf[Array[Any]]) + ) + + StaticInvoke( + ArrayBasedMapData.getClass, + ObjectType(classOf[java.util.Map[_, _]]), + "toJavaMap", + keyData :: valueData :: Nil, + returnNullable = false + ) + } + + case ArrayType(elementType, containsNull) => { + val dataTypeWithClass = elementType.asInstanceOf[DataTypeWithClass] + val mapFunction: Expression => Expression = element => { + // upcast the array element to the data type the encoder expected. + val et = getType(dataTypeWithClass.cls) + val className = getClassNameFromType(et) + val newTypePath = walkedTypePath.recordArray(className) + deserializerForWithNullSafetyAndUpcast( + element, + dataTypeWithClass.dt, + nullable = dataTypeWithClass.nullable, + newTypePath, + (casted, typePath) => { + deserializerFor( + et, casted, typePath, Some(dataTypeWithClass) + .filter(_.isInstanceOf[ComplexWrapper]) + .map(_.asInstanceOf[ComplexWrapper]) + ) + } + ) + } + + UnresolvedMapObjects(mapFunction, path, customCollectionCls = Some(t.cls)) + } + + case StructType(elementType: Array[StructField]) => { + val cls = t.cls + + val arguments = elementType.map { field => + val dataType = field.dataType.asInstanceOf[DataTypeWithClass] + val nullable = dataType.nullable + val clsName = getClassNameFromType(getType(dataType.cls)) + val newTypePath = walkedTypePath.recordField(clsName, field.name) + + // For tuples, we based grab the inner fields by ordinal instead of name. + val newPath = deserializerFor( + getType(dataType.cls), + addToPath(path, field.name, dataType.dt, newTypePath), + newTypePath, + Some(dataType).filter(_.isInstanceOf[ComplexWrapper]) + ) + expressionWithNullSafety( + newPath, + nullable = nullable, + newTypePath + ) + } + val newInstance = NewInstance(cls, arguments, ObjectType(cls), propagateNull = false) + + org.apache.spark.sql.catalyst.expressions.If( + IsNull(path), + org.apache.spark.sql.catalyst.expressions.Literal.create(null, ObjectType(cls)), + newInstance + ) + } + + case _ => { + throw new UnsupportedOperationException( + s"No Encoder found for $tpe\n" + walkedTypePath + ) + } + } + } + } + } + + case t if definedByConstructorParams(t) => { + val params = getConstructorParameters(t) + + val cls = getClassFromType(tpe) + + val arguments = params.zipWithIndex.map { case ((fieldName, fieldType), i) => + val Schema(dataType, nullable) = schemaFor(fieldType) + val clsName = getClassNameFromType(fieldType) + val newTypePath = walkedTypePath.recordField(clsName, fieldName) + + // For tuples, we based grab the inner fields by ordinal instead of name. + val newPath = if (cls.getName startsWith "scala.Tuple") { + deserializerFor( + fieldType, + addToPathOrdinal(path, i, dataType, newTypePath), + newTypePath + ) + } else { + deserializerFor( + fieldType, + addToPath(path, fieldName, dataType, newTypePath), + newTypePath + ) + } + expressionWithNullSafety( + newPath, + nullable = nullable, + newTypePath + ) + } + + val newInstance = NewInstance(cls, arguments, ObjectType(cls), propagateNull = false) + + org.apache.spark.sql.catalyst.expressions.If( + IsNull(path), + org.apache.spark.sql.catalyst.expressions.Literal.create(null, ObjectType(cls)), + newInstance + ) + } + + case _ => { + throw new UnsupportedOperationException( + s"No Encoder found for $tpe\n" + walkedTypePath + ) + } + } + } + + /** + * Returns an expression for serializing an object of type T to Spark SQL representation. The + * input object is located at ordinal 0 of a row, i.e., `BoundReference(0, _)`. + * + * If the given type is not supported, i.e. there is no encoder can be built for this type, + * an [[UnsupportedOperationException]] will be thrown with detailed error message to explain + * the type path walked so far and which class we are not supporting. + * There are 4 kinds of type path: + * * the root type: `root class: "abc.xyz.MyClass"` + * * the value type of [[Option]]: `option value class: "abc.xyz.MyClass"` + * * the element type of [[Array]] or [[Seq]]: `array element class: "abc.xyz.MyClass"` + * * the field of [[Product]]: `field (class: "abc.xyz.MyClass", name: "myField")` + */ + def serializerForType(tpe: `Type`): Expression = ScalaReflection.cleanUpReflectionObjects { + val clsName = getClassNameFromType(tpe) + val walkedTypePath = WalkedTypePath().recordRoot(clsName) + + // The input object to `ExpressionEncoder` is located at first column of an row. + val isPrimitive = tpe.typeSymbol.asClass.isPrimitive + val inputObject = BoundReference(0, dataTypeFor(tpe), nullable = !isPrimitive) + + serializerFor(inputObject, tpe, walkedTypePath) + } + + def getType[T](clazz: Class[T]): universe.Type = { + clazz match { + case _ if clazz == classOf[Array[Byte]] => localTypeOf[Array[Byte]] + case _ => { + val mir = runtimeMirror(clazz.getClassLoader) + mir.classSymbol(clazz).toType + } + } + + } + + def deserializerFor(cls: java.lang.Class[_], dt: DataTypeWithClass): Expression = { + val tpe = getType(cls) + val clsName = getClassNameFromType(tpe) + val walkedTypePath = WalkedTypePath().recordRoot(clsName) + + // Assumes we are deserializing the first column of a row. + deserializerForWithNullSafetyAndUpcast( + GetColumnByOrdinal(0, dt.dt), + dt.dt, + nullable = dt.nullable, + walkedTypePath, + (casted, typePath) => deserializerFor(tpe, casted, typePath, Some(dt)) + ) + } + + + def serializerFor(cls: java.lang.Class[_], dt: DataTypeWithClass): Expression = { + val tpe = getType(cls) + val clsName = getClassNameFromType(tpe) + val walkedTypePath = WalkedTypePath().recordRoot(clsName) + val inputObject = BoundReference(0, ObjectType(cls), nullable = true) + serializerFor(inputObject, tpe, walkedTypePath, predefinedDt = Some(dt)) + } + + /** + * Returns an expression for serializing the value of an input expression into Spark SQL + * internal representation. + */ + private def serializerFor( + inputObject: Expression, + tpe: `Type`, + walkedTypePath: WalkedTypePath, + seenTypeSet: Set[`Type`] = Set.empty, + predefinedDt: Option[DataTypeWithClass] = None, + ): Expression = cleanUpReflectionObjects { + + def toCatalystArray( + input: Expression, + elementType: `Type`, + predefinedDt: Option[DataTypeWithClass] = None, + ): Expression = { + val dataType = predefinedDt + .map(_.dt) + .getOrElse { + dataTypeFor(elementType) + } + + dataType match { + + case dt @ (MapType(_, _, _) | ArrayType(_, _) | StructType(_)) => { + val clsName = getClassNameFromType(elementType) + val newPath = walkedTypePath.recordArray(clsName) + createSerializerForMapObjects( + input, ObjectType(predefinedDt.get.cls), + serializerFor(_, elementType, newPath, seenTypeSet, predefinedDt) + ) + } + + case dt: ObjectType => { + val clsName = getClassNameFromType(elementType) + val newPath = walkedTypePath.recordArray(clsName) + createSerializerForMapObjects( + input, dt, + serializerFor(_, elementType, newPath, seenTypeSet) + ) + } + + // case dt: ByteType => + // createSerializerForPrimitiveArray(input, dt) + + case dt @ (BooleanType | ByteType | ShortType | IntegerType | LongType | FloatType | DoubleType) => { + val cls = input.dataType.asInstanceOf[ObjectType].cls + if (cls.isArray && cls.getComponentType.isPrimitive) { + createSerializerForPrimitiveArray(input, dt) + } else { + createSerializerForGenericArray( + inputObject = input, + dataType = dt, + nullable = predefinedDt + .map(_.nullable) + .getOrElse( + schemaFor(elementType).nullable + ), + ) + } + } + + case _: StringType => { + val clsName = getClassNameFromType(typeOf[String]) + val newPath = walkedTypePath.recordArray(clsName) + createSerializerForMapObjects( + input, ObjectType(Class.forName(getClassNameFromType(elementType))), + serializerFor(_, elementType, newPath, seenTypeSet) + ) + } + + case dt => { + createSerializerForGenericArray( + inputObject = input, + dataType = dt, + nullable = predefinedDt + .map(_.nullable) + .getOrElse { + schemaFor(elementType).nullable + }, + ) + } + } + } + + baseType(tpe) match { + + // + case _ if !inputObject.dataType.isInstanceOf[ObjectType] && + !predefinedDt.exists(_.isInstanceOf[ComplexWrapper]) => { + inputObject + } + case t if isSubtype(t, localTypeOf[Option[_]]) => { + val TypeRef(_, _, Seq(optType)) = t + val className = getClassNameFromType(optType) + val newPath = walkedTypePath.recordOption(className) + val unwrapped = UnwrapOption(dataTypeFor(optType), inputObject) + serializerFor(unwrapped, optType, newPath, seenTypeSet) + } + + // Since List[_] also belongs to localTypeOf[Product], we put this case before + // "case t if definedByConstructorParams(t)" to make sure it will match to the + // case "localTypeOf[Seq[_]]" + case t if isSubtype(t, localTypeOf[Seq[_]]) => { + val TypeRef(_, _, Seq(elementType)) = t + toCatalystArray(inputObject, elementType) + } + + case t if isSubtype(t, localTypeOf[Array[_]]) && predefinedDt.isEmpty => { + val TypeRef(_, _, Seq(elementType)) = t + toCatalystArray(inputObject, elementType) + } + + case t if isSubtype(t, localTypeOf[Map[_, _]]) => { + val TypeRef(_, _, Seq(keyType, valueType)) = t + val keyClsName = getClassNameFromType(keyType) + val valueClsName = getClassNameFromType(valueType) + val keyPath = walkedTypePath.recordKeyForMap(keyClsName) + val valuePath = walkedTypePath.recordValueForMap(valueClsName) + + createSerializerForMap( + inputObject, + MapElementInformation( + dataTypeFor(keyType), + nullable = !keyType.typeSymbol.asClass.isPrimitive, + serializerFor(_, keyType, keyPath, seenTypeSet) + ), + MapElementInformation( + dataTypeFor(valueType), + nullable = !valueType.typeSymbol.asClass.isPrimitive, + serializerFor(_, valueType, valuePath, seenTypeSet) + ) + ) + } + + case t if isSubtype(t, localTypeOf[scala.collection.Set[_]]) => { + val TypeRef(_, _, Seq(elementType)) = t + + // There's no corresponding Catalyst type for `Set`, we serialize a `Set` to Catalyst array. + // Note that the property of `Set` is only kept when manipulating the data as domain object. + val newInput = + Invoke( + inputObject, + "toSeq", + ObjectType(classOf[Seq[_]]) + ) + + toCatalystArray(newInput, elementType) + } + + case t if isSubtype(t, localTypeOf[String]) => { + createSerializerForString(inputObject) + } + case t if isSubtype(t, localTypeOf[java.time.Instant]) => { + createSerializerForJavaInstant(inputObject) + } + case t if isSubtype(t, localTypeOf[java.sql.Timestamp]) => { + createSerializerForSqlTimestamp(inputObject) + } + case t if isSubtype(t, localTypeOf[java.time.LocalDateTime]) => { + throw new IllegalArgumentException("java.time.LocalDateTime is supported in Spark 3.2+") + } + case t if isSubtype(t, localTypeOf[java.time.LocalDate]) => { + createSerializerForJavaLocalDate(inputObject) + } + case t if isSubtype(t, localTypeOf[java.sql.Date]) => { + createSerializerForSqlDate(inputObject) + } + case t if isSubtype(t, localTypeOf[java.time.Duration]) => { + throw new IllegalArgumentException("java.time.Duration is supported in Spark 3.2+") + } + case t if isSubtype(t, localTypeOf[java.time.Period]) => { + throw new IllegalArgumentException("java.time.Period is supported in Spark 3.2+") + } + case t if isSubtype(t, localTypeOf[BigDecimal]) => { + createSerializerForScalaBigDecimal(inputObject) + } + case t if isSubtype(t, localTypeOf[java.math.BigDecimal]) => { + createSerializerForJavaBigDecimal(inputObject) + } + case t if isSubtype(t, localTypeOf[java.math.BigInteger]) => { + createSerializerForJavaBigInteger(inputObject) + } + case t if isSubtype(t, localTypeOf[scala.math.BigInt]) => { + createSerializerForScalaBigInt(inputObject) + } + + case t if isSubtype(t, localTypeOf[java.lang.Integer]) => { + createSerializerForInteger(inputObject) + } + case t if isSubtype(t, localTypeOf[Int]) => { + createSerializerForInteger(inputObject) + } + case t if isSubtype(t, localTypeOf[java.lang.Long]) => { + createSerializerForLong(inputObject) + } + case t if isSubtype(t, localTypeOf[Long]) => { + createSerializerForLong(inputObject) + } + case t if isSubtype(t, localTypeOf[java.lang.Double]) => { + createSerializerForDouble(inputObject) + } + case t if isSubtype(t, localTypeOf[Double]) => { + createSerializerForDouble(inputObject) + } + case t if isSubtype(t, localTypeOf[java.lang.Float]) => { + createSerializerForFloat(inputObject) + } + case t if isSubtype(t, localTypeOf[Float]) => { + createSerializerForFloat(inputObject) + } + case t if isSubtype(t, localTypeOf[java.lang.Short]) => { + createSerializerForShort(inputObject) + } + case t if isSubtype(t, localTypeOf[Short]) => { + createSerializerForShort(inputObject) + } + case t if isSubtype(t, localTypeOf[java.lang.Byte]) => { + createSerializerForByte(inputObject) + } + case t if isSubtype(t, localTypeOf[Byte]) => { + createSerializerForByte(inputObject) + } + case t if isSubtype(t, localTypeOf[java.lang.Boolean]) => { + createSerializerForBoolean(inputObject) + } + case t if isSubtype(t, localTypeOf[Boolean]) => { + createSerializerForBoolean(inputObject) + } + case t if isSubtype(t, localTypeOf[java.lang.Enum[_]]) => { + createSerializerForString( + Invoke(inputObject, "name", ObjectType(classOf[String]), returnNullable = false) + ) + } + case t if t.typeSymbol.annotations.exists(_.tree.tpe =:= typeOf[SQLUserDefinedType]) => { + val udt = getClassFromType(t) + .getAnnotation(classOf[SQLUserDefinedType]).udt().getConstructor().newInstance() + val udtClass = udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt() + createSerializerForUserDefinedType(inputObject, udt, udtClass) + } + + case t if UDTRegistration.exists(getClassNameFromType(t)) => { + val udt = UDTRegistration.getUDTFor(getClassNameFromType(t)).get.getConstructor(). + newInstance().asInstanceOf[UserDefinedType[_]] + val udtClass = udt.getClass + createSerializerForUserDefinedType(inputObject, udt, udtClass) + } + // + + // Kotlin specific cases + case t if predefinedDt.isDefined => { + +// if (seenTypeSet.contains(t)) { +// throw new UnsupportedOperationException( +// s"cannot have circular references in class, but got the circular reference of class $t" +// ) +// } + + predefinedDt.get match { + + // Kotlin data class + case dataType: KDataTypeWrapper => { + val cls = dataType.cls + val properties = getJavaBeanReadableProperties(cls) + val structFields = dataType.dt.fields.map(_.asInstanceOf[KStructField]) + val fields: Array[(String, Expression)] = structFields.map { structField => + val maybeProp = properties.find(it => it.getReadMethod.getName == structField.getterName) + if (maybeProp.isEmpty) throw new IllegalArgumentException(s"Field ${ + structField.name + } is not found among available props, which are: ${properties.map(_.getName).mkString(", ")}" + ) + val fieldName = structField.name + val propClass = structField.dataType.asInstanceOf[DataTypeWithClass].cls + val propDt = structField.dataType.asInstanceOf[DataTypeWithClass] + + val fieldValue = Invoke( + inputObject, + maybeProp.get.getReadMethod.getName, + inferExternalType(propClass), + returnNullable = structField.nullable + ) + val newPath = walkedTypePath.recordField(propClass.getName, fieldName) + + val tpe = getType(propClass) + + val serializer = serializerFor( + inputObject = fieldValue, + tpe = tpe, + walkedTypePath = newPath, + seenTypeSet = seenTypeSet, + predefinedDt = if (propDt.isInstanceOf[ComplexWrapper]) Some(propDt) else None + ) + + (fieldName, serializer) + } + createSerializerForObject(inputObject, fields) + } + + case otherTypeWrapper: ComplexWrapper => { + + otherTypeWrapper.dt match { + + case MapType(kt, vt, _) => { + val Seq(keyType, valueType) = Seq(kt, vt).map(_.asInstanceOf[DataTypeWithClass].cls) + .map(getType(_)) + val Seq(keyDT, valueDT) = Seq(kt, vt).map(_.asInstanceOf[DataTypeWithClass]) + val keyClsName = getClassNameFromType(keyType) + val valueClsName = getClassNameFromType(valueType) + val keyPath = walkedTypePath.recordKeyForMap(keyClsName) + val valuePath = walkedTypePath.recordValueForMap(valueClsName) + + createSerializerForMap( + inputObject, + MapElementInformation( + dataTypeFor(keyType), + nullable = !keyType.typeSymbol.asClass.isPrimitive, + serializerFor( + _, keyType, keyPath, seenTypeSet, Some(keyDT) + .filter(_.isInstanceOf[ComplexWrapper]) + ) + ), + MapElementInformation( + dataTypeFor(valueType), + nullable = !valueType.typeSymbol.asClass.isPrimitive, + serializerFor( + _, valueType, valuePath, seenTypeSet, Some(valueDT) + .filter(_.isInstanceOf[ComplexWrapper]) + ) + ) + ) + } + + case ArrayType(elementType, _) => { + toCatalystArray( + inputObject, + getType(elementType.asInstanceOf[DataTypeWithClass].cls + ), Some(elementType.asInstanceOf[DataTypeWithClass]) + ) + } + + case StructType(elementType: Array[StructField]) => { + val cls = otherTypeWrapper.cls + val names = elementType.map(_.name) + + val beanInfo = Introspector.getBeanInfo(cls) + val methods = beanInfo.getMethodDescriptors.filter(it => names.contains(it.getName)) + + + val fields = elementType.map { structField => + + val maybeProp = methods.find(it => it.getName == structField.name) + if (maybeProp.isEmpty) throw new IllegalArgumentException(s"Field ${ + structField.name + } is not found among available props, which are: ${ + methods.map(_.getName).mkString(", ") + }" + ) + val fieldName = structField.name + val propClass = structField.dataType.asInstanceOf[DataTypeWithClass].cls + val propDt = structField.dataType.asInstanceOf[DataTypeWithClass] + val fieldValue = Invoke( + inputObject, + maybeProp.get.getName, + inferExternalType(propClass), + returnNullable = propDt.nullable + ) + val newPath = walkedTypePath.recordField(propClass.getName, fieldName) + (fieldName, serializerFor( + fieldValue, getType(propClass), newPath, seenTypeSet, if (propDt + .isInstanceOf[ComplexWrapper]) Some(propDt) else None + )) + + } + createSerializerForObject(inputObject, fields) + } + + case _ => { + throw new UnsupportedOperationException( + s"No Encoder found for $tpe\n" + walkedTypePath + ) + } + } + } + } + } + + case t if definedByConstructorParams(t) => { + if (seenTypeSet.contains(t)) { + throw new UnsupportedOperationException( + s"cannot have circular references in class, but got the circular reference of class $t" + ) + } + + val params = getConstructorParameters(t) + val fields = params.map { case (fieldName, fieldType) => + if (javaKeywords.contains(fieldName)) { + throw new UnsupportedOperationException(s"`$fieldName` is a reserved keyword and " + + "cannot be used as field name\n" + walkedTypePath + ) + } + + // SPARK-26730 inputObject won't be null with If's guard below. And KnownNotNul + // is necessary here. Because for a nullable nested inputObject with struct data + // type, e.g. StructType(IntegerType, StringType), it will return nullable=true + // for IntegerType without KnownNotNull. And that's what we do not expect to. + val fieldValue = Invoke( + KnownNotNull(inputObject), fieldName, dataTypeFor(fieldType), + returnNullable = !fieldType.typeSymbol.asClass.isPrimitive + ) + val clsName = getClassNameFromType(fieldType) + val newPath = walkedTypePath.recordField(clsName, fieldName) + (fieldName, serializerFor(fieldValue, fieldType, newPath, seenTypeSet + t)) + } + createSerializerForObject(inputObject, fields) + } + + case _ => { + throw new UnsupportedOperationException( + s"No Encoder found for $tpe\n" + walkedTypePath + ) + } + } + } + + def createDeserializerForString(path: Expression, returnNullable: Boolean): Expression = { + Invoke( + path, "toString", ObjectType(classOf[java.lang.String]), + returnNullable = returnNullable + ) + } + + def getJavaBeanReadableProperties(beanClass: Class[_]): Array[PropertyDescriptor] = { + val beanInfo = Introspector.getBeanInfo(beanClass) + beanInfo.getPropertyDescriptors.filterNot(_.getName == "class") + .filterNot(_.getName == "declaringClass") + .filter(_.getReadMethod != null) + } + + /* + * Retrieves the runtime class corresponding to the provided type. + */ + def getClassFromType(tpe: Type): Class[_] = mirror.runtimeClass(tpe.dealias.typeSymbol.asClass) + + case class Schema(dataType: DataType, nullable: Boolean) + + /** Returns a catalyst DataType and its nullability for the given Scala Type using reflection. */ + def schemaFor(tpe: `Type`): Schema = cleanUpReflectionObjects { + + baseType(tpe) match { + // this must be the first case, since all objects in scala are instances of Null, therefore + // Null type would wrongly match the first of them, which is Option as of now + case t if isSubtype(t, definitions.NullTpe) => Schema(NullType, nullable = true) + + case t if t.typeSymbol.annotations.exists(_.tree.tpe =:= typeOf[SQLUserDefinedType]) => { + val udt = getClassFromType(t).getAnnotation(classOf[SQLUserDefinedType]).udt(). + getConstructor().newInstance() + Schema(udt, nullable = true) + } + case t if UDTRegistration.exists(getClassNameFromType(t)) => { + val udt = UDTRegistration + .getUDTFor(getClassNameFromType(t)) + .get + .getConstructor() + .newInstance() + .asInstanceOf[UserDefinedType[_]] + Schema(udt, nullable = true) + } + case t if isSubtype(t, localTypeOf[Option[_]]) => { + val TypeRef(_, _, Seq(optType)) = t + Schema(schemaFor(optType).dataType, nullable = true) + } + case t if isSubtype(t, localTypeOf[Array[Byte]]) => { + Schema(BinaryType, nullable = true) + } + case t if isSubtype(t, localTypeOf[Array[_]]) => { + val TypeRef(_, _, Seq(elementType)) = t + val Schema(dataType, nullable) = schemaFor(elementType) + Schema(ArrayType(dataType, containsNull = nullable), nullable = true) + } + case t if isSubtype(t, localTypeOf[Seq[_]]) => { + val TypeRef(_, _, Seq(elementType)) = t + val Schema(dataType, nullable) = schemaFor(elementType) + Schema(ArrayType(dataType, containsNull = nullable), nullable = true) + } + case t if isSubtype(t, localTypeOf[Map[_, _]]) => { + val TypeRef(_, _, Seq(keyType, valueType)) = t + val Schema(valueDataType, valueNullable) = schemaFor(valueType) + Schema( + MapType( + schemaFor(keyType).dataType, + valueDataType, valueContainsNull = valueNullable + ), nullable = true + ) + } + case t if isSubtype(t, localTypeOf[Set[_]]) => { + val TypeRef(_, _, Seq(elementType)) = t + val Schema(dataType, nullable) = schemaFor(elementType) + Schema(ArrayType(dataType, containsNull = nullable), nullable = true) + } + case t if isSubtype(t, localTypeOf[String]) => { + Schema(StringType, nullable = true) + } + case t if isSubtype(t, localTypeOf[java.time.Instant]) => { + Schema(TimestampType, nullable = true) + } + case t if isSubtype(t, localTypeOf[java.sql.Timestamp]) => { + Schema(TimestampType, nullable = true) + } + // SPARK-36227: Remove TimestampNTZ type support in Spark 3.2 with minimal code changes. + case t if isSubtype(t, localTypeOf[java.time.LocalDateTime]) && Utils.isTesting => { + throw new IllegalArgumentException("TimestampNTZType is supported in spark 3.2+") + } + case t if isSubtype(t, localTypeOf[java.time.LocalDate]) => { + Schema(DateType, nullable = true) + } + case t if isSubtype(t, localTypeOf[java.sql.Date]) => { + Schema(DateType, nullable = true) + } + case t if isSubtype(t, localTypeOf[CalendarInterval]) => { + Schema(CalendarIntervalType, nullable = true) + } + case t if isSubtype(t, localTypeOf[java.time.Duration]) => { + throw new IllegalArgumentException("DayTimeIntervalType for java.time.Duration is supported in spark 3.2+") + } + case t if isSubtype(t, localTypeOf[java.time.Period]) => { + throw new IllegalArgumentException("YearMonthIntervalType for java.time.Period is supported in spark 3.2+") + } + case t if isSubtype(t, localTypeOf[BigDecimal]) => { + Schema(DecimalType.SYSTEM_DEFAULT, nullable = true) + } + case t if isSubtype(t, localTypeOf[java.math.BigDecimal]) => { + Schema(DecimalType.SYSTEM_DEFAULT, nullable = true) + } + case t if isSubtype(t, localTypeOf[java.math.BigInteger]) => { + Schema(DecimalType.BigIntDecimal, nullable = true) + } + case t if isSubtype(t, localTypeOf[scala.math.BigInt]) => { + Schema(DecimalType.BigIntDecimal, nullable = true) + } + case t if isSubtype(t, localTypeOf[Decimal]) => { + Schema(DecimalType.SYSTEM_DEFAULT, nullable = true) + } + case t if isSubtype(t, localTypeOf[java.lang.Integer]) => Schema(IntegerType, nullable = true) + case t if isSubtype(t, localTypeOf[java.lang.Long]) => Schema(LongType, nullable = true) + case t if isSubtype(t, localTypeOf[java.lang.Double]) => Schema(DoubleType, nullable = true) + case t if isSubtype(t, localTypeOf[java.lang.Float]) => Schema(FloatType, nullable = true) + case t if isSubtype(t, localTypeOf[java.lang.Short]) => Schema(ShortType, nullable = true) + case t if isSubtype(t, localTypeOf[java.lang.Byte]) => Schema(ByteType, nullable = true) + case t if isSubtype(t, localTypeOf[java.lang.Boolean]) => Schema(BooleanType, nullable = true) + case t if isSubtype(t, definitions.IntTpe) => Schema(IntegerType, nullable = false) + case t if isSubtype(t, definitions.LongTpe) => Schema(LongType, nullable = false) + case t if isSubtype(t, definitions.DoubleTpe) => Schema(DoubleType, nullable = false) + case t if isSubtype(t, definitions.FloatTpe) => Schema(FloatType, nullable = false) + case t if isSubtype(t, definitions.ShortTpe) => Schema(ShortType, nullable = false) + case t if isSubtype(t, definitions.ByteTpe) => Schema(ByteType, nullable = false) + case t if isSubtype(t, definitions.BooleanTpe) => Schema(BooleanType, nullable = false) + case t if definedByConstructorParams(t) => { + val params = getConstructorParameters(t) + Schema( + StructType( + params.map { case (fieldName, fieldType) => + val Schema(dataType, nullable) = schemaFor(fieldType) + StructField(fieldName, dataType, nullable) + } + ), nullable = true + ) + } + case other => { + throw new UnsupportedOperationException(s"Schema for type $other is not supported") + } + } + } + + /** + * Whether the fields of the given type is defined entirely by its constructor parameters. + */ + def definedByConstructorParams(tpe: Type): Boolean = cleanUpReflectionObjects { + tpe.dealias match { + // `Option` is a `Product`, but we don't wanna treat `Option[Int]` as a struct type. + case t if isSubtype(t, localTypeOf[Option[_]]) => definedByConstructorParams(t.typeArgs.head) + case _ => { + isSubtype(tpe.dealias, localTypeOf[Product]) || + isSubtype(tpe.dealias, localTypeOf[DefinedByConstructorParams]) + } + } + } + + private val javaKeywords = Set( + "abstract", "assert", "boolean", "break", "byte", "case", "catch", + "char", "class", "const", "continue", "default", "do", "double", "else", "extends", "false", + "final", "finally", "float", "for", "goto", "if", "implements", "import", "instanceof", "int", + "interface", "long", "native", "new", "null", "package", "private", "protected", "public", + "return", "short", "static", "strictfp", "super", "switch", "synchronized", "this", "throw", + "throws", "transient", "true", "try", "void", "volatile", "while" + ) + + + @scala.annotation.tailrec + def javaBoxedType(dt: DataType): Class[_] = dt match { + case _: DecimalType => classOf[Decimal] + case BinaryType => classOf[Array[Byte]] + case StringType => classOf[UTF8String] + case CalendarIntervalType => classOf[CalendarInterval] + case _: StructType => classOf[InternalRow] + case _: ArrayType => classOf[ArrayType] + case _: MapType => classOf[MapType] + case udt: UserDefinedType[_] => javaBoxedType(udt.sqlType) + case ObjectType(cls) => cls + case _ => ScalaReflection.typeBoxedJavaMapping.getOrElse(dt, classOf[java.lang.Object]) + } + +} + +/** + * Support for generating catalyst schemas for scala objects. Note that unlike its companion + * object, this trait able to work in both the runtime and the compile time (macro) universe. + */ +trait KotlinReflection extends Logging { + /** The universe we work in (runtime or macro) */ + val universe: scala.reflect.api.Universe + + /** The mirror used to access types in the universe */ + def mirror: universe.Mirror + + import universe._ + + // The Predef.Map is scala.collection.immutable.Map. + // Since the map values can be mutable, we explicitly import scala.collection.Map at here. + + /** + * Any codes calling `scala.reflect.api.Types.TypeApi.<:<` should be wrapped by this method to + * clean up the Scala reflection garbage automatically. Otherwise, it will leak some objects to + * `scala.reflect.runtime.JavaUniverse.undoLog`. + * + * @see https://github.com/scala/bug/issues/8302 + */ + def cleanUpReflectionObjects[T](func: => T): T = { + universe.asInstanceOf[scala.reflect.runtime.JavaUniverse].undoLog.undo(func) + } + + /** + * Return the Scala Type for `T` in the current classloader mirror. + * + * Use this method instead of the convenience method `universe.typeOf`, which + * assumes that all types can be found in the classloader that loaded scala-reflect classes. + * That's not necessarily the case when running using Eclipse launchers or even + * Sbt console or test (without `fork := true`). + * + * @see SPARK-5281 + */ + def localTypeOf[T: TypeTag]: `Type` = { + val tag = implicitly[TypeTag[T]] + tag.in(mirror).tpe.dealias + } + + /** + * Returns the full class name for a type. The returned name is the canonical + * Scala name, where each component is separated by a period. It is NOT the + * Java-equivalent runtime name (no dollar signs). + * + * In simple cases, both the Scala and Java names are the same, however when Scala + * generates constructs that do not map to a Java equivalent, such as singleton objects + * or nested classes in package objects, it uses the dollar sign ($) to create + * synthetic classes, emulating behaviour in Java bytecode. + */ + def getClassNameFromType(tpe: `Type`): String = { + tpe.dealias.erasure.typeSymbol.asClass.fullName + } + + /** + * Returns the parameter names and types for the primary constructor of this type. + * + * Note that it only works for scala classes with primary constructor, and currently doesn't + * support inner class. + */ + def getConstructorParameters(tpe: Type): Seq[(String, Type)] = { + val dealiasedTpe = tpe.dealias + val formalTypeArgs = dealiasedTpe.typeSymbol.asClass.typeParams + val TypeRef(_, _, actualTypeArgs) = dealiasedTpe + val params = constructParams(dealiasedTpe) + // if there are type variables to fill in, do the substitution (SomeClass[T] -> SomeClass[Int]) + if (actualTypeArgs.nonEmpty) { + params.map { p => + p.name.decodedName.toString -> + p.typeSignature.substituteTypes(formalTypeArgs, actualTypeArgs) + } + } else { + params.map { p => + p.name.decodedName.toString -> p.typeSignature + } + } + } + + /** + * If our type is a Scala trait it may have a companion object that + * only defines a constructor via `apply` method. + */ + private def getCompanionConstructor(tpe: Type): Symbol = { + def throwUnsupportedOperation = { + throw new UnsupportedOperationException(s"Unable to find constructor for $tpe. " + + s"This could happen if $tpe is an interface, or a trait without companion object " + + "constructor." + ) + } + + tpe.typeSymbol.asClass.companion match { + case NoSymbol => throwUnsupportedOperation + case sym => { + sym.asTerm.typeSignature.member(universe.TermName("apply")) match { + case NoSymbol => throwUnsupportedOperation + case constructorSym => constructorSym + } + } + } + } + + protected def constructParams(tpe: Type): Seq[Symbol] = { + val constructorSymbol = tpe.member(termNames.CONSTRUCTOR) match { + case NoSymbol => getCompanionConstructor(tpe) + case sym => sym + } + val params = if (constructorSymbol.isMethod) { + constructorSymbol.asMethod.paramLists + } else { + // Find the primary constructor, and use its parameter ordering. + val primaryConstructorSymbol: Option[Symbol] = constructorSymbol.asTerm.alternatives.find( + s => s.isMethod && s.asMethod.isPrimaryConstructor + ) + if (primaryConstructorSymbol.isEmpty) { + sys.error("Internal SQL error: Product object did not have a primary constructor.") + } else { + primaryConstructorSymbol.get.asMethod.paramLists + } + } + params.flatten + } + +} + diff --git a/core/3.0/src/main/scala/org/apache/spark/sql/KotlinWrappers.scala b/core/3.0/src/main/scala/org/apache/spark/sql/KotlinWrappers.scala new file mode 100644 index 00000000..f9b5a52f --- /dev/null +++ b/core/3.0/src/main/scala/org/apache/spark/sql/KotlinWrappers.scala @@ -0,0 +1,226 @@ +/*- + * =LICENSE= + * Kotlin Spark API: Examples + * ---------- + * Copyright (C) 2019 - 2020 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.apache.spark.sql + +import org.apache.spark.sql.catalyst.analysis.Resolver +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression} +import org.apache.spark.sql.catalyst.util.StringUtils +import org.apache.spark.sql.types.{DataType, Metadata, StructField, StructType} + + +trait DataTypeWithClass { + val dt: DataType + val cls: Class[ _ ] + val nullable: Boolean +} + +trait ComplexWrapper extends DataTypeWithClass + +class KDataTypeWrapper( + val dt: StructType, + val cls: Class[ _ ], + val nullable: Boolean = true, +) extends StructType with ComplexWrapper { + + override def fieldNames: Array[ String ] = dt.fieldNames + + override def names: Array[ String ] = dt.names + + override def equals(that: Any): Boolean = dt.equals(that) + + override def hashCode(): Int = dt.hashCode() + + override def add(field: StructField): StructType = dt.add(field) + + override def add(name: String, dataType: DataType): StructType = dt.add(name, dataType) + + override def add(name: String, dataType: DataType, nullable: Boolean): StructType = dt.add(name, dataType, nullable) + + override def add(name: String, dataType: DataType, nullable: Boolean, metadata: Metadata): StructType = dt + .add(name, dataType, nullable, metadata) + + override def add(name: String, dataType: DataType, nullable: Boolean, comment: String): StructType = dt + .add(name, dataType, nullable, comment) + + override def add(name: String, dataType: String): StructType = dt.add(name, dataType) + + override def add(name: String, dataType: String, nullable: Boolean): StructType = dt.add(name, dataType, nullable) + + override def add(name: String, dataType: String, nullable: Boolean, metadata: Metadata): StructType = dt + .add(name, dataType, nullable, metadata) + + override def add(name: String, dataType: String, nullable: Boolean, comment: String): StructType = dt + .add(name, dataType, nullable, comment) + + override def apply(name: String): StructField = dt.apply(name) + + override def apply(names: Set[ String ]): StructType = dt.apply(names) + + override def fieldIndex(name: String): Int = dt.fieldIndex(name) + + override private[ sql ] def getFieldIndex(name: String) = dt.getFieldIndex(name) + + override private[ sql ] def findNestedField(fieldNames: Seq[ String ], includeCollections: Boolean, resolver: Resolver) = + dt.findNestedField(fieldNames, includeCollections, resolver) + + override private[ sql ] def buildFormattedString(prefix: String, stringConcat: StringUtils.StringConcat, maxDepth: Int): Unit = + dt.buildFormattedString(prefix, stringConcat, maxDepth) + + override protected[ sql ] def toAttributes: Seq[ AttributeReference ] = dt.toAttributes + + override def treeString: String = dt.treeString + + override def treeString(maxDepth: Int): String = dt.treeString(maxDepth) + + override def printTreeString(): Unit = dt.printTreeString() + + private[ sql ] override def jsonValue = dt.jsonValue + + override def apply(fieldIndex: Int): StructField = dt.apply(fieldIndex) + + override def length: Int = dt.length + + override def iterator: Iterator[ StructField ] = dt.iterator + + override def defaultSize: Int = dt.defaultSize + + override def simpleString: String = dt.simpleString + + override def catalogString: String = dt.catalogString + + override def sql: String = dt.sql + + override def toDDL: String = dt.toDDL + + private[ sql ] override def simpleString(maxNumberFields: Int) = dt.simpleString(maxNumberFields) + + override private[ sql ] def merge(that: StructType) = dt.merge(that) + + private[ spark ] override def asNullable = dt.asNullable + + private[ spark ] override def existsRecursively(f: DataType => Boolean) = dt.existsRecursively(f) + + override private[ sql ] lazy val interpretedOrdering = dt.interpretedOrdering + + override def toString = s"KDataTypeWrapper(dt=$dt, cls=$cls, nullable=$nullable)" +} + +case class KComplexTypeWrapper(dt: DataType, cls: Class[ _ ], nullable: Boolean) extends DataType with ComplexWrapper { + + override private[ sql ] def unapply(e: Expression) = dt.unapply(e) + + override def typeName: String = dt.typeName + + override private[ sql ] def jsonValue = dt.jsonValue + + override def json: String = dt.json + + override def prettyJson: String = dt.prettyJson + + override def simpleString: String = dt.simpleString + + override def catalogString: String = dt.catalogString + + override private[ sql ] def simpleString(maxNumberFields: Int) = dt.simpleString(maxNumberFields) + + override def sql: String = dt.sql + + override private[ spark ] def sameType(other: DataType) = dt.sameType(other) + + override private[ spark ] def existsRecursively(f: DataType => Boolean) = dt.existsRecursively(f) + + private[ sql ] override def defaultConcreteType = dt.defaultConcreteType + + private[ sql ] override def acceptsType(other: DataType) = dt.acceptsType(other) + + override def defaultSize: Int = dt.defaultSize + + override private[ spark ] def asNullable = dt.asNullable + +} + +case class KSimpleTypeWrapper(dt: DataType, cls: Class[ _ ], nullable: Boolean) extends DataType with DataTypeWithClass { + override private[ sql ] def unapply(e: Expression) = dt.unapply(e) + + override def typeName: String = dt.typeName + + override private[ sql ] def jsonValue = dt.jsonValue + + override def json: String = dt.json + + override def prettyJson: String = dt.prettyJson + + override def simpleString: String = dt.simpleString + + override def catalogString: String = dt.catalogString + + override private[ sql ] def simpleString(maxNumberFields: Int) = dt.simpleString(maxNumberFields) + + override def sql: String = dt.sql + + override private[ spark ] def sameType(other: DataType) = dt.sameType(other) + + override private[ spark ] def existsRecursively(f: DataType => Boolean) = dt.existsRecursively(f) + + private[ sql ] override def defaultConcreteType = dt.defaultConcreteType + + private[ sql ] override def acceptsType(other: DataType) = dt.acceptsType(other) + + override def defaultSize: Int = dt.defaultSize + + override private[ spark ] def asNullable = dt.asNullable +} + +class KStructField(val getterName: String, val delegate: StructField) extends StructField { + + override private[ sql ] def buildFormattedString(prefix: String, stringConcat: StringUtils.StringConcat, maxDepth: Int): Unit = + delegate.buildFormattedString(prefix, stringConcat, maxDepth) + + override def toString(): String = delegate.toString() + + override private[ sql ] def jsonValue = delegate.jsonValue + + override def withComment(comment: String): StructField = delegate.withComment(comment) + + override def getComment(): Option[ String ] = delegate.getComment() + + override def toDDL: String = delegate.toDDL + + override def productElement(n: Int): Any = delegate.productElement(n) + + override def productArity: Int = delegate.productArity + + override def productIterator: Iterator[ Any ] = delegate.productIterator + + override def productPrefix: String = delegate.productPrefix + + override val dataType: DataType = delegate.dataType + + override def canEqual(that: Any): Boolean = delegate.canEqual(that) + + override val metadata: Metadata = delegate.metadata + override val name: String = delegate.name + override val nullable: Boolean = delegate.nullable +} + +object helpme { + + def listToSeq(i: java.util.List[ _ ]): Seq[ _ ] = Seq(i.toArray: _*) +} \ No newline at end of file diff --git a/core/3.2/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala b/core/3.0/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala similarity index 100% rename from core/3.2/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala rename to core/3.0/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala diff --git a/core/3.0/src/main/scala/org/jetbrains/kotlinx/spark/extensions/DemoCaseClass.scala b/core/3.0/src/main/scala/org/jetbrains/kotlinx/spark/extensions/DemoCaseClass.scala new file mode 100644 index 00000000..eb5a1a47 --- /dev/null +++ b/core/3.0/src/main/scala/org/jetbrains/kotlinx/spark/extensions/DemoCaseClass.scala @@ -0,0 +1,3 @@ +package org.jetbrains.kotlinx.spark.extensions + +case class DemoCaseClass[T](a: Int, b: T) diff --git a/core/3.2/src/main/scala/org/jetbrains/kotlinx/spark/extensions/KSparkExtensions.scala b/core/3.0/src/main/scala/org/jetbrains/kotlinx/spark/extensions/KSparkExtensions.scala similarity index 100% rename from core/3.2/src/main/scala/org/jetbrains/kotlinx/spark/extensions/KSparkExtensions.scala rename to core/3.0/src/main/scala/org/jetbrains/kotlinx/spark/extensions/KSparkExtensions.scala diff --git a/core/3.2/pom_2.12.xml b/core/3.2/pom_2.12.xml deleted file mode 100644 index 8cbd069b..00000000 --- a/core/3.2/pom_2.12.xml +++ /dev/null @@ -1,75 +0,0 @@ - - - 4.0.0 - - Kotlin Spark API: Scala core for Spark 3.2+ (Scala 2.12) - Scala-Spark 3.2+ compatibility layer for Kotlin for Apache Spark - core-3.2_2.12 - - org.jetbrains.kotlinx.spark - kotlin-spark-api-parent_2.12 - 1.0.3 - ../../pom_2.12.xml - - - - - org.scala-lang - scala-library - ${scala.version} - - - org.jetbrains.kotlin - kotlin-reflect - - - - org.apache.spark - spark-sql_${scala.compat.version} - ${spark3.version} - provided - - - - - src/main/scala - src/test/scala - target/${scala.compat.version} - - - net.alchim31.maven - scala-maven-plugin - ${scala-maven-plugin.version} - - - compile - - compile - testCompile - - - - -dependencyfile - ${project.build.directory}/.scala_dependencies - - - - - docjar - - doc-jar - - pre-integration-test - - - - - org.apache.maven.plugins - maven-site-plugin - - true - - - - - diff --git a/core/3.2/src/main/scala/org/apache/spark/sql/KotlinReflection.scala b/core/3.2/src/main/scala/org/apache/spark/sql/KotlinReflection.scala deleted file mode 100644 index be808af0..00000000 --- a/core/3.2/src/main/scala/org/apache/spark/sql/KotlinReflection.scala +++ /dev/null @@ -1,1110 +0,0 @@ -/*- - * =LICENSE= - * Kotlin Spark API: Examples - * ---------- - * Copyright (C) 2019 - 2020 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.apache.spark.sql - -import org.apache.spark.internal.Logging -import org.apache.spark.sql.catalyst.DeserializerBuildHelper._ -import org.apache.spark.sql.catalyst.SerializerBuildHelper._ -import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal -import org.apache.spark.sql.catalyst.expressions.objects._ -import org.apache.spark.sql.catalyst.expressions.{Expression, _} -import org.apache.spark.sql.catalyst.util.ArrayBasedMapData -import org.apache.spark.sql.catalyst.{InternalRow, ScalaReflection, WalkedTypePath} -import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} - -import java.beans.{Introspector, PropertyDescriptor} - - -/** - * A helper trait to create [[org.apache.spark.sql.catalyst.encoders.ExpressionEncoder]]s - * for classes whose fields are entirely defined by constructor params but should not be - * case classes. - */ -trait DefinedByConstructorParams - -/** - * KotlinReflection is heavily inspired by ScalaReflection and even extends it just to add several methods - */ -object KotlinReflection extends KotlinReflection { - /** - * Returns the Spark SQL DataType for a given java class. Where this is not an exact mapping - * to a native type, an ObjectType is returned. - * - * Unlike `inferDataType`, this function doesn't do any massaging of types into the Spark SQL type - * system. As a result, ObjectType will be returned for things like boxed Integers. - */ - private def inferExternalType(cls: Class[_]): DataType = cls match { - case c if c == java.lang.Boolean.TYPE => BooleanType - case c if c == java.lang.Byte.TYPE => ByteType - case c if c == java.lang.Short.TYPE => ShortType - case c if c == java.lang.Integer.TYPE => IntegerType - case c if c == java.lang.Long.TYPE => LongType - case c if c == java.lang.Float.TYPE => FloatType - case c if c == java.lang.Double.TYPE => DoubleType - case c if c == classOf[Array[Byte]] => BinaryType - case _ => ObjectType(cls) - } - - val universe: scala.reflect.runtime.universe.type = scala.reflect.runtime.universe - - // Since we are creating a runtime mirror using the class loader of current thread, - // we need to use def at here. So, every time we call mirror, it is using the - // class loader of the current thread. - override def mirror: universe.Mirror = { - universe.runtimeMirror(Thread.currentThread().getContextClassLoader) - } - - import universe._ - - // The Predef.Map is scala.collection.immutable.Map. - // Since the map values can be mutable, we explicitly import scala.collection.Map at here. - import scala.collection.Map - - - def isSubtype(t: universe.Type, t2: universe.Type): Boolean = t <:< t2 - - /** - * Synchronize to prevent concurrent usage of `<:<` operator. - * This operator is not thread safe in any current version of scala; i.e. - * (2.11.12, 2.12.10, 2.13.0-M5). - * - * See https://github.com/scala/bug/issues/10766 - */ - /* - private[catalyst] def isSubtype(tpe1: `Type`, tpe2: `Type`): Boolean = { - ScalaReflection.ScalaSubtypeLock.synchronized { - tpe1 <:< tpe2 - } - } - */ - - private def dataTypeFor(tpe: `Type`): DataType = cleanUpReflectionObjects { - tpe.dealias match { - case t if isSubtype(t, definitions.NullTpe) => NullType - case t if isSubtype(t, definitions.IntTpe) => IntegerType - case t if isSubtype(t, definitions.LongTpe) => LongType - case t if isSubtype(t, definitions.DoubleTpe) => DoubleType - case t if isSubtype(t, definitions.FloatTpe) => FloatType - case t if isSubtype(t, definitions.ShortTpe) => ShortType - case t if isSubtype(t, definitions.ByteTpe) => ByteType - case t if isSubtype(t, definitions.BooleanTpe) => BooleanType - case t if isSubtype(t, localTypeOf[Array[Byte]]) => BinaryType - case t if isSubtype(t, localTypeOf[CalendarInterval]) => CalendarIntervalType - case t if isSubtype(t, localTypeOf[Decimal]) => DecimalType.SYSTEM_DEFAULT - case _ => - val className = getClassNameFromType(tpe) - className match { - case "scala.Array" => - val TypeRef(_, _, Seq(elementType)) = tpe - arrayClassFor(elementType) - case _ => - val clazz = getClassFromType(tpe) - ObjectType(clazz) - } - } - } - - /** - * Given a type `T` this function constructs `ObjectType` that holds a class of type - * `Array[T]`. - * - * Special handling is performed for primitive types to map them back to their raw - * JVM form instead of the Scala Array that handles auto boxing. - */ - private def arrayClassFor(tpe: `Type`): ObjectType = cleanUpReflectionObjects { - val cls = tpe.dealias match { - case t if isSubtype(t, definitions.IntTpe) => classOf[Array[Int]] - case t if isSubtype(t, definitions.LongTpe) => classOf[Array[Long]] - case t if isSubtype(t, definitions.DoubleTpe) => classOf[Array[Double]] - case t if isSubtype(t, definitions.FloatTpe) => classOf[Array[Float]] - case t if isSubtype(t, definitions.ShortTpe) => classOf[Array[Short]] - case t if isSubtype(t, definitions.ByteTpe) => classOf[Array[Byte]] - case t if isSubtype(t, definitions.BooleanTpe) => classOf[Array[Boolean]] - case other => - // There is probably a better way to do this, but I couldn't find it... - val elementType = dataTypeFor(other).asInstanceOf[ObjectType].cls - java.lang.reflect.Array.newInstance(elementType, 0).getClass - - } - ObjectType(cls) - } - - /** - * Returns true if the value of this data type is same between internal and external. - */ - def isNativeType(dt: DataType): Boolean = dt match { - case NullType | BooleanType | ByteType | ShortType | IntegerType | LongType | - FloatType | DoubleType | BinaryType | CalendarIntervalType => true - case _ => false - } - - private def baseType(tpe: `Type`): `Type` = { - tpe.dealias match { - case annotatedType: AnnotatedType => annotatedType.underlying - case other => other - } - } - - /** - * Returns an expression that can be used to deserialize a Spark SQL representation to an object - * of type `T` with a compatible schema. The Spark SQL representation is located at ordinal 0 of - * a row, i.e., `GetColumnByOrdinal(0, _)`. Nested classes will have their fields accessed using - * `UnresolvedExtractValue`. - * - * The returned expression is used by `ExpressionEncoder`. The encoder will resolve and bind this - * deserializer expression when using it. - */ - def deserializerForType(tpe: `Type`): Expression = { - val clsName = getClassNameFromType(tpe) - val walkedTypePath = WalkedTypePath().recordRoot(clsName) - val Schema(dataType, nullable) = schemaFor(tpe) - - // Assumes we are deserializing the first column of a row. - deserializerForWithNullSafetyAndUpcast(GetColumnByOrdinal(0, dataType), dataType, - nullable = nullable, walkedTypePath, - (casted, typePath) => deserializerFor(tpe, casted, typePath)) - } - - - /** - * Returns an expression that can be used to deserialize an input expression to an object of type - * `T` with a compatible schema. - * - * @param tpe The `Type` of deserialized object. - * @param path The expression which can be used to extract serialized value. - * @param walkedTypePath The paths from top to bottom to access current field when deserializing. - */ - private def deserializerFor( - tpe: `Type`, - path: Expression, - walkedTypePath: WalkedTypePath, - predefinedDt: Option[DataTypeWithClass] = None - ): Expression = cleanUpReflectionObjects { - baseType(tpe) match { - - // - case t if isSubtype(t, localTypeOf[java.lang.Integer]) => - createDeserializerForTypesSupportValueOf(path, - classOf[java.lang.Integer]) - - case t if isSubtype(t, localTypeOf[Int]) => - createDeserializerForTypesSupportValueOf(path, - classOf[java.lang.Integer]) - - case t if isSubtype(t, localTypeOf[java.lang.Long]) => - createDeserializerForTypesSupportValueOf(path, - classOf[java.lang.Long]) - case t if isSubtype(t, localTypeOf[Long]) => - createDeserializerForTypesSupportValueOf(path, - classOf[java.lang.Long]) - - case t if isSubtype(t, localTypeOf[java.lang.Double]) => - createDeserializerForTypesSupportValueOf(path, - classOf[java.lang.Double]) - case t if isSubtype(t, localTypeOf[Double]) => - createDeserializerForTypesSupportValueOf(path, - classOf[java.lang.Double]) - - case t if isSubtype(t, localTypeOf[java.lang.Float]) => - createDeserializerForTypesSupportValueOf(path, - classOf[java.lang.Float]) - case t if isSubtype(t, localTypeOf[Float]) => - createDeserializerForTypesSupportValueOf(path, - classOf[java.lang.Float]) - - case t if isSubtype(t, localTypeOf[java.lang.Short]) => - createDeserializerForTypesSupportValueOf(path, - classOf[java.lang.Short]) - case t if isSubtype(t, localTypeOf[Short]) => - createDeserializerForTypesSupportValueOf(path, - classOf[java.lang.Short]) - - case t if isSubtype(t, localTypeOf[java.lang.Byte]) => - createDeserializerForTypesSupportValueOf(path, - classOf[java.lang.Byte]) - case t if isSubtype(t, localTypeOf[Byte]) => - createDeserializerForTypesSupportValueOf(path, - classOf[java.lang.Byte]) - - case t if isSubtype(t, localTypeOf[java.lang.Boolean]) => - createDeserializerForTypesSupportValueOf(path, - classOf[java.lang.Boolean]) - case t if isSubtype(t, localTypeOf[Boolean]) => - createDeserializerForTypesSupportValueOf(path, - classOf[java.lang.Boolean]) - - case t if isSubtype(t, localTypeOf[java.time.LocalDate]) => - createDeserializerForLocalDate(path) - - case t if isSubtype(t, localTypeOf[java.sql.Date]) => - createDeserializerForSqlDate(path) - // - - case t if isSubtype(t, localTypeOf[java.time.Instant]) => - createDeserializerForInstant(path) - - case t if isSubtype(t, localTypeOf[java.sql.Timestamp]) => - createDeserializerForSqlTimestamp(path) - - case t if isSubtype(t, localTypeOf[java.lang.String]) => - createDeserializerForString(path, returnNullable = false) - - case t if isSubtype(t, localTypeOf[java.math.BigDecimal]) => - createDeserializerForJavaBigDecimal(path, returnNullable = false) - - case t if isSubtype(t, localTypeOf[BigDecimal]) => - createDeserializerForScalaBigDecimal(path, returnNullable = false) - - case t if isSubtype(t, localTypeOf[java.math.BigInteger]) => - createDeserializerForJavaBigInteger(path, returnNullable = false) - - case t if isSubtype(t, localTypeOf[scala.math.BigInt]) => - createDeserializerForScalaBigInt(path) - - case t if isSubtype(t, localTypeOf[Array[_]]) => - var TypeRef(_, _, Seq(elementType)) = t - if (predefinedDt.isDefined && !elementType.dealias.typeSymbol.isClass) - elementType = getType(predefinedDt.get.asInstanceOf[KComplexTypeWrapper].dt.asInstanceOf[ArrayType].elementType.asInstanceOf[DataTypeWithClass].cls) - val Schema(dataType, elementNullable) = predefinedDt.map(it => { - val elementInfo = it.asInstanceOf[KComplexTypeWrapper].dt.asInstanceOf[ArrayType].elementType.asInstanceOf[DataTypeWithClass] - Schema(elementInfo.dt, elementInfo.nullable) - }) - .getOrElse(schemaFor(elementType)) - val className = getClassNameFromType(elementType) - val newTypePath = walkedTypePath.recordArray(className) - - val mapFunction: Expression => Expression = element => { - // upcast the array element to the data type the encoder expected. - deserializerForWithNullSafetyAndUpcast( - element, - dataType, - nullable = elementNullable, - newTypePath, - (casted, typePath) => deserializerFor(elementType, casted, typePath, predefinedDt.map(_.asInstanceOf[KComplexTypeWrapper].dt.asInstanceOf[ArrayType].elementType).filter(_.isInstanceOf[ComplexWrapper]).map(_.asInstanceOf[ComplexWrapper]))) - } - - val arrayData = UnresolvedMapObjects(mapFunction, path) - val arrayCls = arrayClassFor(elementType) - - val methodName = elementType match { - case t if isSubtype(t, definitions.IntTpe) => "toIntArray" - case t if isSubtype(t, definitions.LongTpe) => "toLongArray" - case t if isSubtype(t, definitions.DoubleTpe) => "toDoubleArray" - case t if isSubtype(t, definitions.FloatTpe) => "toFloatArray" - case t if isSubtype(t, definitions.ShortTpe) => "toShortArray" - case t if isSubtype(t, definitions.ByteTpe) => "toByteArray" - case t if isSubtype(t, definitions.BooleanTpe) => "toBooleanArray" - // non-primitive - case _ => "array" - } - Invoke(arrayData, methodName, arrayCls, returnNullable = false) - - // We serialize a `Set` to Catalyst array. When we deserialize a Catalyst array - // to a `Set`, if there are duplicated elements, the elements will be de-duplicated. - - case t if isSubtype(t, localTypeOf[Map[_, _]]) => - val TypeRef(_, _, Seq(keyType, valueType)) = t - - val classNameForKey = getClassNameFromType(keyType) - val classNameForValue = getClassNameFromType(valueType) - - val newTypePath = walkedTypePath.recordMap(classNameForKey, classNameForValue) - - UnresolvedCatalystToExternalMap( - path, - p => deserializerFor(keyType, p, newTypePath), - p => deserializerFor(valueType, p, newTypePath), - mirror.runtimeClass(t.typeSymbol.asClass) - ) - - case t if isSubtype(t, localTypeOf[java.lang.Enum[_]]) => - createDeserializerForTypesSupportValueOf( - createDeserializerForString(path, returnNullable = false), Class.forName(t.toString)) - - case t if t.typeSymbol.annotations.exists(_.tree.tpe =:= typeOf[SQLUserDefinedType]) => - val udt = getClassFromType(t).getAnnotation(classOf[SQLUserDefinedType]).udt(). - getConstructor().newInstance() - val obj = NewInstance( - udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt(), - Nil, - dataType = ObjectType(udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt())) - Invoke(obj, "deserialize", ObjectType(udt.userClass), path :: Nil) - - case t if UDTRegistration.exists(getClassNameFromType(t)) => - val udt = UDTRegistration.getUDTFor(getClassNameFromType(t)).get.getConstructor(). - newInstance().asInstanceOf[UserDefinedType[_]] - val obj = NewInstance( - udt.getClass, - Nil, - dataType = ObjectType(udt.getClass)) - Invoke(obj, "deserialize", ObjectType(udt.userClass), path :: Nil) - - case _ if predefinedDt.isDefined => - predefinedDt.get match { - case wrapper: KDataTypeWrapper => - val structType = wrapper.dt - val cls = wrapper.cls - val arguments = structType - .fields - .map(field => { - val dataType = field.dataType.asInstanceOf[DataTypeWithClass] - val nullable = dataType.nullable - val clsName = getClassNameFromType(getType(dataType.cls)) - val newTypePath = walkedTypePath.recordField(clsName, field.name) - - // For tuples, we based grab the inner fields by ordinal instead of name. - val newPath = deserializerFor( - getType(dataType.cls), - addToPath(path, field.name, dataType.dt, newTypePath), - newTypePath, - Some(dataType).filter(_.isInstanceOf[ComplexWrapper]) - ) - expressionWithNullSafety( - newPath, - nullable = nullable, - newTypePath - ) - - }) - val newInstance = NewInstance(cls, arguments, ObjectType(cls), propagateNull = false) - - org.apache.spark.sql.catalyst.expressions.If( - IsNull(path), - org.apache.spark.sql.catalyst.expressions.Literal.create(null, ObjectType(cls)), - newInstance - ) - - case t: ComplexWrapper => - t.dt match { - case MapType(kt, vt, _) => - val Seq(keyType, valueType) = Seq(kt, vt).map(_.asInstanceOf[DataTypeWithClass].cls).map(getType(_)) - val Seq(keyDT, valueDT) = Seq(kt, vt).map(_.asInstanceOf[DataTypeWithClass]) - val classNameForKey = getClassNameFromType(keyType) - val classNameForValue = getClassNameFromType(valueType) - - val newTypePath = walkedTypePath.recordMap(classNameForKey, classNameForValue) - - val keyData = - Invoke( - UnresolvedMapObjects( - p => deserializerFor(keyType, p, newTypePath, Some(keyDT).filter(_.isInstanceOf[ComplexWrapper])), - MapKeys(path)), - "array", - ObjectType(classOf[Array[Any]])) - - val valueData = - Invoke( - UnresolvedMapObjects( - p => deserializerFor(valueType, p, newTypePath, Some(valueDT).filter(_.isInstanceOf[ComplexWrapper])), - MapValues(path)), - "array", - ObjectType(classOf[Array[Any]])) - - StaticInvoke( - ArrayBasedMapData.getClass, - ObjectType(classOf[java.util.Map[_, _]]), - "toJavaMap", - keyData :: valueData :: Nil, - returnNullable = false) - - case ArrayType(elementType, containsNull) => - val dataTypeWithClass = elementType.asInstanceOf[DataTypeWithClass] - val mapFunction: Expression => Expression = element => { - // upcast the array element to the data type the encoder expected. - val et = getType(dataTypeWithClass.cls) - val className = getClassNameFromType(et) - val newTypePath = walkedTypePath.recordArray(className) - deserializerForWithNullSafetyAndUpcast( - element, - dataTypeWithClass.dt, - nullable = dataTypeWithClass.nullable, - newTypePath, - (casted, typePath) => { - deserializerFor(et, casted, typePath, Some(dataTypeWithClass).filter(_.isInstanceOf[ComplexWrapper]).map(_.asInstanceOf[ComplexWrapper])) - }) - } - - UnresolvedMapObjects(mapFunction, path, customCollectionCls = Some(t.cls)) - - case StructType(elementType: Array[StructField]) => - val cls = t.cls - - val arguments = elementType.map { field => - val dataType = field.dataType.asInstanceOf[DataTypeWithClass] - val nullable = dataType.nullable - val clsName = getClassNameFromType(getType(dataType.cls)) - val newTypePath = walkedTypePath.recordField(clsName, field.name) - - // For tuples, we based grab the inner fields by ordinal instead of name. - val newPath = deserializerFor( - getType(dataType.cls), - addToPath(path, field.name, dataType.dt, newTypePath), - newTypePath, - Some(dataType).filter(_.isInstanceOf[ComplexWrapper]) - ) - expressionWithNullSafety( - newPath, - nullable = nullable, - newTypePath - ) - } - val newInstance = NewInstance(cls, arguments, ObjectType(cls), propagateNull = false) - - org.apache.spark.sql.catalyst.expressions.If( - IsNull(path), - org.apache.spark.sql.catalyst.expressions.Literal.create(null, ObjectType(cls)), - newInstance - ) - - - case _ => - throw new UnsupportedOperationException( - s"No Encoder found for $tpe\n" + walkedTypePath) - } - } - - case t if definedByConstructorParams(t) => - val params = getConstructorParameters(t) - - val cls = getClassFromType(tpe) - - val arguments = params.zipWithIndex.map { case ((fieldName, fieldType), i) => - val Schema(dataType, nullable) = schemaFor(fieldType) - val clsName = getClassNameFromType(fieldType) - val newTypePath = walkedTypePath.recordField(clsName, fieldName) - - // For tuples, we based grab the inner fields by ordinal instead of name. - val newPath = if (cls.getName startsWith "scala.Tuple") { - deserializerFor( - fieldType, - addToPathOrdinal(path, i, dataType, newTypePath), - newTypePath) - } else { - deserializerFor( - fieldType, - addToPath(path, fieldName, dataType, newTypePath), - newTypePath) - } - expressionWithNullSafety( - newPath, - nullable = nullable, - newTypePath) - } - - val newInstance = NewInstance(cls, arguments, ObjectType(cls), propagateNull = false) - - org.apache.spark.sql.catalyst.expressions.If( - IsNull(path), - org.apache.spark.sql.catalyst.expressions.Literal.create(null, ObjectType(cls)), - newInstance - ) - - case _ => - throw new UnsupportedOperationException( - s"No Encoder found for $tpe\n" + walkedTypePath) - } - } - - /** - * Returns an expression for serializing an object of type T to Spark SQL representation. The - * input object is located at ordinal 0 of a row, i.e., `BoundReference(0, _)`. - * - * If the given type is not supported, i.e. there is no encoder can be built for this type, - * an [[UnsupportedOperationException]] will be thrown with detailed error message to explain - * the type path walked so far and which class we are not supporting. - * There are 4 kinds of type path: - * * the root type: `root class: "abc.xyz.MyClass"` - * * the value type of [[Option]]: `option value class: "abc.xyz.MyClass"` - * * the element type of [[Array]] or [[Seq]]: `array element class: "abc.xyz.MyClass"` - * * the field of [[Product]]: `field (class: "abc.xyz.MyClass", name: "myField")` - */ - def serializerForType(tpe: `Type`): Expression = ScalaReflection.cleanUpReflectionObjects { - val clsName = getClassNameFromType(tpe) - val walkedTypePath = WalkedTypePath().recordRoot(clsName) - - // The input object to `ExpressionEncoder` is located at first column of an row. - val isPrimitive = tpe.typeSymbol.asClass.isPrimitive - val inputObject = BoundReference(0, dataTypeFor(tpe), nullable = !isPrimitive) - - serializerFor(inputObject, tpe, walkedTypePath) - } - - def getType[T](clazz: Class[T]): universe.Type = { - val mir = runtimeMirror(clazz.getClassLoader) - mir.classSymbol(clazz).toType - } - - def deserializerFor(cls: java.lang.Class[_], dt: DataTypeWithClass): Expression = { - val tpe = getType(cls) - val clsName = getClassNameFromType(tpe) - val walkedTypePath = WalkedTypePath().recordRoot(clsName) - - // Assumes we are deserializing the first column of a row. - deserializerForWithNullSafetyAndUpcast( - GetColumnByOrdinal(0, dt.dt), - dt.dt, - nullable = dt.nullable, - walkedTypePath, - (casted, typePath) => deserializerFor(tpe, casted, typePath, Some(dt)) - ) - } - - - def serializerFor(cls: java.lang.Class[_], dt: DataTypeWithClass): Expression = { - - val tpe = getType(cls) - val clsName = getClassNameFromType(tpe) - val walkedTypePath = WalkedTypePath().recordRoot(clsName) - val inputObject = BoundReference(0, ObjectType(cls), nullable = true) - serializerFor(inputObject, tpe, walkedTypePath, predefinedDt = Some(dt)) - } - - /** - * Returns an expression for serializing the value of an input expression into Spark SQL - * internal representation. - */ - private def serializerFor( - inputObject: Expression, - tpe: `Type`, - walkedTypePath: WalkedTypePath, - seenTypeSet: Set[`Type`] = Set.empty, - predefinedDt: Option[DataTypeWithClass] = None - ): Expression = cleanUpReflectionObjects { - - def toCatalystArray(input: Expression, elementType: `Type`, predefinedDt: Option[DataTypeWithClass] = None): Expression = { - predefinedDt.map(_.dt).getOrElse(dataTypeFor(elementType)) match { - - case dt@(MapType(_, _, _) | ArrayType(_, _) | StructType(_)) => - val clsName = getClassNameFromType(elementType) - val newPath = walkedTypePath.recordArray(clsName) - createSerializerForMapObjects(input, ObjectType(predefinedDt.get.cls), - serializerFor(_, elementType, newPath, seenTypeSet, predefinedDt)) - - case dt: ObjectType => - val clsName = getClassNameFromType(elementType) - val newPath = walkedTypePath.recordArray(clsName) - createSerializerForMapObjects(input, dt, - serializerFor(_, elementType, newPath, seenTypeSet)) - - case dt@(BooleanType | ByteType | ShortType | IntegerType | LongType | - FloatType | DoubleType) => - val cls = input.dataType.asInstanceOf[ObjectType].cls - if (cls.isArray && cls.getComponentType.isPrimitive) { - createSerializerForPrimitiveArray(input, dt) - } else { - createSerializerForGenericArray(input, dt, nullable = predefinedDt.map(_.nullable).getOrElse(schemaFor(elementType).nullable)) - } - - case _: StringType => - val clsName = getClassNameFromType(typeOf[String]) - val newPath = walkedTypePath.recordArray(clsName) - createSerializerForMapObjects(input, ObjectType(Class.forName(getClassNameFromType(elementType))), - serializerFor(_, elementType, newPath, seenTypeSet)) - - - case dt => - createSerializerForGenericArray(input, dt, nullable = predefinedDt.map(_.nullable).getOrElse(schemaFor(elementType).nullable)) - } - } - - baseType(tpe) match { - - // - case _ if !inputObject.dataType.isInstanceOf[ObjectType] && !predefinedDt.exists(_.isInstanceOf[ComplexWrapper]) => inputObject - - case t if isSubtype(t, localTypeOf[Option[_]]) => - val TypeRef(_, _, Seq(optType)) = t - val className = getClassNameFromType(optType) - val newPath = walkedTypePath.recordOption(className) - val unwrapped = UnwrapOption(dataTypeFor(optType), inputObject) - serializerFor(unwrapped, optType, newPath, seenTypeSet) - - // Since List[_] also belongs to localTypeOf[Product], we put this case before - // "case t if definedByConstructorParams(t)" to make sure it will match to the - // case "localTypeOf[Seq[_]]" - case t if isSubtype(t, localTypeOf[Seq[_]]) => - val TypeRef(_, _, Seq(elementType)) = t - toCatalystArray(inputObject, elementType) - - case t if isSubtype(t, localTypeOf[Array[_]]) && predefinedDt.isEmpty => - val TypeRef(_, _, Seq(elementType)) = t - toCatalystArray(inputObject, elementType) - - case t if isSubtype(t, localTypeOf[Map[_, _]]) => - val TypeRef(_, _, Seq(keyType, valueType)) = t - val keyClsName = getClassNameFromType(keyType) - val valueClsName = getClassNameFromType(valueType) - val keyPath = walkedTypePath.recordKeyForMap(keyClsName) - val valuePath = walkedTypePath.recordValueForMap(valueClsName) - - createSerializerForMap( - inputObject, - MapElementInformation( - dataTypeFor(keyType), - nullable = !keyType.typeSymbol.asClass.isPrimitive, - serializerFor(_, keyType, keyPath, seenTypeSet)), - MapElementInformation( - dataTypeFor(valueType), - nullable = !valueType.typeSymbol.asClass.isPrimitive, - serializerFor(_, valueType, valuePath, seenTypeSet)) - ) - - case t if isSubtype(t, localTypeOf[scala.collection.Set[_]]) => - val TypeRef(_, _, Seq(elementType)) = t - - // There's no corresponding Catalyst type for `Set`, we serialize a `Set` to Catalyst array. - // Note that the property of `Set` is only kept when manipulating the data as domain object. - val newInput = - Invoke( - inputObject, - "toSeq", - ObjectType(classOf[Seq[_]])) - - toCatalystArray(newInput, elementType) - - case t if isSubtype(t, localTypeOf[String]) => - createSerializerForString(inputObject) - case t if isSubtype(t, localTypeOf[java.time.Instant]) => - createSerializerForJavaInstant(inputObject) - - case t if isSubtype(t, localTypeOf[java.sql.Timestamp]) => - createSerializerForSqlTimestamp(inputObject) - - case t if isSubtype(t, localTypeOf[java.time.LocalDate]) => - createSerializerForJavaLocalDate(inputObject) - - case t if isSubtype(t, localTypeOf[java.sql.Date]) => createSerializerForSqlDate(inputObject) - - case t if isSubtype(t, localTypeOf[BigDecimal]) => - createSerializerForScalaBigDecimal(inputObject) - - case t if isSubtype(t, localTypeOf[java.math.BigDecimal]) => - createSerializerForJavaBigDecimal(inputObject) - - case t if isSubtype(t, localTypeOf[java.math.BigInteger]) => - createSerializerForJavaBigInteger(inputObject) - - case t if isSubtype(t, localTypeOf[scala.math.BigInt]) => - createSerializerForScalaBigInt(inputObject) - - case t if isSubtype(t, localTypeOf[java.lang.Integer]) => - createSerializerForInteger(inputObject) - case t if isSubtype(t, localTypeOf[Int]) => - createSerializerForInteger(inputObject) - case t if isSubtype(t, localTypeOf[java.lang.Long]) => createSerializerForLong(inputObject) - case t if isSubtype(t, localTypeOf[Long]) => createSerializerForLong(inputObject) - case t if isSubtype(t, localTypeOf[java.lang.Double]) => createSerializerForDouble(inputObject) - case t if isSubtype(t, localTypeOf[Double]) => createSerializerForDouble(inputObject) - case t if isSubtype(t, localTypeOf[java.lang.Float]) => createSerializerForFloat(inputObject) - case t if isSubtype(t, localTypeOf[Float]) => createSerializerForFloat(inputObject) - case t if isSubtype(t, localTypeOf[java.lang.Short]) => createSerializerForShort(inputObject) - case t if isSubtype(t, localTypeOf[Short]) => createSerializerForShort(inputObject) - case t if isSubtype(t, localTypeOf[java.lang.Byte]) => createSerializerForByte(inputObject) - case t if isSubtype(t, localTypeOf[Byte]) => createSerializerForByte(inputObject) - case t if isSubtype(t, localTypeOf[java.lang.Boolean]) => createSerializerForBoolean(inputObject) - case t if isSubtype(t, localTypeOf[Boolean]) => createSerializerForBoolean(inputObject) - - case t if isSubtype(t, localTypeOf[java.lang.Enum[_]]) => - createSerializerForString( - Invoke(inputObject, "name", ObjectType(classOf[String]), returnNullable = false)) - - case t if t.typeSymbol.annotations.exists(_.tree.tpe =:= typeOf[SQLUserDefinedType]) => - val udt = getClassFromType(t) - .getAnnotation(classOf[SQLUserDefinedType]).udt().getConstructor().newInstance() - val udtClass = udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt() - createSerializerForUserDefinedType(inputObject, udt, udtClass) - - case t if UDTRegistration.exists(getClassNameFromType(t)) => - val udt = UDTRegistration.getUDTFor(getClassNameFromType(t)).get.getConstructor(). - newInstance().asInstanceOf[UserDefinedType[_]] - val udtClass = udt.getClass - createSerializerForUserDefinedType(inputObject, udt, udtClass) - // - - case _ if predefinedDt.isDefined => - predefinedDt.get match { - case dataType: KDataTypeWrapper => - val cls = dataType.cls - val properties = getJavaBeanReadableProperties(cls) - val structFields = dataType.dt.fields.map(_.asInstanceOf[KStructField]) - val fields = structFields.map { structField => - val maybeProp = properties.find(it => it.getReadMethod.getName == structField.getterName) - if (maybeProp.isEmpty) throw new IllegalArgumentException(s"Field ${structField.name} is not found among available props, which are: ${properties.map(_.getName).mkString(", ")}") - val fieldName = structField.name - val propClass = structField.dataType.asInstanceOf[DataTypeWithClass].cls - val propDt = structField.dataType.asInstanceOf[DataTypeWithClass] - val fieldValue = Invoke( - inputObject, - maybeProp.get.getReadMethod.getName, - inferExternalType(propClass), - returnNullable = structField.nullable - ) - val newPath = walkedTypePath.recordField(propClass.getName, fieldName) - (fieldName, serializerFor(fieldValue, getType(propClass), newPath, seenTypeSet, if (propDt.isInstanceOf[ComplexWrapper]) Some(propDt) else None)) - - } - createSerializerForObject(inputObject, fields) - - case otherTypeWrapper: ComplexWrapper => - otherTypeWrapper.dt match { - case MapType(kt, vt, _) => - val Seq(keyType, valueType) = Seq(kt, vt).map(_.asInstanceOf[DataTypeWithClass].cls).map(getType(_)) - val Seq(keyDT, valueDT) = Seq(kt, vt).map(_.asInstanceOf[DataTypeWithClass]) - val keyClsName = getClassNameFromType(keyType) - val valueClsName = getClassNameFromType(valueType) - val keyPath = walkedTypePath.recordKeyForMap(keyClsName) - val valuePath = walkedTypePath.recordValueForMap(valueClsName) - - createSerializerForMap( - inputObject, - MapElementInformation( - dataTypeFor(keyType), - nullable = !keyType.typeSymbol.asClass.isPrimitive, - serializerFor(_, keyType, keyPath, seenTypeSet, Some(keyDT).filter(_.isInstanceOf[ComplexWrapper]))), - MapElementInformation( - dataTypeFor(valueType), - nullable = !valueType.typeSymbol.asClass.isPrimitive, - serializerFor(_, valueType, valuePath, seenTypeSet, Some(valueDT).filter(_.isInstanceOf[ComplexWrapper]))) - ) - case ArrayType(elementType, _) => - toCatalystArray(inputObject, getType(elementType.asInstanceOf[DataTypeWithClass].cls), Some(elementType.asInstanceOf[DataTypeWithClass])) - - case StructType(elementType: Array[StructField]) => - val cls = otherTypeWrapper.cls - val names = elementType.map(_.name) - - val beanInfo = Introspector.getBeanInfo(cls) - val methods = beanInfo.getMethodDescriptors.filter(it => names.contains(it.getName)) - - - val fields = elementType.map { structField => - - val maybeProp = methods.find(it => it.getName == structField.name) - if (maybeProp.isEmpty) throw new IllegalArgumentException(s"Field ${structField.name} is not found among available props, which are: ${methods.map(_.getName).mkString(", ")}") - val fieldName = structField.name - val propClass = structField.dataType.asInstanceOf[DataTypeWithClass].cls - val propDt = structField.dataType.asInstanceOf[DataTypeWithClass] - val fieldValue = Invoke( - inputObject, - maybeProp.get.getName, - inferExternalType(propClass), - returnNullable = propDt.nullable - ) - val newPath = walkedTypePath.recordField(propClass.getName, fieldName) - (fieldName, serializerFor(fieldValue, getType(propClass), newPath, seenTypeSet, if (propDt.isInstanceOf[ComplexWrapper]) Some(propDt) else None)) - - } - createSerializerForObject(inputObject, fields) - - case _ => - throw new UnsupportedOperationException( - s"No Encoder found for $tpe\n" + walkedTypePath) - - } - } - - case t if definedByConstructorParams(t) => - if (seenTypeSet.contains(t)) { - throw new UnsupportedOperationException( - s"cannot have circular references in class, but got the circular reference of class $t") - } - - val params = getConstructorParameters(t) - val fields = params.map { case (fieldName, fieldType) => - if (javaKeywords.contains(fieldName)) { - throw new UnsupportedOperationException(s"`$fieldName` is a reserved keyword and " + - "cannot be used as field name\n" + walkedTypePath) - } - - // SPARK-26730 inputObject won't be null with If's guard below. And KnownNotNul - // is necessary here. Because for a nullable nested inputObject with struct data - // type, e.g. StructType(IntegerType, StringType), it will return nullable=true - // for IntegerType without KnownNotNull. And that's what we do not expect to. - val fieldValue = Invoke(KnownNotNull(inputObject), fieldName, dataTypeFor(fieldType), - returnNullable = !fieldType.typeSymbol.asClass.isPrimitive) - val clsName = getClassNameFromType(fieldType) - val newPath = walkedTypePath.recordField(clsName, fieldName) - (fieldName, serializerFor(fieldValue, fieldType, newPath, seenTypeSet + t)) - } - createSerializerForObject(inputObject, fields) - - case _ => - throw new UnsupportedOperationException( - s"No Encoder found for $tpe\n" + walkedTypePath) - } - } - - def createDeserializerForString(path: Expression, returnNullable: Boolean): Expression = { - Invoke(path, "toString", ObjectType(classOf[java.lang.String]), - returnNullable = returnNullable) - } - - def getJavaBeanReadableProperties(beanClass: Class[_]): Array[PropertyDescriptor] = { - val beanInfo = Introspector.getBeanInfo(beanClass) - beanInfo.getPropertyDescriptors.filterNot(_.getName == "class") - .filterNot(_.getName == "declaringClass") - .filter(_.getReadMethod != null) - } - - /* - * Retrieves the runtime class corresponding to the provided type. - */ - def getClassFromType(tpe: Type): Class[_] = mirror.runtimeClass(tpe.dealias.typeSymbol.asClass) - - case class Schema(dataType: DataType, nullable: Boolean) - - /** Returns a catalyst DataType and its nullability for the given Scala Type using reflection. */ - def schemaFor(tpe: `Type`): Schema = cleanUpReflectionObjects { - baseType(tpe) match { - // this must be the first case, since all objects in scala are instances of Null, therefore - // Null type would wrongly match the first of them, which is Option as of now - case t if isSubtype(t, definitions.NullTpe) => Schema(NullType, nullable = true) - case t if t.typeSymbol.annotations.exists(_.tree.tpe =:= typeOf[SQLUserDefinedType]) => - val udt = getClassFromType(t).getAnnotation(classOf[SQLUserDefinedType]).udt(). - getConstructor().newInstance() - Schema(udt, nullable = true) - case t if UDTRegistration.exists(getClassNameFromType(t)) => - val udt = UDTRegistration.getUDTFor(getClassNameFromType(t)).get.getConstructor(). - newInstance().asInstanceOf[UserDefinedType[_]] - Schema(udt, nullable = true) - case t if isSubtype(t, localTypeOf[Option[_]]) => - val TypeRef(_, _, Seq(optType)) = t - Schema(schemaFor(optType).dataType, nullable = true) - case t if isSubtype(t, localTypeOf[Array[Byte]]) => Schema(BinaryType, nullable = true) - case t if isSubtype(t, localTypeOf[Array[_]]) => - val TypeRef(_, _, Seq(elementType)) = t - val Schema(dataType, nullable) = schemaFor(elementType) - Schema(ArrayType(dataType, containsNull = nullable), nullable = true) - case t if isSubtype(t, localTypeOf[Seq[_]]) => - val TypeRef(_, _, Seq(elementType)) = t - val Schema(dataType, nullable) = schemaFor(elementType) - Schema(ArrayType(dataType, containsNull = nullable), nullable = true) - case t if isSubtype(t, localTypeOf[Map[_, _]]) => - val TypeRef(_, _, Seq(keyType, valueType)) = t - val Schema(valueDataType, valueNullable) = schemaFor(valueType) - Schema(MapType(schemaFor(keyType).dataType, - valueDataType, valueContainsNull = valueNullable), nullable = true) - case t if isSubtype(t, localTypeOf[Set[_]]) => - val TypeRef(_, _, Seq(elementType)) = t - val Schema(dataType, nullable) = schemaFor(elementType) - Schema(ArrayType(dataType, containsNull = nullable), nullable = true) - case t if isSubtype(t, localTypeOf[String]) => Schema(StringType, nullable = true) - case t if isSubtype(t, localTypeOf[java.time.Instant]) => - Schema(TimestampType, nullable = true) - case t if isSubtype(t, localTypeOf[java.sql.Timestamp]) => - Schema(TimestampType, nullable = true) - case t if isSubtype(t, localTypeOf[java.time.LocalDate]) => Schema(DateType, nullable = true) - case t if isSubtype(t, localTypeOf[java.sql.Date]) => Schema(DateType, nullable = true) - case t if isSubtype(t, localTypeOf[BigDecimal]) => - Schema(DecimalType.SYSTEM_DEFAULT, nullable = true) - case t if isSubtype(t, localTypeOf[java.math.BigDecimal]) => - Schema(DecimalType.SYSTEM_DEFAULT, nullable = true) - case t if isSubtype(t, localTypeOf[java.math.BigInteger]) => - Schema(DecimalType.BigIntDecimal, nullable = true) - case t if isSubtype(t, localTypeOf[scala.math.BigInt]) => - Schema(DecimalType.BigIntDecimal, nullable = true) - case t if isSubtype(t, localTypeOf[Decimal]) => - Schema(DecimalType.SYSTEM_DEFAULT, nullable = true) - case t if isSubtype(t, localTypeOf[java.lang.Integer]) => Schema(IntegerType, nullable = true) - case t if isSubtype(t, localTypeOf[java.lang.Long]) => Schema(LongType, nullable = true) - case t if isSubtype(t, localTypeOf[java.lang.Double]) => Schema(DoubleType, nullable = true) - case t if isSubtype(t, localTypeOf[java.lang.Float]) => Schema(FloatType, nullable = true) - case t if isSubtype(t, localTypeOf[java.lang.Short]) => Schema(ShortType, nullable = true) - case t if isSubtype(t, localTypeOf[java.lang.Byte]) => Schema(ByteType, nullable = true) - case t if isSubtype(t, localTypeOf[java.lang.Boolean]) => Schema(BooleanType, nullable = true) - case t if isSubtype(t, definitions.IntTpe) => Schema(IntegerType, nullable = false) - case t if isSubtype(t, definitions.LongTpe) => Schema(LongType, nullable = false) - case t if isSubtype(t, definitions.DoubleTpe) => Schema(DoubleType, nullable = false) - case t if isSubtype(t, definitions.FloatTpe) => Schema(FloatType, nullable = false) - case t if isSubtype(t, definitions.ShortTpe) => Schema(ShortType, nullable = false) - case t if isSubtype(t, definitions.ByteTpe) => Schema(ByteType, nullable = false) - case t if isSubtype(t, definitions.BooleanTpe) => Schema(BooleanType, nullable = false) - case t if definedByConstructorParams(t) => - val params = getConstructorParameters(t) - Schema(StructType( - params.map { case (fieldName, fieldType) => - val Schema(dataType, nullable) = schemaFor(fieldType) - StructField(fieldName, dataType, nullable) - }), nullable = true) - case other => - throw new UnsupportedOperationException(s"Schema for type $other is not supported") - } - } - - /** - * Whether the fields of the given type is defined entirely by its constructor parameters. - */ - def definedByConstructorParams(tpe: Type): Boolean = cleanUpReflectionObjects { - tpe.dealias match { - // `Option` is a `Product`, but we don't wanna treat `Option[Int]` as a struct type. - case t if isSubtype(t, localTypeOf[Option[_]]) => definedByConstructorParams(t.typeArgs.head) - case _ => isSubtype(tpe.dealias, localTypeOf[Product]) || - isSubtype(tpe.dealias, localTypeOf[DefinedByConstructorParams]) - } - } - - private val javaKeywords = Set("abstract", "assert", "boolean", "break", "byte", "case", "catch", - "char", "class", "const", "continue", "default", "do", "double", "else", "extends", "false", - "final", "finally", "float", "for", "goto", "if", "implements", "import", "instanceof", "int", - "interface", "long", "native", "new", "null", "package", "private", "protected", "public", - "return", "short", "static", "strictfp", "super", "switch", "synchronized", "this", "throw", - "throws", "transient", "true", "try", "void", "volatile", "while") - - - @scala.annotation.tailrec - def javaBoxedType(dt: DataType): Class[_] = dt match { - case _: DecimalType => classOf[Decimal] - case BinaryType => classOf[Array[Byte]] - case StringType => classOf[UTF8String] - case CalendarIntervalType => classOf[CalendarInterval] - case _: StructType => classOf[InternalRow] - case _: ArrayType => classOf[ArrayType] - case _: MapType => classOf[MapType] - case udt: UserDefinedType[_] => javaBoxedType(udt.sqlType) - case ObjectType(cls) => cls - case _ => ScalaReflection.typeBoxedJavaMapping.getOrElse(dt, classOf[java.lang.Object]) - } - -} - -/** - * Support for generating catalyst schemas for scala objects. Note that unlike its companion - * object, this trait able to work in both the runtime and the compile time (macro) universe. - */ -trait KotlinReflection extends Logging { - /** The universe we work in (runtime or macro) */ - val universe: scala.reflect.api.Universe - - /** The mirror used to access types in the universe */ - def mirror: universe.Mirror - - import universe._ - - // The Predef.Map is scala.collection.immutable.Map. - // Since the map values can be mutable, we explicitly import scala.collection.Map at here. - - /** - * Any codes calling `scala.reflect.api.Types.TypeApi.<:<` should be wrapped by this method to - * clean up the Scala reflection garbage automatically. Otherwise, it will leak some objects to - * `scala.reflect.runtime.JavaUniverse.undoLog`. - * - * @see https://github.com/scala/bug/issues/8302 - */ - def cleanUpReflectionObjects[T](func: => T): T = { - universe.asInstanceOf[scala.reflect.runtime.JavaUniverse].undoLog.undo(func) - } - - /** - * Return the Scala Type for `T` in the current classloader mirror. - * - * Use this method instead of the convenience method `universe.typeOf`, which - * assumes that all types can be found in the classloader that loaded scala-reflect classes. - * That's not necessarily the case when running using Eclipse launchers or even - * Sbt console or test (without `fork := true`). - * - * @see SPARK-5281 - */ - def localTypeOf[T: TypeTag]: `Type` = { - val tag = implicitly[TypeTag[T]] - tag.in(mirror).tpe.dealias - } - - /** - * Returns the full class name for a type. The returned name is the canonical - * Scala name, where each component is separated by a period. It is NOT the - * Java-equivalent runtime name (no dollar signs). - * - * In simple cases, both the Scala and Java names are the same, however when Scala - * generates constructs that do not map to a Java equivalent, such as singleton objects - * or nested classes in package objects, it uses the dollar sign ($) to create - * synthetic classes, emulating behaviour in Java bytecode. - */ - def getClassNameFromType(tpe: `Type`): String = { - tpe.dealias.erasure.typeSymbol.asClass.fullName - } - - /** - * Returns the parameter names and types for the primary constructor of this type. - * - * Note that it only works for scala classes with primary constructor, and currently doesn't - * support inner class. - */ - def getConstructorParameters(tpe: Type): Seq[(String, Type)] = { - val dealiasedTpe = tpe.dealias - val formalTypeArgs = dealiasedTpe.typeSymbol.asClass.typeParams - val TypeRef(_, _, actualTypeArgs) = dealiasedTpe - val params = constructParams(dealiasedTpe) - // if there are type variables to fill in, do the substitution (SomeClass[T] -> SomeClass[Int]) - if (actualTypeArgs.nonEmpty) { - params.map { p => - p.name.decodedName.toString -> - p.typeSignature.substituteTypes(formalTypeArgs, actualTypeArgs) - } - } else { - params.map { p => - p.name.decodedName.toString -> p.typeSignature - } - } - } - - /** - * If our type is a Scala trait it may have a companion object that - * only defines a constructor via `apply` method. - */ - private def getCompanionConstructor(tpe: Type): Symbol = { - def throwUnsupportedOperation = { - throw new UnsupportedOperationException(s"Unable to find constructor for $tpe. " + - s"This could happen if $tpe is an interface, or a trait without companion object " + - "constructor.") - } - - tpe.typeSymbol.asClass.companion match { - case NoSymbol => throwUnsupportedOperation - case sym => sym.asTerm.typeSignature.member(universe.TermName("apply")) match { - case NoSymbol => throwUnsupportedOperation - case constructorSym => constructorSym - } - } - } - - protected def constructParams(tpe: Type): Seq[Symbol] = { - val constructorSymbol = tpe.member(termNames.CONSTRUCTOR) match { - case NoSymbol => getCompanionConstructor(tpe) - case sym => sym - } - val params = if (constructorSymbol.isMethod) { - constructorSymbol.asMethod.paramLists - } else { - // Find the primary constructor, and use its parameter ordering. - val primaryConstructorSymbol: Option[Symbol] = constructorSymbol.asTerm.alternatives.find( - s => s.isMethod && s.asMethod.isPrimaryConstructor) - if (primaryConstructorSymbol.isEmpty) { - sys.error("Internal SQL error: Product object did not have a primary constructor.") - } else { - primaryConstructorSymbol.get.asMethod.paramLists - } - } - params.flatten - } - -} - diff --git a/core/3.2/src/main/scala/org/apache/spark/sql/KotlinWrappers.scala b/core/3.2/src/main/scala/org/apache/spark/sql/KotlinWrappers.scala deleted file mode 100644 index 675110be..00000000 --- a/core/3.2/src/main/scala/org/apache/spark/sql/KotlinWrappers.scala +++ /dev/null @@ -1,214 +0,0 @@ -/*- - * =LICENSE= - * Kotlin Spark API: Examples - * ---------- - * Copyright (C) 2019 - 2020 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.apache.spark.sql - -import org.apache.spark.sql.catalyst.analysis.Resolver -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression} -import org.apache.spark.sql.catalyst.util.StringUtils -import org.apache.spark.sql.types.{DataType, Metadata, StructField, StructType} - - -trait DataTypeWithClass { - val dt: DataType - val cls: Class[_] - val nullable: Boolean -} - -trait ComplexWrapper extends DataTypeWithClass - -class KDataTypeWrapper(val dt: StructType - , val cls: Class[_] - , val nullable: Boolean = true) extends StructType with ComplexWrapper { - override def fieldNames: Array[String] = dt.fieldNames - - override def names: Array[String] = dt.names - - override def equals(that: Any): Boolean = dt.equals(that) - - override def hashCode(): Int = dt.hashCode() - - override def add(field: StructField): StructType = dt.add(field) - - override def add(name: String, dataType: DataType): StructType = dt.add(name, dataType) - - override def add(name: String, dataType: DataType, nullable: Boolean): StructType = dt.add(name, dataType, nullable) - - override def add(name: String, dataType: DataType, nullable: Boolean, metadata: Metadata): StructType = dt.add(name, dataType, nullable, metadata) - - override def add(name: String, dataType: DataType, nullable: Boolean, comment: String): StructType = dt.add(name, dataType, nullable, comment) - - override def add(name: String, dataType: String): StructType = dt.add(name, dataType) - - override def add(name: String, dataType: String, nullable: Boolean): StructType = dt.add(name, dataType, nullable) - - override def add(name: String, dataType: String, nullable: Boolean, metadata: Metadata): StructType = dt.add(name, dataType, nullable, metadata) - - override def add(name: String, dataType: String, nullable: Boolean, comment: String): StructType = dt.add(name, dataType, nullable, comment) - - override def apply(name: String): StructField = dt.apply(name) - - override def apply(names: Set[String]): StructType = dt.apply(names) - - override def fieldIndex(name: String): Int = dt.fieldIndex(name) - - override private[sql] def getFieldIndex(name: String) = dt.getFieldIndex(name) - - private[sql] def findNestedField(fieldNames: Seq[String], includeCollections: Boolean, resolver: Resolver) = dt.findNestedField(fieldNames, includeCollections, resolver) - - override private[sql] def buildFormattedString(prefix: String, stringConcat: StringUtils.StringConcat, maxDepth: Int): Unit = dt.buildFormattedString(prefix, stringConcat, maxDepth) - - override protected[sql] def toAttributes: Seq[AttributeReference] = dt.toAttributes - - override def treeString: String = dt.treeString - - override def treeString(maxDepth: Int): String = dt.treeString(maxDepth) - - override def printTreeString(): Unit = dt.printTreeString() - - private[sql] override def jsonValue = dt.jsonValue - - override def apply(fieldIndex: Int): StructField = dt.apply(fieldIndex) - - override def length: Int = dt.length - - override def iterator: Iterator[StructField] = dt.iterator - - override def defaultSize: Int = dt.defaultSize - - override def simpleString: String = dt.simpleString - - override def catalogString: String = dt.catalogString - - override def sql: String = dt.sql - - override def toDDL: String = dt.toDDL - - private[sql] override def simpleString(maxNumberFields: Int) = dt.simpleString(maxNumberFields) - - override private[sql] def merge(that: StructType) = dt.merge(that) - - private[spark] override def asNullable = dt.asNullable - - private[spark] override def existsRecursively(f: DataType => Boolean) = dt.existsRecursively(f) - - override private[sql] lazy val interpretedOrdering = dt.interpretedOrdering - - override def toString = s"KDataTypeWrapper(dt=$dt, cls=$cls, nullable=$nullable)" -} - -case class KComplexTypeWrapper(dt: DataType, cls: Class[_], nullable: Boolean) extends DataType with ComplexWrapper { - override private[sql] def unapply(e: Expression) = dt.unapply(e) - - override def typeName: String = dt.typeName - - override private[sql] def jsonValue = dt.jsonValue - - override def json: String = dt.json - - override def prettyJson: String = dt.prettyJson - - override def simpleString: String = dt.simpleString - - override def catalogString: String = dt.catalogString - - override private[sql] def simpleString(maxNumberFields: Int) = dt.simpleString(maxNumberFields) - - override def sql: String = dt.sql - - override private[spark] def sameType(other: DataType) = dt.sameType(other) - - override private[spark] def existsRecursively(f: DataType => Boolean) = dt.existsRecursively(f) - - private[sql] override def defaultConcreteType = dt.defaultConcreteType - - private[sql] override def acceptsType(other: DataType) = dt.acceptsType(other) - - override def defaultSize: Int = dt.defaultSize - - override private[spark] def asNullable = dt.asNullable - -} - -case class KSimpleTypeWrapper(dt: DataType, cls: Class[_], nullable: Boolean) extends DataType with DataTypeWithClass { - override private[sql] def unapply(e: Expression) = dt.unapply(e) - - override def typeName: String = dt.typeName - - override private[sql] def jsonValue = dt.jsonValue - - override def json: String = dt.json - - override def prettyJson: String = dt.prettyJson - - override def simpleString: String = dt.simpleString - - override def catalogString: String = dt.catalogString - - override private[sql] def simpleString(maxNumberFields: Int) = dt.simpleString(maxNumberFields) - - override def sql: String = dt.sql - - override private[spark] def sameType(other: DataType) = dt.sameType(other) - - override private[spark] def existsRecursively(f: DataType => Boolean) = dt.existsRecursively(f) - - private[sql] override def defaultConcreteType = dt.defaultConcreteType - - private[sql] override def acceptsType(other: DataType) = dt.acceptsType(other) - - override def defaultSize: Int = dt.defaultSize - - override private[spark] def asNullable = dt.asNullable -} - -class KStructField(val getterName: String, val delegate: StructField) extends StructField { - override private[sql] def buildFormattedString(prefix: String, stringConcat: StringUtils.StringConcat, maxDepth: Int): Unit = delegate.buildFormattedString(prefix, stringConcat, maxDepth) - - override def toString(): String = delegate.toString() - - override private[sql] def jsonValue = delegate.jsonValue - - override def withComment(comment: String): StructField = delegate.withComment(comment) - - override def getComment(): Option[String] = delegate.getComment() - - override def toDDL: String = delegate.toDDL - - override def productElement(n: Int): Any = delegate.productElement(n) - - override def productArity: Int = delegate.productArity - - override def productIterator: Iterator[Any] = delegate.productIterator - - override def productPrefix: String = delegate.productPrefix - - override val dataType: DataType = delegate.dataType - - override def canEqual(that: Any): Boolean = delegate.canEqual(that) - - override val metadata: Metadata = delegate.metadata - override val name: String = delegate.name - override val nullable: Boolean = delegate.nullable -} - -object helpme { - - def listToSeq(i: java.util.List[_]): Seq[_] = Seq(i.toArray: _*) -} \ No newline at end of file diff --git a/dummy/pom.xml b/dummy/pom.xml index 041d9c4d..0b92e47d 100644 --- a/dummy/pom.xml +++ b/dummy/pom.xml @@ -1,9 +1,9 @@ - kotlin-spark-api-parent + kotlin-spark-api-parent-3.0 org.jetbrains.kotlinx.spark - 1.0.3 + 1.1.0 pom 4.0.0 @@ -16,7 +16,7 @@ org.jetbrains.kotlinx.spark - examples-3.2_2.12 + examples-3.0_2.12 ${project.parent.version} diff --git a/examples/pom-3.0_2.12.xml b/examples/pom-3.0_2.12.xml new file mode 100644 index 00000000..92ae4dcd --- /dev/null +++ b/examples/pom-3.0_2.12.xml @@ -0,0 +1,109 @@ + + + + 4.0.0 + + Kotlin Spark API: Examples for Spark 3.0+ (Scala 2.12) + Example of usage + examples-3.0_2.12 + + org.jetbrains.kotlinx.spark + kotlin-spark-api-parent-3.0_2.12 + 1.1.0 + ../pom_2.12.xml + + + + + org.jetbrains.kotlinx.spark + kotlin-spark-api-3.0 + ${project.version} + + + org.apache.spark + spark-sql_${scala.compat.version} + ${spark3.version} + + + org.apache.spark + spark-streaming_${scala.compat.version} + ${spark3.version} + + + org.apache.spark + spark-streaming-kafka-0-10_${scala.compat.version} + ${spark3.version} + + + + + src/main/kotlin + src/test/kotlin + target/3.0/${scala.compat.version} + + + org.jetbrains.kotlin + kotlin-maven-plugin + + + compile + + compile + + + + test-compile + + test-compile + + + + + + org.apache.maven.plugins + maven-assembly-plugin + ${maven-assembly-plugin.version} + + + jar-with-dependencies + + + + org.jetbrains.spark.api.examples.WordCountKt + + + + + + org.apache.maven.plugins + maven-site-plugin + + true + + + + org.apache.maven.plugins + maven-deploy-plugin + + true + + + + org.sonatype.plugins + nexus-staging-maven-plugin + + true + + + + org.apache.maven.plugins + maven-compiler-plugin + ${maven-compiler-plugin.version} + + 9 + 9 + + + + + diff --git a/examples/pom-3.2_2.12.xml b/examples/pom-3.2_2.12.xml deleted file mode 100644 index 5a4510d0..00000000 --- a/examples/pom-3.2_2.12.xml +++ /dev/null @@ -1,90 +0,0 @@ - - - - 4.0.0 - - Kotlin Spark API: Examples for Spark 3.2+ (Scala 2.12) - Example of usage - examples-3.2_2.12 - - org.jetbrains.kotlinx.spark - kotlin-spark-api-parent_2.12 - 1.0.3 - ../pom_2.12.xml - - - - - org.jetbrains.kotlinx.spark - kotlin-spark-api-3.2 - ${project.version} - - - org.apache.spark - spark-sql_${scala.compat.version} - ${spark3.version} - - - - - src/main/kotlin - src/test/kotlin - target/3.2/${scala.compat.version} - - - org.jetbrains.kotlin - kotlin-maven-plugin - - - compile - - compile - - - - test-compile - - test-compile - - - - - - org.apache.maven.plugins - maven-assembly-plugin - ${maven-assembly-plugin.version} - - - jar-with-dependencies - - - - org.jetbrains.spark.api.examples.WordCountKt - - - - - - org.apache.maven.plugins - maven-site-plugin - - true - - - - org.apache.maven.plugins - maven-deploy-plugin - - true - - - - org.sonatype.plugins - nexus-staging-maven-plugin - - true - - - - - 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 6ef97b63..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 @@ -31,12 +30,12 @@ data class SomeClass(val a: IntArray, val b: Int) : Serializable fun main() = withSpark { val broadcastVariable = spark.broadcast(SomeClass(a = intArrayOf(5, 6), b = 3)) val result = listOf(1, 2, 3, 4, 5) - .toDS() - .map { - val receivedBroadcast = broadcastVariable.value - it + receivedBroadcast.a.first() - } - .collectAsList() + .toDS() + .map { + val receivedBroadcast = broadcastVariable.value + it + receivedBroadcast.a.first() + } + .collectAsList() println(result) } diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/CachedOperations.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/CachedOperations.kt index 8dad442a..1904de74 100644 --- a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/CachedOperations.kt +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/CachedOperations.kt @@ -20,17 +20,18 @@ package org.jetbrains.kotlinx.spark.examples import org.jetbrains.kotlinx.spark.api.* +import org.jetbrains.kotlinx.spark.api.tuples.* fun main() { withSpark { dsOf(1, 2, 3, 4, 5) - .map { it to (it + 2) } - .withCached { - showDS() + .map { it X (it + 2) } + .withCached { + showDS() - filter { it.first % 2 == 0 }.showDS() - } - .map { c(it.first, it.second, (it.first + it.second) * 2) } - .show() + filter { it._1 % 2 == 0 }.showDS() + } + .map { it.appendedBy(it._1 + it._2 * 2) } + .show() } } diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Collect.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Collect.kt index 685c1ae4..a956bd15 100644 --- a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Collect.kt +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Collect.kt @@ -21,6 +21,7 @@ package org.jetbrains.kotlinx.spark.examples import org.apache.spark.sql.Row import org.jetbrains.kotlinx.spark.api.* +import org.jetbrains.kotlinx.spark.api.tuples.* fun main() { withSpark { @@ -39,7 +40,7 @@ fun main() { } dsOf(1, 2, 3) - .map { c(it, it + 1, it + 2) } + .map { t(it, it + 1, it + 2) } .to() .select("_1") .collectAsList() diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Group.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Group.kt index 89ded8e3..7fde698e 100644 --- a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Group.kt +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Group.kt @@ -20,12 +20,19 @@ package org.jetbrains.kotlinx.spark.examples import org.jetbrains.kotlinx.spark.api.* +import org.jetbrains.kotlinx.spark.api.tuples.* fun main() { withSpark { - dsOf(c(1, "a"), c(1, "b"), c(2, "c")) + dsOf( + 1 X "a", + 1 X "b", + 2 X "c", + ) .groupByKey { it._1 } - .reduceGroupsK { a, b -> c(a._1 + b._1, a._2 + b._2) } + .reduceGroupsK { a, b -> + tupleOf(a._1 + b._1, a._2 + b._2) + } .show() } } diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Join.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Join.kt index 594a2474..684674f2 100644 --- a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Join.kt +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/Join.kt @@ -20,6 +20,7 @@ package org.jetbrains.kotlinx.spark.examples import org.jetbrains.kotlinx.spark.api.* +import org.jetbrains.kotlinx.spark.api.tuples.* data class Left(val id: Int, val name: String) @@ -32,10 +33,12 @@ fun main() { val first = dsOf(Left(1, "a"), Left(2, "b")) val second = dsOf(Right(1, 100), Right(3, 300)) first - .leftJoin(second, first.col("id").eq(second.col("id"))) + .leftJoin(second, first.col("id") eq second.col("id")) .debugCodegen() .also { it.show() } - .map { c(it.first.id, it.first.name, it.second?.value) } + .map { (left, right) -> + left.id X left.name X right?.value + } .show() } diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/JupyterExample.ipynb b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/JupyterExample.ipynb new file mode 100644 index 00000000..33d9e27b --- /dev/null +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/JupyterExample.ipynb @@ -0,0 +1,351 @@ +{ + "cells": [ + { + "cell_type": "markdown", + "source": [ + "By default the latest version of the API and the latest supported Spark version is chosen.\n", + "To specify your own: `%use spark(spark=3.2, v=1.1.0)`" + ], + "metadata": { + "collapsed": false, + "pycharm": { + "name": "#%% md\n" + } + } + }, + { + "cell_type": "code", + "execution_count": 2, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": [ + "Spark session has been started and is running. No `withSpark { }` necessary, you can access `spark` and `sc` directly. To use Spark streaming, use `%use spark-streaming` instead.\n" + ] + } + ], + "source": [ + "%use spark" + ], + "metadata": { + "collapsed": false, + "pycharm": { + "name": "#%%\n" + } + } + }, + { + "cell_type": "markdown", + "source": [ + "Let's define some enums and data classes to work with." + ], + "metadata": { + "collapsed": false, + "pycharm": { + "name": "#%% md\n" + } + } + }, + { + "cell_type": "code", + "execution_count": 3, + "metadata": { + "pycharm": { + "name": "#%%\n" + } + }, + "outputs": [], + "source": [ + "enum class EyeColor {\n", + " BLUE, BROWN, GREEN\n", + "}\n", + "\n", + "enum class Gender {\n", + " MALE, FEMALE, OTHER\n", + "}" + ] + }, + { + "cell_type": "code", + "execution_count": 4, + "metadata": { + "pycharm": { + "name": "#%%\n" + } + }, + "outputs": [], + "source": [ + "data class Person(\n", + " val eyeColor: EyeColor,\n", + " val name: String,\n", + " val gender: Gender,\n", + " val length: Double,\n", + " val age: Int,\n", + ")" + ] + }, + { + "cell_type": "markdown", + "source": [ + "And now we can simply create a Dataset. We can see the contents of a Dataset by simply stating it. As seen below:" + ], + "metadata": { + "collapsed": false, + "pycharm": { + "name": "#%% md\n" + } + } + }, + { + "cell_type": "code", + "execution_count": 6, + "metadata": { + "pycharm": { + "name": "#%%\n" + } + }, + "outputs": [ + { + "data": { + "text/html": "\n \n\n\n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n
eyeColornamegenderlengthage
BLUEAliceFEMALE1.725
BLUEBobMALE1.6725
BROWNCharlieOTHER1.817
\n" + }, + "execution_count": 6, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "val ds: Dataset = dsOf(\n", + " Person(\n", + " eyeColor = EyeColor.BLUE,\n", + " name = \"Alice\",\n", + " gender = Gender.FEMALE,\n", + " length = 1.70,\n", + " age = 25,\n", + " ),\n", + " Person(\n", + " eyeColor = EyeColor.BLUE,\n", + " name = \"Bob\",\n", + " gender = Gender.MALE,\n", + " length = 1.67,\n", + " age = 25,\n", + " ),\n", + " Person(\n", + " eyeColor = EyeColor.BROWN,\n", + " name = \"Charlie\",\n", + " gender = Gender.OTHER,\n", + " length = 1.80,\n", + " age = 17,\n", + " ),\n", + ")\n", + "\n", + "ds" + ] + }, + { + "cell_type": "markdown", + "source": [ + "The effects of operations like filtering can also be seen immediately, as well as sorting, selecting columns etc..." + ], + "metadata": { + "collapsed": false, + "pycharm": { + "name": "#%% md\n" + } + } + }, + { + "cell_type": "code", + "execution_count": 7, + "metadata": { + "pycharm": { + "name": "#%%\n" + } + }, + "outputs": [ + { + "data": { + "text/html": "\n \n\n\n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n
eyeColornamegenderlengthage
BLUEAliceFEMALE1.725
BLUEBobMALE1.6725
\n" + }, + "execution_count": 7, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "ds.filter { it.age > 20 }" + ] + }, + { + "cell_type": "code", + "execution_count": 8, + "outputs": [ + { + "data": { + "text/html": "\n \n\n\n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n
eyeColornamegenderlengthage
BROWNCharlieOTHER1.817
BLUEBobMALE1.6725
BLUEAliceFEMALE1.725
\n" + }, + "execution_count": 8, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "ds.sort(col(Person::age), col(Person::length))" + ], + "metadata": { + "collapsed": false, + "pycharm": { + "name": "#%%\n" + } + } + }, + { + "cell_type": "code", + "execution_count": 9, + "outputs": [ + { + "data": { + "text/html": "\n \n\n\n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n
agelength
251.7
251.67
171.8
\n" + }, + "execution_count": 9, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "ds.selectTyped(col(Person::age), col(Person::length))" + ], + "metadata": { + "collapsed": false, + "pycharm": { + "name": "#%%\n" + } + } + }, + { + "cell_type": "code", + "execution_count": 10, + "outputs": [ + { + "data": { + "text/plain": "Average length: 1.7233333333333334" + }, + "execution_count": 10, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "\"Average length: \" +\n", + " ds\n", + " .map { it.length }\n", + " .reduceK { a, b -> a + b } / ds.count()" + ], + "metadata": { + "collapsed": false, + "pycharm": { + "name": "#%%\n" + } + } + }, + { + "cell_type": "markdown", + "source": [ + "Extension methods that usually only work in the `withSpark {}` context of the Kotlin Spark API work out of the box in Jupyter.\n", + "This means we can also create a Dataset like this:" + ], + "metadata": { + "collapsed": false, + "pycharm": { + "name": "#%% md\n" + } + } + }, + { + "cell_type": "code", + "execution_count": 11, + "outputs": [ + { + "data": { + "text/html": "\n \n\n\n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n
value
1
2
3
4
\n" + }, + "execution_count": 11, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "listOf(1, 2, 3, 4).toDS()" + ], + "metadata": { + "collapsed": false, + "pycharm": { + "name": "#%%\n" + } + } + }, + { + "cell_type": "markdown", + "source": [ + "We can also create RDDs using `sc: JavaSparkContext` which are rendered similarly to Datasets.\n", + "You can see that all Tuple helper functions are immediately available too." + ], + "metadata": { + "collapsed": false, + "pycharm": { + "name": "#%% md\n" + } + } + }, + { + "cell_type": "code", + "execution_count": 12, + "outputs": [ + { + "data": { + "text/html": "\n \n\n\n \n \n \n \n \n \n \n \n \n \n \n \n
Values
[1, aaa]
[2, bbb]
[3, ccc]
\n" + }, + "execution_count": 12, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "val rdd: JavaRDD> = sc.parallelize(\n", + " listOf(\n", + " 1 X \"aaa\",\n", + " t(2, \"bbb\"),\n", + " tupleOf(3, \"ccc\"),\n", + " )\n", + ")\n", + "\n", + "rdd" + ], + "metadata": { + "collapsed": false, + "pycharm": { + "name": "#%%\n" + } + } + } + ], + "metadata": { + "kernelspec": { + "display_name": "Kotlin", + "language": "kotlin", + "name": "kotlin" + }, + "language_info": { + "codemirror_mode": "text/x-kotlin", + "file_extension": ".kt", + "mimetype": "text/x-kotlin", + "name": "kotlin", + "nbconvert_exporter": "", + "pygments_lexer": "kotlin", + "version": "1.7.0-dev-3303" + } + }, + "nbformat": 4, + "nbformat_minor": 1 +} \ No newline at end of file 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 a253c12d..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,37 +19,55 @@ */ 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.Tuple2 +import scala.Tuple3 data class Q(val id: Int, val text: T) + +@Suppress("RedundantLambdaArrow", "UsePropertyAccessSyntax") object Main { @JvmStatic fun main(args: Array) { - val spark = SparkSession .builder() .master("local[2]") - .appName("Simple Application").orCreate - - val triples = spark - .toDS(listOf(Q(1, 1 to null), Q(2, 2 to "22"), Q(3, 3 to "333"))) - .map { (a, b) -> a + b.first to b.second?.length } - .map { it to 1 } - .map { (a, b) -> Triple(a.first, a.second, b) } + .appName("Simple Application") + .getOrCreate() + val triples: Dataset> = spark + .toDS( + listOf( + Q(1, 1 X null), + Q(2, 2 X "22"), + Q(3, 3 X "333"), + ) + ) + .map { (a, b) -> t(a + b._1, b._2?.length) } + .map { it: Tuple2 -> it + 1 } // add counter val pairs = spark - .toDS(listOf(2 to "hell", 4 to "moon", 6 to "berry")) + .toDS( + listOf( + 2 X "hell", + 4 X "moon", + 6 X "berry", + ) + ) triples - .leftJoin(pairs, triples.col("first").multiply(2).eq(pairs.col("first"))) + .leftJoin( + right = pairs, + col = triples("_1").multiply(2) eq pairs("_1"), + ) // .also { it.printSchema() } - .map { (triple, pair) -> Five(triple.first, triple.second, triple.third, pair?.first, pair?.second) } + .map { (triple, pair) -> Five(triple._1, triple._2, triple._3, pair?._1, pair?._2) } .groupByKey { it.a } .reduceGroupsK { v1, v2 -> v1.copy(a = v1.a + v2.a, b = v1.a + v2.a) } - .map { it.second } + .map { it._2 } .repartition(1) .withCached { write() diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/MapAndListOperations.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/MapAndListOperations.kt index 89bfdacd..8d36017d 100644 --- a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/MapAndListOperations.kt +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/MapAndListOperations.kt @@ -20,18 +20,26 @@ package org.jetbrains.kotlinx.spark.examples import org.jetbrains.kotlinx.spark.api.* +import org.jetbrains.kotlinx.spark.api.tuples.* fun main() { withSpark(props = mapOf("spark.sql.codegen.wholeStage" to true)) { - dsOf(mapOf(1 to c(1, 2, 3), 2 to c(1, 2, 3)), mapOf(3 to c(1, 2, 3), 4 to c(1, 2, 3))) - .flatMap { it.toList().map { p -> listOf(p.first, p.second._1, p.second._2, p.second._3) }.iterator() } - .flatten() - .map { c(it) } - .also { it.printSchema() } - .distinct() - .sort("_1") - .debugCodegen() - .show() + dsOf( + mapOf(1 to t(1, 2, 3), 2 to t(1, 2, 3)), + mapOf(3 to t(1, 2, 3), 4 to t(1, 2, 3)), + ) + .flatMap { + it.toList() + .map { (first, tuple) -> (first + tuple).toList() } + .iterator() + } + .flatten() + .map { tupleOf(it) } + .also { it.printSchema() } + .distinct() + .sort("_1") + .debugCodegen() + .show() } } diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/WordCount.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/WordCount.kt index 88f43a62..c08a9df5 100644 --- a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/WordCount.kt +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/WordCount.kt @@ -21,28 +21,29 @@ package org.jetbrains.kotlinx.spark.examples import org.apache.spark.sql.Dataset import org.jetbrains.kotlinx.spark.api.* +import org.jetbrains.kotlinx.spark.api.tuples.* const val MEANINGFUL_WORD_LENGTH = 4 fun main() { withSpark { spark - .read() - .textFile(this::class.java.classLoader.getResource("the-catcher-in-the-rye.txt")?.path) - .map { it.split(Regex("\\s")) } - .flatten() - .cleanup() - .groupByKey { it } - .mapGroups { k, iter -> k to iter.asSequence().count() } - .sort { arrayOf(it.col("second").desc()) } - .limit(20) - .map { it.second to it.first } - .show(false) + .read() + .textFile(this::class.java.classLoader.getResource("the-catcher-in-the-rye.txt")?.path) + .map { it.split(Regex("\\s")) } + .flatten() + .cleanup() + .groupByKey { it } + .mapGroups { k, iter -> k X iter.asSequence().count() } + .sort { arrayOf(it(colName = "_2").desc()) } + .limit(20) + .map { it.swap() } + .show(false) } } -fun Dataset.cleanup() = - filter { it.isNotBlank() } - .map { it.trim(',', ' ', '\n', ':', '.', ';', '?', '!', '"', '\'', '\t', ' ') } - .filter { !it.endsWith("n’t") } - .filter { it.length >= MEANINGFUL_WORD_LENGTH } +fun Dataset.cleanup(): Dataset = + filter { it.isNotBlank() } + .map { it.trim(',', ' ', '\n', ':', '.', ';', '?', '!', '"', '\'', '\t', ' ') } + .filter { !it.endsWith("n’t") } + .filter { it.length >= MEANINGFUL_WORD_LENGTH } diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/JupyterStreamingExample.ipynb b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/JupyterStreamingExample.ipynb new file mode 100644 index 00000000..2159cfa9 --- /dev/null +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/JupyterStreamingExample.ipynb @@ -0,0 +1,191 @@ +{ + "cells": [ + { + "cell_type": "markdown", + "source": [ + "By default the latest version of the API and the latest supported Spark version is chosen. To specify your own: %use spark-streaming(spark=3.2, v=1.1.0)" + ], + "metadata": { + "collapsed": false, + "pycharm": { + "name": "#%% md\n" + } + } + }, + { + "cell_type": "code", + "execution_count": 2, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": [ + "To start a spark streaming session, simply use `withSparkStreaming { }` inside a cell. To use Spark normally, use `withSpark { }` in a cell, or use `%use spark` to start a Spark session for the whole notebook.\n" + ] + } + ], + "source": [ + "%use spark-streaming" + ], + "metadata": { + "collapsed": false, + "pycharm": { + "name": "#%%\n" + } + } + }, + { + "cell_type": "markdown", + "source": [ + "Let's define some data class to work with." + ], + "metadata": { + "collapsed": false, + "pycharm": { + "name": "#%% md\n" + } + } + }, + { + "cell_type": "code", + "execution_count": 4, + "outputs": [], + "source": [ + "data class TestRow(\n", + " val word: String,\n", + ")" + ], + "metadata": { + "collapsed": false, + "pycharm": { + "name": "#%%\n" + } + } + }, + { + "cell_type": "markdown", + "source": [ + "To run this on your local machine, you need to first run a Netcat server: `$ nc -lk 9999`.\n", + "\n", + "This example will collect the data from this stream for 10 seconds and 1 second intervals, splitting and counting the input per word." + ], + "metadata": { + "collapsed": false, + "pycharm": { + "name": "#%% md\n" + } + } + }, + { + "cell_type": "code", + "execution_count": 5, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": [ + "+---+--------+\n", + "|key|count(1)|\n", + "+---+--------+\n", + "+---+--------+\n", + "\n", + "+-----+--------+\n", + "| key|count(1)|\n", + "+-----+--------+\n", + "|hello| 8|\n", + "|Hello| 6|\n", + "|world| 3|\n", + "| | 2|\n", + "| test| 4|\n", + "+-----+--------+\n", + "\n", + "+-----+--------+\n", + "| key|count(1)|\n", + "+-----+--------+\n", + "|hello| 3|\n", + "+-----+--------+\n", + "\n", + "+---+--------+\n", + "|key|count(1)|\n", + "+---+--------+\n", + "+---+--------+\n", + "\n", + "+---+--------+\n", + "|key|count(1)|\n", + "+---+--------+\n", + "+---+--------+\n", + "\n", + "+---+--------+\n", + "|key|count(1)|\n", + "+---+--------+\n", + "+---+--------+\n", + "\n", + "+---+--------+\n", + "|key|count(1)|\n", + "+---+--------+\n", + "+---+--------+\n", + "\n", + "+---+--------+\n", + "|key|count(1)|\n", + "+---+--------+\n", + "+---+--------+\n", + "\n", + "+-----+--------+\n", + "| key|count(1)|\n", + "+-----+--------+\n", + "|hello| 1|\n", + "|world| 2|\n", + "+-----+--------+\n", + "\n", + "+---+--------+\n", + "|key|count(1)|\n", + "+---+--------+\n", + "+---+--------+\n", + "\n" + ] + } + ], + "source": [ + "withSparkStreaming(batchDuration = Durations.seconds(1), timeout = 10_000) { // this: KSparkStreamingSession\n", + "\n", + " val lines: JavaReceiverInputDStream = ssc.socketTextStream(\"localhost\", 9999)\n", + " val words: JavaDStream = lines.flatMap { it.split(\" \").iterator() }\n", + "\n", + " words.foreachRDD { rdd: JavaRDD, _: Time ->\n", + " withSpark(rdd) { // this: KSparkSession\n", + " val dataframe: Dataset = rdd.map { TestRow(it) }.toDS()\n", + " dataframe\n", + " .groupByKey { it.word }\n", + " .count()\n", + " .show()\n", + " }\n", + " }\n", + "}" + ], + "metadata": { + "collapsed": false, + "pycharm": { + "name": "#%%\n" + } + } + } + ], + "metadata": { + "kernelspec": { + "display_name": "Kotlin", + "language": "kotlin", + "name": "kotlin" + }, + "language_info": { + "name": "kotlin", + "version": "1.7.0-dev-1825", + "mimetype": "text/x-kotlin", + "file_extension": ".kt", + "pygments_lexer": "kotlin", + "codemirror_mode": "text/x-kotlin", + "nbconvert_exporter": "" + } + }, + "nbformat": 4, + "nbformat_minor": 0 +} \ No newline at end of file diff --git a/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/KotlinDirectKafkaWordCount.kt b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/KotlinDirectKafkaWordCount.kt new file mode 100644 index 00000000..476815e2 --- /dev/null +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/KotlinDirectKafkaWordCount.kt @@ -0,0 +1,114 @@ +/*- + * =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.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.kafka010.ConsumerStrategies +import org.apache.spark.streaming.kafka010.KafkaUtils +import org.apache.spark.streaming.kafka010.LocationStrategies +import org.jetbrains.kotlinx.spark.api.reduceByKey +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 + + +/** + * 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 = "KotlinDirectKafkaWordCount") { + + 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: JavaDStream> = words + .map { it X 1 } + .reduceByKey { a: Int, b: Int -> a + b } + + wordCounts.print() + + } + } +} 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 new file mode 100644 index 00000000..32db58f7 --- /dev/null +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/KotlinRecoverableNetworkWordCount.kt @@ -0,0 +1,225 @@ +/*- + * =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= + */ +@file:OptIn(ExperimentalTime::class) + +package org.jetbrains.kotlinx.spark.examples.streaming + +import com.google.common.io.Files +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 org.jetbrains.kotlinx.spark.api.tuples.* +import java.io.File +import java.nio.charset.Charset +import java.util.regex.Pattern +import kotlin.system.exitProcess +import kotlin.time.ExperimentalTime + + +/** + * Use this singleton to get or register a Broadcast variable. + */ +internal object KotlinWordExcludeList { + + @Volatile + private var instance: Broadcast>? = null + + fun getInstance(sc: JavaSparkContext): Broadcast> { + if (instance == null) synchronized(KotlinWordExcludeList::class) { + 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 KotlinDroppedWordsCounter { + + @Volatile + private var instance: LongAccumulator? = null + + fun getInstance(sc: JavaSparkContext): LongAccumulator { + if (instance == null) synchronized(KotlinDroppedWordsCounter::class) { + if (instance == null) + instance = sc.sc().longAccumulator("DroppedWordsCounter") + } + return instance!! + } +} + +/** + * 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. + * + * 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 + * + * 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.KotlinRecoverableNetworkWordCount \ + * 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 KotlinRecoverableNetworkWordCount { + + 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: 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 + + 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, + batchDuration = Durations.seconds(1), + appName = "KotlinRecoverableNetworkWordCount", + ) { + createContext( + ip = ip, + port = port, + outputPath = outputPath, + ) + } + } + + 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).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') + val lines = ssc.socketTextStream(ip, port) + val words = lines.flatMap { it.split(SPACE).iterator() } + val wordCounts = words + .map { t(it, 1) } + .reduceByKey { a, b -> a + b } + + // in normal streaming context we can create a SparkSession from ssc: JavaStreamingContext + // normally `ssc.sparkContext().conf` + withSpark(ssc) { + listOf(1, 2, 3).toDS().show() + } + + setRunAfterStart { + println("Context is created and started running!") + } + + wordCounts.foreachRDD { rdd, time: Time -> + // 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 = 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) + + @Suppress("UnstableApiUsage") + Files.append( + """ + $output + + """.trimIndent(), + outputFile, + Charset.defaultCharset(), + ) + } + } + } +} 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..51060a20 --- /dev/null +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/KotlinSqlNetworkWordCount.kt @@ -0,0 +1,99 @@ +/*- + * =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 +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/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..2c938ead --- /dev/null +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/KotlinStatefulNetworkCount.kt @@ -0,0 +1,104 @@ +/*- + * =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 +import org.apache.spark.api.java.StorageLevels +import org.apache.spark.streaming.Durations +import org.apache.spark.streaming.State +import org.apache.spark.streaming.StateSpec +import org.jetbrains.kotlinx.spark.api.getOrElse +import org.jetbrains.kotlinx.spark.api.mapWithState +import org.jetbrains.kotlinx.spark.api.toPairRDD +import org.jetbrains.kotlinx.spark.api.tuples.X +import org.jetbrains.kotlinx.spark.api.withSparkStreaming +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() + } + } +} 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 new file mode 100644 index 00000000..85db9775 --- /dev/null +++ b/examples/src/main/kotlin/org/jetbrains/kotlinx/spark/examples/streaming/Streaming.kt @@ -0,0 +1,53 @@ +/*- + * =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 +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( + val word: String, +) + +/** + * To run this on your local machine, you need to first run a Netcat server + * + * `$ nc -lk 9999` + */ +fun main() = withSparkStreaming(batchDuration = Durations.seconds(1), timeout = 10_000) { // this: KSparkStreamingSession + + 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 diff --git a/jupyter/pom.xml b/jupyter/pom.xml new file mode 100644 index 00000000..a8e6a107 --- /dev/null +++ b/jupyter/pom.xml @@ -0,0 +1,182 @@ + + + + 4.0.0 + + Kotlin Spark API: Jupyter integration for Spark 3.0+ (Scala 2.12) + kotlin-spark-api-jupyter-3.0 + Jupyter integration + + org.jetbrains.kotlinx.spark + kotlin-spark-api-parent-3.0_2.12 + 1.1.0 + ../pom_2.12.xml + + jar + + + + 11 + 11 + + + + + kotlinx-html + kotlinx-html + https://maven.pkg.jetbrains.space/public/p/kotlinx-html/maven + + + kotlin + kotlin + https://maven.pkg.jetbrains.space/kotlin/p/kotlin/dev + + + + + + org.jetbrains.kotlinx.spark + kotlin-spark-api-3.0 + ${project.version} + + + org.jetbrains.kotlinx + kotlinx-html-jvm + ${kotlinx.html.version} + + + org.apache.spark + spark-sql_${scala.compat.version} + ${spark3.version} + + + org.apache.spark + spark-repl_${scala.compat.version} + ${spark3.version} + + + org.apache.spark + spark-streaming_${scala.compat.version} + ${spark3.version} + + + + + + + + org.jetbrains.kotlinx + kotlin-jupyter-api + ${kotlin-jupyter-api.version} + + + + + io.kotest + kotest-runner-junit5-jvm + ${kotest.version} + test + + + org.jetbrains.kotlinx + kotlin-jupyter-test-kit + ${kotlin-jupyter-api.version} + test + + + + + src/main/kotlin + src/test/kotlin + target/${scala.compat.version} + + + org.jetbrains.dokka + dokka-maven-plugin + ${dokka.version} + + 8 + + + + dokka + + dokka + + pre-site + + + javadocjar + + javadocJar + + pre-integration-test + + + + + org.jetbrains.kotlin + kotlin-maven-plugin + + + compile + + compile + + + + test-compile + + test-compile + + + + + + org.apache.maven.plugins + maven-assembly-plugin + ${maven-assembly-plugin.version} + + + jar-with-dependencies + + + + org.jetbrains.spark.api.examples.WordCountKt + + + + + + org.apache.maven.plugins + maven-site-plugin + + true + + + + org.apache.maven.plugins + maven-deploy-plugin + + false + + + + org.sonatype.plugins + nexus-staging-maven-plugin + + false + + + + org.apache.maven.plugins + maven-compiler-plugin + ${maven-compiler-plugin.version} + + 9 + 9 + + + + + \ No newline at end of file diff --git a/jupyter/src/main/kotlin/org/jetbrains/kotlinx/spark/api/jupyter/HtmlRendering.kt b/jupyter/src/main/kotlin/org/jetbrains/kotlinx/spark/api/jupyter/HtmlRendering.kt new file mode 100644 index 00000000..ad083962 --- /dev/null +++ b/jupyter/src/main/kotlin/org/jetbrains/kotlinx/spark/api/jupyter/HtmlRendering.kt @@ -0,0 +1,136 @@ +/*- + * =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.jupyter + +import kotlinx.html.* +import kotlinx.html.stream.appendHTML +import org.apache.spark.SparkException +import org.apache.spark.api.java.JavaRDDLike +import org.apache.spark.sql.Dataset +import org.apache.spark.unsafe.array.ByteArrayMethods +import org.jetbrains.kotlinx.spark.api.asKotlinIterable +import org.jetbrains.kotlinx.spark.api.asKotlinIterator +import org.jetbrains.kotlinx.spark.api.asKotlinList +import scala.Product +import java.io.InputStreamReader +import java.io.Serializable + +private fun createHtmlTable(fillTable: TABLE.() -> Unit): String = buildString { + appendHTML().head { + style("text/css") { + unsafe { + val resource = "/table.css" + val res = SparkIntegration::class.java + .getResourceAsStream(resource) ?: error("Resource '$resource' not found") + val readRes = InputStreamReader(res).readText() + raw("\n" + readRes) + } + } + } + + appendHTML().table("dataset", fillTable) +} + + +internal fun JavaRDDLike.toHtml(limit: Int = 20, truncate: Int = 30): String = try { + createHtmlTable { + val numRows = limit.coerceIn(0 until ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) + val tmpRows = take(numRows).toList() + + val hasMoreData = tmpRows.size - 1 > numRows + val rows = tmpRows.take(numRows) + + tr { th { +"Values" } } + + for (row in rows) tr { + td { + val string = when (row) { + is ByteArray -> row.joinToString(prefix = "[", postfix = "]") { "%02X".format(it) } + + is CharArray -> row.iterator().asSequence().toList().toString() + is ShortArray -> row.iterator().asSequence().toList().toString() + is IntArray -> row.iterator().asSequence().toList().toString() + is LongArray -> row.iterator().asSequence().toList().toString() + is FloatArray -> row.iterator().asSequence().toList().toString() + is DoubleArray -> row.iterator().asSequence().toList().toString() + is BooleanArray -> row.iterator().asSequence().toList().toString() + is Array<*> -> row.iterator().asSequence().toList().toString() + is Iterable<*> -> row.iterator().asSequence().toList().toString() + is scala.collection.Iterable<*> -> row.asKotlinIterable().iterator().asSequence().toList().toString() + is Iterator<*> -> row.asSequence().toList().toString() + is scala.collection.Iterator<*> -> row.asKotlinIterator().asSequence().toList().toString() + is Product -> row.productIterator().asKotlinIterator().asSequence().toList().toString() + is Serializable -> row.toString() + // maybe others? + + is Any? -> row.toString() + else -> row.toString() + } + + +if (truncate > 0 && string.length > truncate) { + // do not show ellipses for strings shorter than 4 characters. + if (truncate < 4) string.substring(0, truncate) + else string.substring(0, truncate - 3) + "..." + } else { + string + } + } + } + + if (hasMoreData) tr { + +"only showing top $numRows ${if (numRows == 1) "row" else "rows"}" + } + } +} catch (e: SparkException) { + // Whenever toString() on the contents doesn't work, since the class might be unknown... + """${toString()} + |Cannot render this RDD of this class.""".trimMargin() +} + +internal fun Dataset.toHtml(limit: Int = 20, truncate: Int = 30): String = createHtmlTable { + val numRows = limit.coerceIn(0 until ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) + val tmpRows = getRows(numRows, truncate).asKotlinList().map { it.asKotlinList() } + + val hasMoreData = tmpRows.size - 1 > numRows + val rows = tmpRows.take(numRows + 1) + + tr { + for (header in rows.first()) th { + +if (truncate > 0 && header.length > truncate) { + // do not show ellipses for strings shorter than 4 characters. + if (truncate < 4) header.substring(0, truncate) + else header.substring(0, truncate - 3) + "..." + } else { + header + } + + } + } + + for (row in rows.drop(1)) tr { + for (item in row) td { + +item + } + } + + if (hasMoreData) tr { + +"only showing top $numRows ${if (numRows == 1) "row" else "rows"}" + } +} diff --git a/jupyter/src/main/kotlin/org/jetbrains/kotlinx/spark/api/jupyter/Integration.kt b/jupyter/src/main/kotlin/org/jetbrains/kotlinx/spark/api/jupyter/Integration.kt new file mode 100644 index 00000000..327af92b --- /dev/null +++ b/jupyter/src/main/kotlin/org/jetbrains/kotlinx/spark/api/jupyter/Integration.kt @@ -0,0 +1,168 @@ +/*- + * =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.jupyter + +import org.apache.spark.api.java.JavaRDDLike +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.Dataset +import org.jetbrains.kotlinx.jupyter.api.* +import org.jetbrains.kotlinx.jupyter.api.libraries.JupyterIntegration +import kotlin.reflect.typeOf + +abstract class Integration : JupyterIntegration() { + + private val kotlinVersion = "1.6.21" + private val scalaCompatVersion = "2.12" + private val scalaVersion = "2.12.15" + private val spark3Version = "3.0.3" + + private val displayLimit = "DISPLAY_LIMIT" + private val displayLimitDefault = 20 + private val displayTruncate = "DISPLAY_TRUNCATE" + private val displayTruncateDefault = 30 + + /** + * Will be run after importing all dependencies + */ + open fun KotlinKernelHost.onLoaded() = Unit + + open fun KotlinKernelHost.onShutdown() = Unit + + open fun KotlinKernelHost.onInterrupt() = Unit + + open fun KotlinKernelHost.beforeCellExecution() = Unit + + open fun KotlinKernelHost.afterCellExecution(snippetInstance: Any, result: FieldValue) = Unit + + open fun Builder.onLoadedAlsoDo() = Unit + + open val dependencies: Array = arrayOf( + "org.apache.spark:spark-repl_$scalaCompatVersion:$spark3Version", + "org.jetbrains.kotlin:kotlin-stdlib-jdk8:$kotlinVersion", + "org.jetbrains.kotlin:kotlin-reflect:$kotlinVersion", + "org.apache.spark:spark-sql_$scalaCompatVersion:$spark3Version", + "org.apache.spark:spark-streaming_$scalaCompatVersion:$spark3Version", + "org.apache.spark:spark-mllib_$scalaCompatVersion:$spark3Version", + "org.apache.spark:spark-sql_$scalaCompatVersion:$spark3Version", + "org.apache.spark:spark-graphx_$scalaCompatVersion:$spark3Version", + "org.apache.spark:spark-launcher_$scalaCompatVersion:$spark3Version", + "org.apache.spark:spark-catalyst_$scalaCompatVersion:$spark3Version", + "org.apache.spark:spark-streaming_$scalaCompatVersion:$spark3Version", + "org.apache.spark:spark-core_$scalaCompatVersion:$spark3Version", + "org.scala-lang:scala-library:$scalaVersion", + "org.scala-lang.modules:scala-xml_$scalaCompatVersion:2.0.1", + "org.scala-lang:scala-reflect:$scalaVersion", + "org.scala-lang:scala-compiler:$scalaVersion", + "commons-io:commons-io:2.11.0", + ) + + open val imports: Array = arrayOf( + "org.jetbrains.kotlinx.spark.api.*", + "org.jetbrains.kotlinx.spark.api.tuples.*", + *(1..22).map { "scala.Tuple$it" }.toTypedArray(), + "org.apache.spark.sql.functions.*", + "org.apache.spark.*", + "org.apache.spark.sql.*", + "org.apache.spark.api.java.*", + "scala.collection.Seq", + "org.apache.spark.rdd.*", + "java.io.Serializable", + "org.apache.spark.streaming.api.java.*", + "org.apache.spark.streaming.api.*", + "org.apache.spark.streaming.*", + ) + + override fun Builder.onLoaded() { + dependencies(*dependencies) + import(*imports) + + onLoaded { + declare( + VariableDeclaration( + name = displayLimit, + value = displayLimitDefault, + type = typeOf(), + isMutable = true, + ), + VariableDeclaration( + name = displayTruncate, + value = displayTruncateDefault, + type = typeOf(), + isMutable = true, + ), + ) + + onLoaded() + } + + beforeCellExecution { + execute("""scala.Console.setOut(System.out)""") + + beforeCellExecution() + } + + afterCellExecution { snippetInstance, result -> + afterCellExecution(snippetInstance, result) + } + + onInterrupt { + onInterrupt() + } + + onShutdown { + onShutdown() + } + + fun getLimitAndTruncate() = Pair( + notebook + .variablesState[displayLimit] + ?.value + ?.getOrNull() as? Int + ?: displayLimitDefault, + notebook + .variablesState[displayTruncate] + ?.value + ?.getOrNull() as? Int + ?: displayTruncateDefault + ) + + + // Render Dataset + render> { + val (limit, truncate) = getLimitAndTruncate() + + HTML(it.toHtml(limit = limit, truncate = truncate)) + } + + render> { + val (limit, truncate) = getLimitAndTruncate() + + HTML(it.toJavaRDD().toHtml(limit = limit, truncate = truncate)) + } + + render> { + val (limit, truncate) = getLimitAndTruncate() + + HTML(it.toHtml(limit = limit, truncate = truncate)) + } + + onLoadedAlsoDo() + } +} diff --git a/jupyter/src/main/kotlin/org/jetbrains/kotlinx/spark/api/jupyter/SparkIntegration.kt b/jupyter/src/main/kotlin/org/jetbrains/kotlinx/spark/api/jupyter/SparkIntegration.kt new file mode 100644 index 00000000..e817b282 --- /dev/null +++ b/jupyter/src/main/kotlin/org/jetbrains/kotlinx/spark/api/jupyter/SparkIntegration.kt @@ -0,0 +1,76 @@ +/*- + * =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.jupyter + + +import org.intellij.lang.annotations.Language +import org.jetbrains.kotlinx.jupyter.api.FieldValue +import org.jetbrains.kotlinx.jupyter.api.KotlinKernelHost + +/** + * %use spark + */ +@Suppress("UNUSED_VARIABLE", "LocalVariableName") +@OptIn(ExperimentalStdlibApi::class) +internal class SparkIntegration : Integration() { + + override fun KotlinKernelHost.onLoaded() { + val _0 = execute("""%dumpClassesForSpark""") + + @Language("kts") + val _1 = listOf( + """ + val spark = org.jetbrains.kotlinx.spark.api.SparkSession + .builder() + .master(SparkConf().get("spark.master", "local[*]")) + .appName("Jupyter") + .config("spark.sql.codegen.wholeStage", false) + .getOrCreate()""".trimIndent(), + """ + spark.sparkContext.setLogLevel(org.jetbrains.kotlinx.spark.api.SparkLogLevel.ERROR)""".trimIndent(), + """ + val sc by lazy { + org.apache.spark.api.java.JavaSparkContext(spark.sparkContext) + }""".trimIndent(), + """ + println("Spark session has been started and is running. No `withSpark { }` necessary, you can access `spark` and `sc` directly. To use Spark streaming, use `%use spark-streaming` instead.")""".trimIndent(), + """ + inline fun List.toDS(): Dataset = toDS(spark)""".trimIndent(), + """ + inline fun Array.toDS(): Dataset = spark.dsOf(*this)""".trimIndent(), + """ + inline fun dsOf(vararg arg: T): Dataset = spark.dsOf(*arg)""".trimIndent(), + """ + inline fun RDD.toDS(): Dataset = toDS(spark)""".trimIndent(), + """ + inline fun JavaRDDLike.toDS(): Dataset = toDS(spark)""".trimIndent(), + """ + inline fun RDD.toDF(): Dataset = toDF(spark)""".trimIndent(), + """ + inline fun JavaRDDLike.toDF(): Dataset = toDF(spark)""".trimIndent(), + """ + val udf: UDFRegistration get() = spark.udf()""".trimIndent(), + ).map(::execute) + } + + override fun KotlinKernelHost.onShutdown() { + execute("""spark.stop()""") + } +} diff --git a/jupyter/src/main/kotlin/org/jetbrains/kotlinx/spark/api/jupyter/SparkStreamingIntegration.kt b/jupyter/src/main/kotlin/org/jetbrains/kotlinx/spark/api/jupyter/SparkStreamingIntegration.kt new file mode 100644 index 00000000..441672d3 --- /dev/null +++ b/jupyter/src/main/kotlin/org/jetbrains/kotlinx/spark/api/jupyter/SparkStreamingIntegration.kt @@ -0,0 +1,143 @@ +/*- + * =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.jupyter + + +import org.apache.spark.streaming.StreamingContextState +import org.apache.spark.streaming.api.java.JavaStreamingContext +import org.intellij.lang.annotations.Language +import org.jetbrains.kotlinx.jupyter.api.KotlinKernelHost +import org.jetbrains.kotlinx.jupyter.api.VariableDeclaration +import org.jetbrains.kotlinx.jupyter.api.declare +import kotlin.reflect.typeOf + +/** + * %use spark-streaming + */ +@Suppress("UNUSED_VARIABLE", "LocalVariableName") +internal class SparkStreamingIntegration : Integration() { + + override val imports: Array = super.imports + arrayOf( + "org.apache.spark.deploy.SparkHadoopUtil", + "org.apache.hadoop.conf.Configuration", + ) + + private val sscCollection = mutableSetOf() + + override fun KotlinKernelHost.onLoaded() { + + declare( + VariableDeclaration( + name = ::sscCollection.name, + value = sscCollection, + isMutable = false, + type = typeOf>(), + ) + ) + + val _0 = execute("""%dumpClassesForSpark""") + + @Language("kts") + val _1 = listOf( + """ + @JvmOverloads + 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, + startStreamingContext: Boolean = true, + func: KSparkStreamingSession.() -> Unit, + ) { + + // will only be set when a new context is created + var kSparkStreamingSession: KSparkStreamingSession? = null + + val 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 + } + + val ssc = when { + checkpointPath != null -> + JavaStreamingContext.getOrCreate(checkpointPath, creatingFunc, hadoopConf, createOnError) + + else -> creatingFunc() + } + sscCollection += ssc + + if (startStreamingContext) { + ssc.start() + kSparkStreamingSession?.invokeRunAfterStart() + } + ssc.awaitTerminationOrTimeout(timeout) + ssc.stop() + } + """.trimIndent(), + """ + println("To start a spark streaming session, simply use `withSparkStreaming { }` inside a cell. To use Spark normally, use `withSpark { }` in a cell, or use `%use spark` to start a Spark session for the whole notebook.")""".trimIndent(), + ).map(::execute) + } + + private fun cleanUp(e: Throwable): String { + while (sscCollection.isNotEmpty()) + sscCollection.first().let { + while (it.state != StreamingContextState.STOPPED) { + try { + it.stop(true, true) + } catch (_: Exception) { + } + } + sscCollection.remove(it) + } + + return "Spark streams cleaned up. Cause: $e" + } + + override fun Builder.onLoadedAlsoDo() { + renderThrowable { + cleanUp(it) + } + } + + override fun KotlinKernelHost.onInterrupt() { + println( + cleanUp(InterruptedException("Kernel was interrupted.")) + ) + } +} diff --git a/jupyter/src/main/resources/META-INF/kotlin-jupyter-libraries/libraries.json b/jupyter/src/main/resources/META-INF/kotlin-jupyter-libraries/libraries.json new file mode 100644 index 00000000..82c7354e --- /dev/null +++ b/jupyter/src/main/resources/META-INF/kotlin-jupyter-libraries/libraries.json @@ -0,0 +1,11 @@ +{ + "definitions": [], + "producers": [ + { + "fqn": "org.jetbrains.kotlinx.spark.api.jupyter.SparkIntegration" + }, + { + "fqn": "org.jetbrains.kotlinx.spark.api.jupyter.SparkStreamingIntegration" + } + ] +} diff --git a/jupyter/src/main/resources/table.css b/jupyter/src/main/resources/table.css new file mode 100644 index 00000000..f656add9 --- /dev/null +++ b/jupyter/src/main/resources/table.css @@ -0,0 +1,146 @@ +:root { + --background: #fff; + --background-odd: #f5f5f5; + --background-hover: #d9edfd; + --header-text-color: #474747; + --text-color: #848484; + --text-color-dark: #000; + --text-color-medium: #737373; + --text-color-pale: #b3b3b3; + --inner-border-color: #aaa; + --bold-border-color: #000; + --link-color: #296eaa; + --link-color-pale: #296eaa; + --link-hover: #1a466c; +} + +:root[theme="dark"], :root [data-jp-theme-light="false"]{ + --background: #303030; + --background-odd: #3c3c3c; + --background-hover: #464646; + --header-text-color: #dddddd; + --text-color: #b3b3b3; + --text-color-dark: #dddddd; + --text-color-medium: #b2b2b2; + --text-color-pale: #737373; + --inner-border-color: #707070; + --bold-border-color: #777777; + --link-color: #008dc0; + --link-color-pale: #97e1fb; + --link-hover: #00688e; +} + +table.dataset { + font-family: "Helvetica Neue", Helvetica, Arial, sans-serif; + font-size: 12px; + background-color: var(--background); + color: var(--text-color); + border: none; + border-collapse: collapse; +} + +table.dataset th, td { + padding: 6px; + border: 1px solid transparent; + text-align: left; +} + +table.dataset th { + background-color: var(--background); + color: var(--header-text-color); +} + +table.dataset td { + vertical-align: top; +} + +table.dataset th.bottomBorder { + border-bottom-color: var(--bold-border-color); +} + +table.dataset tbody > tr:nth-child(odd) { + background: var(--background-odd); +} + +table.dataset tbody > tr:nth-child(even) { + background: var(--background); +} + +table.dataset tbody > tr:hover { + background: var(--background-hover); +} + +table.dataset a { + cursor: pointer; + color: var(--link-color); + text-decoration: none; +} + +table.dataset tr:hover > td a { + color: var(--link-color-pale); +} + +table.dataset a:hover { + color: var(--link-hover); + text-decoration: underline; +} + +table.dataset img { + max-width: fit-content; +} + +table.dataset th.complex { + background-color: var(--background); + border: 1px solid var(--background); +} + +table.dataset .leftBorder { + border-left-color: var(--inner-border-color); +} + +table.dataset .rightBorder { + border-right-color: var(--inner-border-color); +} + +table.dataset .rightAlign { + text-align: right; +} + +table.dataset .expanderSvg { + width: 8px; + height: 8px; + margin-right: 3px; +} + +table.dataset .expander { + display: flex; + align-items: center; +} + +/* formatting */ + +table.dataset .null { + color: var(--text-color-pale); +} + +table.dataset .structural { + color: var(--text-color-medium); + font-weight: bold; +} + +table.dataset .datasetCaption { + font-weight: bold; +} + +table.dataset .numbers { + color: var(--text-color-dark); +} + +table.dataset td:hover .formatted .structural, .null { + color: var(--text-color-dark); +} + +table.dataset tr:hover .formatted .structural, .null { + color: var(--text-color-dark); +} + diff --git a/jupyter/src/test/kotlin/org/jetbrains/kotlinx/spark/api/jupyter/JupyterTests.kt b/jupyter/src/test/kotlin/org/jetbrains/kotlinx/spark/api/jupyter/JupyterTests.kt new file mode 100644 index 00000000..b4b750ee --- /dev/null +++ b/jupyter/src/test/kotlin/org/jetbrains/kotlinx/spark/api/jupyter/JupyterTests.kt @@ -0,0 +1,358 @@ +/*- + * =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.jupyter + +import io.kotest.assertions.throwables.shouldThrowAny +import io.kotest.core.spec.style.ShouldSpec +import io.kotest.matchers.nulls.shouldNotBeNull +import io.kotest.matchers.shouldBe +import io.kotest.matchers.shouldNotBe +import io.kotest.matchers.string.shouldContain +import io.kotest.matchers.types.shouldBeInstanceOf +import jupyter.kotlin.DependsOn +import org.apache.spark.api.java.JavaSparkContext +import org.apache.spark.streaming.api.java.JavaStreamingContext +import org.intellij.lang.annotations.Language +import org.jetbrains.kotlinx.jupyter.EvalRequestData +import org.jetbrains.kotlinx.jupyter.ReplForJupyter +import org.jetbrains.kotlinx.jupyter.ReplForJupyterImpl +import org.jetbrains.kotlinx.jupyter.api.Code +import org.jetbrains.kotlinx.jupyter.api.MimeTypedResult +import org.jetbrains.kotlinx.jupyter.libraries.EmptyResolutionInfoProvider +import org.jetbrains.kotlinx.jupyter.repl.EvalResultEx +import org.jetbrains.kotlinx.jupyter.testkit.ReplProvider +import org.jetbrains.kotlinx.jupyter.util.PatternNameAcceptanceRule +import org.jetbrains.kotlinx.spark.api.SparkSession +import java.io.Serializable +import kotlin.script.experimental.jvm.util.classpathFromClassloader + +class JupyterTests : ShouldSpec({ + val replProvider = ReplProvider { classpath -> + ReplForJupyterImpl( + resolutionInfoProvider = EmptyResolutionInfoProvider, + scriptClasspath = classpath, + isEmbedded = true, + ).apply { + eval { + librariesScanner.addLibrariesFromClassLoader( + classLoader = currentClassLoader, + host = this, + integrationTypeNameRules = listOf( + PatternNameAcceptanceRule(false, "org.jetbrains.kotlinx.spark.api.jupyter.**"), + PatternNameAcceptanceRule(true, "org.jetbrains.kotlinx.spark.api.jupyter.SparkIntegration"), + ), + ) + } + } + } + + val currentClassLoader = DependsOn::class.java.classLoader + val scriptClasspath = classpathFromClassloader(currentClassLoader).orEmpty() + + fun createRepl(): ReplForJupyter = replProvider(scriptClasspath) + suspend fun withRepl(action: suspend ReplForJupyter.() -> Unit): Unit = createRepl().action() + + context("Jupyter") { + withRepl { + + should("Allow functions on local data classes") { + @Language("kts") + val klass = exec("""data class Test(val a: Int, val b: String)""") + + @Language("kts") + val ds = exec("""val ds = dsOf(Test(1, "hi"), Test(2, "something"))""") + + @Language("kts") + val filtered = exec("""val filtered = ds.filter { it.a > 1 }""") + + @Language("kts") + val filteredShow = exec("""filtered.show()""") + } + + should("Have spark instance") { + @Language("kts") + val spark = exec("""spark""") + spark as? SparkSession shouldNotBe null + } + + should("Have JavaSparkContext instance") { + @Language("kts") + val sc = exec("""sc""") + sc as? JavaSparkContext shouldNotBe null + } + + should("render Datasets") { + @Language("kts") + val html = execHtml( + """ + val ds = listOf(1, 2, 3).toDS() + ds + """.trimIndent() + ) + println(html) + + html shouldContain "value" + html shouldContain "1" + html shouldContain "2" + html shouldContain "3" + } + + should("render JavaRDDs") { + @Language("kts") + val html = execHtml( + """ + val rdd: JavaRDD> = sc.parallelize(listOf( + listOf(1, 2, 3), + listOf(4, 5, 6), + )) + rdd + """.trimIndent() + ) + println(html) + + html shouldContain "1, 2, 3" + html shouldContain "4, 5, 6" + } + + should("render JavaRDDs with Arrays") { + @Language("kts") + val html = execHtml( + """ + val rdd: JavaRDD = sc.parallelize(listOf( + intArrayOf(1, 2, 3), + intArrayOf(4, 5, 6), + )) + rdd + """.trimIndent() + ) + println(html) + + html shouldContain "1, 2, 3" + html shouldContain "4, 5, 6" + } + + should("render JavaRDDs with custom class") { + + @Language("kts") + val klass = exec( + """ + data class Test( + val longFirstName: String, + val second: LongArray, + val somethingSpecial: Map, + ): Serializable + """.trimIndent() + ) + + @Language("kts") + val html = execHtml( + """ + val rdd = sc.parallelize( + listOf( + Test("aaaaaaaaa", longArrayOf(1L, 100000L, 24L), mapOf(1 to "one", 2 to "two")), + Test("aaaaaaaaa", longArrayOf(1L, 100000L, 24L), mapOf(1 to "one", 2 to "two")), + ) + ) + rdd + """.trimIndent() + ) + html shouldContain "Test(longFirstName=aaaaaaaa..." + } + + should("render JavaPairRDDs") { + @Language("kts") + val html = execHtml( + """ + val rdd: JavaPairRDD = sc.parallelizePairs(listOf( + c(1, 2).toTuple(), + c(3, 4).toTuple(), + )) + rdd + """.trimIndent() + ) + println(html) + + html shouldContain "1, 2" + html shouldContain "3, 4" + } + + should("render JavaDoubleRDD") { + @Language("kts") + val html = execHtml( + """ + val rdd: JavaDoubleRDD = sc.parallelizeDoubles(listOf(1.0, 2.0, 3.0, 4.0,)) + rdd + """.trimIndent() + ) + println(html) + + html shouldContain "1.0" + html shouldContain "2.0" + html shouldContain "3.0" + html shouldContain "4.0" + } + + should("render Scala RDD") { + @Language("kts") + val html = execHtml( + """ + val rdd: RDD> = sc.parallelize(listOf( + listOf(1, 2, 3), + listOf(4, 5, 6), + )).rdd() + rdd + """.trimIndent() + ) + println(html) + + html shouldContain "1, 2, 3" + html shouldContain "4, 5, 6" + } + + @Language("kts") + val _stop = exec("""spark.stop()""") + } + } +}) + +class JupyterStreamingTests : ShouldSpec({ + val replProvider = ReplProvider { classpath -> + ReplForJupyterImpl( + resolutionInfoProvider = EmptyResolutionInfoProvider, + scriptClasspath = classpath, + isEmbedded = true, + ).apply { + eval { + librariesScanner.addLibrariesFromClassLoader( + classLoader = currentClassLoader, + host = this, + integrationTypeNameRules = listOf( + PatternNameAcceptanceRule(false, "org.jetbrains.kotlinx.spark.api.jupyter.**"), + PatternNameAcceptanceRule( + true, + "org.jetbrains.kotlinx.spark.api.jupyter.SparkStreamingIntegration" + ), + ), + ) + } + } + } + + val currentClassLoader = DependsOn::class.java.classLoader + val scriptClasspath = classpathFromClassloader(currentClassLoader).orEmpty() + + fun createRepl(): ReplForJupyter = replProvider(scriptClasspath) + suspend fun withRepl(action: suspend ReplForJupyter.() -> Unit): Unit = createRepl().action() + + context("Jupyter") { + withRepl { + + // For when onInterrupt is implemented in the Jupyter kernel + should("Have sscCollection instance") { + + @Language("kts") + val sscCollection = exec("""sscCollection""") + sscCollection as? MutableSet shouldNotBe null + } + + should("Not have spark instance") { + shouldThrowAny { + @Language("kts") + val spark = exec("""spark""") + Unit + } + } + + should("Not have sc instance") { + shouldThrowAny { + @Language("kts") + val sc = exec("""sc""") + Unit + } + } + + xshould("stream") { + + @Language("kts") + val value = exec( + """ + import java.util.LinkedList + import org.apache.spark.api.java.function.ForeachFunction + import org.apache.spark.util.LongAccumulator + + + val input = arrayListOf("aaa", "bbb", "aaa", "ccc") + + @Volatile + var counter: LongAccumulator? = null + + withSparkStreaming(Duration(10), timeout = 1_000) { + + val queue = withSpark(ssc) { + LinkedList(listOf(sc.parallelize(input))) + } + + val inputStream = ssc.queueStream(queue) + + inputStream.foreachRDD { rdd, _ -> + withSpark(rdd) { + if (counter == null) + counter = sc.sc().longAccumulator() + + rdd.toDS().showDS().forEach { + if (it !in input) error(it + " should be in input") + counter!!.add(1L) + } + } + } + } + counter!!.sum() + """.trimIndent() + ) as Long + + value shouldBe 4L + } + + } + } +}) + + +private fun ReplForJupyter.execEx(code: Code): EvalResultEx = evalEx(EvalRequestData(code)) + +private fun ReplForJupyter.exec(code: Code): Any? = execEx(code).renderedValue + +private fun ReplForJupyter.execRaw(code: Code): Any? = execEx(code).rawValue + +@JvmName("execTyped") +private inline fun ReplForJupyter.exec(code: Code): T { + val res = exec(code) + res.shouldBeInstanceOf() + return res +} + +private fun ReplForJupyter.execHtml(code: Code): String { + val res = exec(code) + val html = res["text/html"] + html.shouldNotBeNull() + return html +} + +class Counter(@Volatile var value: Int) : Serializable diff --git a/kotlin-spark-api/3.0/pom_2.12.xml b/kotlin-spark-api/3.0/pom_2.12.xml new file mode 100644 index 00000000..2b24557f --- /dev/null +++ b/kotlin-spark-api/3.0/pom_2.12.xml @@ -0,0 +1,179 @@ + + + + 4.0.0 + + Kotlin Spark API: API for Spark 3.0+ (Scala 2.12) + kotlin-spark-api-3.0 + Kotlin API compatible with spark 3.0.3 Kotlin for Apache Spark + + org.jetbrains.kotlinx.spark + kotlin-spark-api-parent-3.0_2.12 + 1.1.0 + ../../pom_2.12.xml + + jar + + + + + org.jetbrains.kotlin + kotlin-stdlib-jdk8 + + + org.jetbrains.kotlin + kotlin-reflect + + + org.jetbrains.kotlinx.spark + core-3.0_${scala.compat.version} + + + org.jetbrains.kotlinx.spark + scala-tuples-in-kotlin + + + + + org.apache.spark + spark-sql_${scala.compat.version} + ${spark3.version} + provided + + + org.apache.spark + spark-streaming_${scala.compat.version} + ${spark3.version} + provided + + + org.apache.hadoop + hadoop-client + ${hadoop.version} + provided + + + + + org.apache.spark + spark-streaming-kafka-0-10_${scala.compat.version} + ${spark3.version} + test + + + io.kotest + kotest-runner-junit5-jvm + ${kotest.version} + test + + + io.kotest.extensions + kotest-extensions-testcontainers + ${kotest-extensions-testcontainers.version} + test + + + io.github.embeddedkafka + embedded-kafka_${scala.compat.version} + ${embedded-kafka.version} + test + + + com.beust + klaxon + ${klaxon.version} + test + + + ch.tutteli.atrium + atrium-fluent-en_GB + ${atrium.version} + test + + + org.apache.spark + spark-streaming_${scala.compat.version} + ${spark3.version} + tests + test + + + org.apache.kafka + kafka-streams-test-utils + 3.1.0 + test + + + + + src/main/kotlin + src/test/kotlin + target/${scala.compat.version} + + + + org.jetbrains.kotlin + kotlin-maven-plugin + + + compile + + compile + + + + test-compile + + test-compile + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + + org.jetbrains.dokka + dokka-maven-plugin + ${dokka.version} + + 8 + + + + dokka + + dokka + + pre-site + + + javadocjar + + javadocJar + + pre-integration-test + + + + + + org.jacoco + jacoco-maven-plugin + + + org.apache.maven.plugins + maven-compiler-plugin + ${maven-compiler-plugin.version} + + 8 + 8 + + + + + + diff --git a/kotlin-spark-api/common/src/main/kotlin/org/jetbrains/kotlinx/spark/api/VarArities.kt b/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Arities.kt similarity index 82% rename from kotlin-spark-api/common/src/main/kotlin/org/jetbrains/kotlinx/spark/api/VarArities.kt rename to kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Arities.kt index a4b2bdd7..93371fee 100644 --- a/kotlin-spark-api/common/src/main/kotlin/org/jetbrains/kotlinx/spark/api/VarArities.kt +++ b/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Arities.kt @@ -17,385 +17,838 @@ * limitations under the License. * =LICENSEEND= */ + /** - * Helper classes and functions to work with unnamed tuples + * Helper classes and functions to work with unnamed tuples we call Arities. + * Arities are easier to work with in Kotlin than Scala Tuples since they are Kotlin data classes. + * This means they can be destructured, copied, etc. + * Finally, the Arities are Serializable, meaning they can be used inside RDDs and they can be broadcast. + * + * Example: + * ```kotlin + * // creation + * val tuple: Arity3 = c(1, "test", 1.0) + * + * // addition + * val newTuple: Arity5 = tuple + c(1, 2) + * + * // destructuring + * val dataset: Dataset> = ... + * dataset.map { (a: Int, b: Double) -> + * (a + b).toString() + * } + * ``` */ + package org.jetbrains.kotlinx.spark.api -data class Arity1(val _1: T1) -data class Arity2(val _1: T1, val _2: T2) -data class Arity3(val _1: T1, val _2: T2, val _3: T3) -data class Arity4(val _1: T1, val _2: T2, val _3: T3, val _4: T4) -data class Arity5(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5) -data class Arity6(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6) -data class Arity7(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7) -data class Arity8(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8) -data class Arity9(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9) -data class Arity10(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9, val _10: T10) -data class Arity11(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9, val _10: T10, val _11: T11) -data class Arity12(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9, val _10: T10, val _11: T11, val _12: T12) -data class Arity13(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9, val _10: T10, val _11: T11, val _12: T12, val _13: T13) -data class Arity14(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9, val _10: T10, val _11: T11, val _12: T12, val _13: T13, val _14: T14) -data class Arity15(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9, val _10: T10, val _11: T11, val _12: T12, val _13: T13, val _14: T14, val _15: T15) -data class Arity16(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9, val _10: T10, val _11: T11, val _12: T12, val _13: T13, val _14: T14, val _15: T15, val _16: T16) -data class Arity17(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9, val _10: T10, val _11: T11, val _12: T12, val _13: T13, val _14: T14, val _15: T15, val _16: T16, val _17: T17) -data class Arity18(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9, val _10: T10, val _11: T11, val _12: T12, val _13: T13, val _14: T14, val _15: T15, val _16: T16, val _17: T17, val _18: T18) -data class Arity19(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9, val _10: T10, val _11: T11, val _12: T12, val _13: T13, val _14: T14, val _15: T15, val _16: T16, val _17: T17, val _18: T18, val _19: T19) -data class Arity20(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9, val _10: T10, val _11: T11, val _12: T12, val _13: T13, val _14: T14, val _15: T15, val _16: T16, val _17: T17, val _18: T18, val _19: T19, val _20: T20) -data class Arity21(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9, val _10: T10, val _11: T11, val _12: T12, val _13: T13, val _14: T14, val _15: T15, val _16: T16, val _17: T17, val _18: T18, val _19: T19, val _20: T20, val _21: T21) -data class Arity22(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9, val _10: T10, val _11: T11, val _12: T12, val _13: T13, val _14: T14, val _15: T15, val _16: T16, val _17: T17, val _18: T18, val _19: T19, val _20: T20, val _21: T21, val _22: T22) -data class Arity23(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9, val _10: T10, val _11: T11, val _12: T12, val _13: T13, val _14: T14, val _15: T15, val _16: T16, val _17: T17, val _18: T18, val _19: T19, val _20: T20, val _21: T21, val _22: T22, val _23: T23) -data class Arity24(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9, val _10: T10, val _11: T11, val _12: T12, val _13: T13, val _14: T14, val _15: T15, val _16: T16, val _17: T17, val _18: T18, val _19: T19, val _20: T20, val _21: T21, val _22: T22, val _23: T23, val _24: T24) -data class Arity25(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9, val _10: T10, val _11: T11, val _12: T12, val _13: T13, val _14: T14, val _15: T15, val _16: T16, val _17: T17, val _18: T18, val _19: T19, val _20: T20, val _21: T21, val _22: T22, val _23: T23, val _24: T24, val _25: T25) -data class Arity26(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9, val _10: T10, val _11: T11, val _12: T12, val _13: T13, val _14: T14, val _15: T15, val _16: T16, val _17: T17, val _18: T18, val _19: T19, val _20: T20, val _21: T21, val _22: T22, val _23: T23, val _24: T24, val _25: T25, val _26: T26) -fun c(_1: T1) = Arity1(_1) -fun c(_1: T1, _2: T2) = Arity2(_1, _2) -fun c(_1: T1, _2: T2, _3: T3) = Arity3(_1, _2, _3) -fun c(_1: T1, _2: T2, _3: T3, _4: T4) = Arity4(_1, _2, _3, _4) -fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5) = Arity5(_1, _2, _3, _4, _5) -fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6) = Arity6(_1, _2, _3, _4, _5, _6) -fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7) = Arity7(_1, _2, _3, _4, _5, _6, _7) -fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8) = Arity8(_1, _2, _3, _4, _5, _6, _7, _8) -fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9) = Arity9(_1, _2, _3, _4, _5, _6, _7, _8, _9) -fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10) = Arity10(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10) -fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11) = Arity11(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11) -fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12) = Arity12(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12) -fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13) = Arity13(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13) -fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14) = Arity14(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14) -fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15) = Arity15(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15) -fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16) = Arity16(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16) -fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16, _17: T17) = Arity17(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17) -fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16, _17: T17, _18: T18) = Arity18(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18) -fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16, _17: T17, _18: T18, _19: T19) = Arity19(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19) -fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16, _17: T17, _18: T18, _19: T19, _20: T20) = Arity20(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19, _20) -fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16, _17: T17, _18: T18, _19: T19, _20: T20, _21: T21) = Arity21(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19, _20, _21) -fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16, _17: T17, _18: T18, _19: T19, _20: T20, _21: T21, _22: T22) = Arity22(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19, _20, _21, _22) -fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16, _17: T17, _18: T18, _19: T19, _20: T20, _21: T21, _22: T22, _23: T23) = Arity23(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19, _20, _21, _22, _23) -fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16, _17: T17, _18: T18, _19: T19, _20: T20, _21: T21, _22: T22, _23: T23, _24: T24) = Arity24(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19, _20, _21, _22, _23, _24) -fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16, _17: T17, _18: T18, _19: T19, _20: T20, _21: T21, _22: T22, _23: T23, _24: T24, _25: T25) = Arity25(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19, _20, _21, _22, _23, _24, _25) -fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16, _17: T17, _18: T18, _19: T19, _20: T20, _21: T21, _22: T22, _23: T23, _24: T24, _25: T25, _26: T26) = Arity26(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19, _20, _21, _22, _23, _24, _25, _26) +import java.io.Serializable + +@Deprecated("Use Scala tuples instead.", ReplaceWith("Tuple1(_1)", "scala.Tuple1")) +data class Arity1(val _1: T1): Serializable + +@Deprecated("Use Scala tuples instead.", ReplaceWith("Tuple2(_1, _2)", "scala.Tuple2")) +data class Arity2(val _1: T1, val _2: T2): Serializable + +@Deprecated("Use Scala tuples instead.", ReplaceWith("Tuple3(_1, _2, _3)", "scala.Tuple3")) +data class Arity3(val _1: T1, val _2: T2, val _3: T3): Serializable + +@Deprecated("Use Scala tuples instead.", ReplaceWith("Tuple4(_1, _2, _3, _4)", "scala.Tuple4")) +data class Arity4(val _1: T1, val _2: T2, val _3: T3, val _4: T4): Serializable + +@Deprecated("Use Scala tuples instead.", ReplaceWith("Tuple5(_1, _2, _3, _4, _5)", "scala.Tuple5")) +data class Arity5(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5): Serializable + +@Deprecated("Use Scala tuples instead.", ReplaceWith("Tuple6(_1, _2, _3, _4, _5, _6)", "scala.Tuple6")) +data class Arity6(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6): Serializable + +@Deprecated("Use Scala tuples instead.", ReplaceWith("Tuple7(_1, _2, _3, _4, _5, _6, _7)", "scala.Tuple7")) +data class Arity7(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7): Serializable + +@Deprecated("Use Scala tuples instead.", ReplaceWith("Tuple8(_1, _2, _3, _4, _5, _6, _7, _8)", "scala.Tuple8")) +data class Arity8(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8): Serializable + +@Deprecated("Use Scala tuples instead.", ReplaceWith("Tuple9(_1, _2, _3, _4, _5, _6, _7, _8, _9)", "scala.Tuple9")) +data class Arity9(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9): Serializable + +@Deprecated("Use Scala tuples instead.", ReplaceWith("Tuple10(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10)", "scala.Tuple10")) +data class Arity10(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9, val _10: T10): Serializable + +@Deprecated("Use Scala tuples instead.", ReplaceWith("Tuple11(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11)", "scala.Tuple11")) +data class Arity11(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9, val _10: T10, val _11: T11): Serializable + +@Deprecated("Use Scala tuples instead.", ReplaceWith("Tuple12(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12)", "scala.Tuple12")) +data class Arity12(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9, val _10: T10, val _11: T11, val _12: T12): Serializable + +@Deprecated("Use Scala tuples instead.", ReplaceWith("Tuple13(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13)", "scala.Tuple13")) +data class Arity13(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9, val _10: T10, val _11: T11, val _12: T12, val _13: T13): Serializable + +@Deprecated("Use Scala tuples instead.", ReplaceWith("Tuple14(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14)", "scala.Tuple14")) +data class Arity14(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9, val _10: T10, val _11: T11, val _12: T12, val _13: T13, val _14: T14): Serializable + +@Deprecated("Use Scala tuples instead.", ReplaceWith("Tuple15(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15)", "scala.Tuple15")) +data class Arity15(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9, val _10: T10, val _11: T11, val _12: T12, val _13: T13, val _14: T14, val _15: T15): Serializable + +@Deprecated("Use Scala tuples instead.", ReplaceWith("Tuple16(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16)", "scala.Tuple16")) +data class Arity16(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9, val _10: T10, val _11: T11, val _12: T12, val _13: T13, val _14: T14, val _15: T15, val _16: T16): Serializable + +@Deprecated("Use Scala tuples instead.", ReplaceWith("Tuple17(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17)", "scala.Tuple17")) +data class Arity17(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9, val _10: T10, val _11: T11, val _12: T12, val _13: T13, val _14: T14, val _15: T15, val _16: T16, val _17: T17): Serializable + +@Deprecated("Use Scala tuples instead.", ReplaceWith("Tuple18(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18)", "scala.Tuple18")) +data class Arity18(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9, val _10: T10, val _11: T11, val _12: T12, val _13: T13, val _14: T14, val _15: T15, val _16: T16, val _17: T17, val _18: T18): Serializable + +@Deprecated("Use Scala tuples instead.", ReplaceWith("Tuple19(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19)", "scala.Tuple19")) +data class Arity19(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9, val _10: T10, val _11: T11, val _12: T12, val _13: T13, val _14: T14, val _15: T15, val _16: T16, val _17: T17, val _18: T18, val _19: T19): Serializable + +@Deprecated("Use Scala tuples instead.", ReplaceWith("Tuple20(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19, _20)", "scala.Tuple20")) +data class Arity20(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9, val _10: T10, val _11: T11, val _12: T12, val _13: T13, val _14: T14, val _15: T15, val _16: T16, val _17: T17, val _18: T18, val _19: T19, val _20: T20): Serializable + +@Deprecated("Use Scala tuples instead.", ReplaceWith("Tuple21(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19, _20, _21)", "scala.Tuple21")) +data class Arity21(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9, val _10: T10, val _11: T11, val _12: T12, val _13: T13, val _14: T14, val _15: T15, val _16: T16, val _17: T17, val _18: T18, val _19: T19, val _20: T20, val _21: T21): Serializable + +@Deprecated("Use Scala tuples instead.", ReplaceWith("Tuple22(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19, _20, _21, _22)", "scala.Tuple22")) +data class Arity22(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9, val _10: T10, val _11: T11, val _12: T12, val _13: T13, val _14: T14, val _15: T15, val _16: T16, val _17: T17, val _18: T18, val _19: T19, val _20: T20, val _21: T21, val _22: T22): Serializable + +@Deprecated("Use Scala tuples instead. They only reach 22 values.") +data class Arity23(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9, val _10: T10, val _11: T11, val _12: T12, val _13: T13, val _14: T14, val _15: T15, val _16: T16, val _17: T17, val _18: T18, val _19: T19, val _20: T20, val _21: T21, val _22: T22, val _23: T23): Serializable + +@Deprecated("Use Scala tuples instead. They only reach 22 values.") +data class Arity24(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9, val _10: T10, val _11: T11, val _12: T12, val _13: T13, val _14: T14, val _15: T15, val _16: T16, val _17: T17, val _18: T18, val _19: T19, val _20: T20, val _21: T21, val _22: T22, val _23: T23, val _24: T24): Serializable + +@Deprecated("Use Scala tuples instead. They only reach 22 values.") +data class Arity25(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9, val _10: T10, val _11: T11, val _12: T12, val _13: T13, val _14: T14, val _15: T15, val _16: T16, val _17: T17, val _18: T18, val _19: T19, val _20: T20, val _21: T21, val _22: T22, val _23: T23, val _24: T24, val _25: T25): Serializable + +@Deprecated("Use Scala tuples instead. They only reach 22 values.") +data class Arity26(val _1: T1, val _2: T2, val _3: T3, val _4: T4, val _5: T5, val _6: T6, val _7: T7, val _8: T8, val _9: T9, val _10: T10, val _11: T11, val _12: T12, val _13: T13, val _14: T14, val _15: T15, val _16: T16, val _17: T17, val _18: T18, val _19: T19, val _20: T20, val _21: T21, val _22: T22, val _23: T23, val _24: T24, val _25: T25, val _26: T26): Serializable + + +@Deprecated("Use Scala tuples instead.", ReplaceWith("t(_1)")) +fun c(_1: T1): Arity1 = Arity1(_1) + +@Deprecated("Use Scala tuples instead.", ReplaceWith("t(_1, _2)")) +fun c(_1: T1, _2: T2): Arity2 = Arity2(_1, _2) + +@Deprecated("Use Scala tuples instead.", ReplaceWith("t(_1, _2, _3)")) +fun c(_1: T1, _2: T2, _3: T3): Arity3 = Arity3(_1, _2, _3) + +@Deprecated("Use Scala tuples instead.", ReplaceWith("t(_1, _2, _3, _4)")) +fun c(_1: T1, _2: T2, _3: T3, _4: T4): Arity4 = Arity4(_1, _2, _3, _4) + +@Deprecated("Use Scala tuples instead.", ReplaceWith("t(_1, _2, _3, _4, _5)")) +fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5): Arity5 = Arity5(_1, _2, _3, _4, _5) + +@Deprecated("Use Scala tuples instead.", ReplaceWith("t(_1, _2, _3, _4, _5, _6)")) +fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6): Arity6 = Arity6(_1, _2, _3, _4, _5, _6) + +@Deprecated("Use Scala tuples instead.", ReplaceWith("t(_1, _2, _3, _4, _5, _6, _7)")) +fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7): Arity7 = Arity7(_1, _2, _3, _4, _5, _6, _7) + +@Deprecated("Use Scala tuples instead.", ReplaceWith("t(_1, _2, _3, _4, _5, _6, _7, _8)")) +fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8): Arity8 = Arity8(_1, _2, _3, _4, _5, _6, _7, _8) + +@Deprecated("Use Scala tuples instead.", ReplaceWith("t(_1, _2, _3, _4, _5, _6, _7, _8, _9)")) +fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9): Arity9 = Arity9(_1, _2, _3, _4, _5, _6, _7, _8, _9) + +@Deprecated("Use Scala tuples instead.", ReplaceWith("t(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10)")) +fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10): Arity10 = Arity10(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10) + +@Deprecated("Use Scala tuples instead.", ReplaceWith("t(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11)")) +fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11): Arity11 = Arity11(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11) + +@Deprecated("Use Scala tuples instead.", ReplaceWith("t(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12)")) +fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12): Arity12 = Arity12(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12) + +@Deprecated("Use Scala tuples instead.", ReplaceWith("t(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13)")) +fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13): Arity13 = Arity13(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13) + +@Deprecated("Use Scala tuples instead.", ReplaceWith("t(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14)")) +fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14): Arity14 = Arity14(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14) + +@Deprecated("Use Scala tuples instead.", ReplaceWith("t(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15)")) +fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15): Arity15 = Arity15(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15) + +@Deprecated("Use Scala tuples instead.", ReplaceWith("t(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16)")) +fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16): Arity16 = Arity16(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16) + +@Deprecated("Use Scala tuples instead.", ReplaceWith("t(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17)")) +fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16, _17: T17): Arity17 = Arity17(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17) + +@Deprecated("Use Scala tuples instead.", ReplaceWith("t(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18)")) +fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16, _17: T17, _18: T18): Arity18 = Arity18(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18) + +@Deprecated("Use Scala tuples instead.", ReplaceWith("t(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19)")) +fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16, _17: T17, _18: T18, _19: T19): Arity19 = Arity19(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19) + +@Deprecated("Use Scala tuples instead.", ReplaceWith("t(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19, _20)")) +fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16, _17: T17, _18: T18, _19: T19, _20: T20): Arity20 = Arity20(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19, _20) + +@Deprecated("Use Scala tuples instead.", ReplaceWith("t(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19, _20, _21)")) +fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16, _17: T17, _18: T18, _19: T19, _20: T20, _21: T21): Arity21 = Arity21(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19, _20, _21) + +@Deprecated("Use Scala tuples instead.", ReplaceWith("t(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19, _20, _21, _22)")) +fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16, _17: T17, _18: T18, _19: T19, _20: T20, _21: T21, _22: T22): Arity22 = Arity22(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19, _20, _21, _22) + +@Deprecated("Use Scala tuples instead. They only reach 22 values.") +fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16, _17: T17, _18: T18, _19: T19, _20: T20, _21: T21, _22: T22, _23: T23): Arity23 = Arity23(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19, _20, _21, _22, _23) + +@Deprecated("Use Scala tuples instead. They only reach 22 values.") +fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16, _17: T17, _18: T18, _19: T19, _20: T20, _21: T21, _22: T22, _23: T23, _24: T24): Arity24 = Arity24(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19, _20, _21, _22, _23, _24) + +@Deprecated("Use Scala tuples instead. They only reach 22 values.") +fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16, _17: T17, _18: T18, _19: T19, _20: T20, _21: T21, _22: T22, _23: T23, _24: T24, _25: T25): Arity25 = Arity25(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19, _20, _21, _22, _23, _24, _25) + +@Deprecated("Use Scala tuples instead. They only reach 22 values.") +fun c(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16, _17: T17, _18: T18, _19: T19, _20: T20, _21: T21, _22: T22, _23: T23, _24: T24, _25: T25, _26: T26): Arity26 = Arity26(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19, _20, _21, _22, _23, _24, _25, _26) + + +@Deprecated("Use Scala tuples instead.") infix operator fun Arity1.plus(that: Arity1) = Arity2(this._1, that._1) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity1.plus(that: Arity2) = Arity3(this._1, that._1, that._2) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity2.plus(that: Arity1) = Arity3(this._1, this._2, that._1) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity1.plus(that: Arity3) = Arity4(this._1, that._1, that._2, that._3) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity2.plus(that: Arity2) = Arity4(this._1, this._2, that._1, that._2) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity3.plus(that: Arity1) = Arity4(this._1, this._2, this._3, that._1) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity1.plus(that: Arity4) = Arity5(this._1, that._1, that._2, that._3, that._4) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity2.plus(that: Arity3) = Arity5(this._1, this._2, that._1, that._2, that._3) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity3.plus(that: Arity2) = Arity5(this._1, this._2, this._3, that._1, that._2) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity4.plus(that: Arity1) = Arity5(this._1, this._2, this._3, this._4, that._1) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity1.plus(that: Arity5) = Arity6(this._1, that._1, that._2, that._3, that._4, that._5) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity2.plus(that: Arity4) = Arity6(this._1, this._2, that._1, that._2, that._3, that._4) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity3.plus(that: Arity3) = Arity6(this._1, this._2, this._3, that._1, that._2, that._3) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity4.plus(that: Arity2) = Arity6(this._1, this._2, this._3, this._4, that._1, that._2) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity5.plus(that: Arity1) = Arity6(this._1, this._2, this._3, this._4, this._5, that._1) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity1.plus(that: Arity6) = Arity7(this._1, that._1, that._2, that._3, that._4, that._5, that._6) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity2.plus(that: Arity5) = Arity7(this._1, this._2, that._1, that._2, that._3, that._4, that._5) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity3.plus(that: Arity4) = Arity7(this._1, this._2, this._3, that._1, that._2, that._3, that._4) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity4.plus(that: Arity3) = Arity7(this._1, this._2, this._3, this._4, that._1, that._2, that._3) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity5.plus(that: Arity2) = Arity7(this._1, this._2, this._3, this._4, this._5, that._1, that._2) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity6.plus(that: Arity1) = Arity7(this._1, this._2, this._3, this._4, this._5, this._6, that._1) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity1.plus(that: Arity7) = Arity8(this._1, that._1, that._2, that._3, that._4, that._5, that._6, that._7) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity2.plus(that: Arity6) = Arity8(this._1, this._2, that._1, that._2, that._3, that._4, that._5, that._6) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity3.plus(that: Arity5) = Arity8(this._1, this._2, this._3, that._1, that._2, that._3, that._4, that._5) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity4.plus(that: Arity4) = Arity8(this._1, this._2, this._3, this._4, that._1, that._2, that._3, that._4) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity5.plus(that: Arity3) = Arity8(this._1, this._2, this._3, this._4, this._5, that._1, that._2, that._3) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity6.plus(that: Arity2) = Arity8(this._1, this._2, this._3, this._4, this._5, this._6, that._1, that._2) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity7.plus(that: Arity1) = Arity8(this._1, this._2, this._3, this._4, this._5, this._6, this._7, that._1) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity1.plus(that: Arity8) = Arity9(this._1, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity2.plus(that: Arity7) = Arity9(this._1, this._2, that._1, that._2, that._3, that._4, that._5, that._6, that._7) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity3.plus(that: Arity6) = Arity9(this._1, this._2, this._3, that._1, that._2, that._3, that._4, that._5, that._6) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity4.plus(that: Arity5) = Arity9(this._1, this._2, this._3, this._4, that._1, that._2, that._3, that._4, that._5) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity5.plus(that: Arity4) = Arity9(this._1, this._2, this._3, this._4, this._5, that._1, that._2, that._3, that._4) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity6.plus(that: Arity3) = Arity9(this._1, this._2, this._3, this._4, this._5, this._6, that._1, that._2, that._3) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity7.plus(that: Arity2) = Arity9(this._1, this._2, this._3, this._4, this._5, this._6, this._7, that._1, that._2) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity8.plus(that: Arity1) = Arity9(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, that._1) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity1.plus(that: Arity9) = Arity10(this._1, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity2.plus(that: Arity8) = Arity10(this._1, this._2, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity3.plus(that: Arity7) = Arity10(this._1, this._2, this._3, that._1, that._2, that._3, that._4, that._5, that._6, that._7) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity4.plus(that: Arity6) = Arity10(this._1, this._2, this._3, this._4, that._1, that._2, that._3, that._4, that._5, that._6) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity5.plus(that: Arity5) = Arity10(this._1, this._2, this._3, this._4, this._5, that._1, that._2, that._3, that._4, that._5) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity6.plus(that: Arity4) = Arity10(this._1, this._2, this._3, this._4, this._5, this._6, that._1, that._2, that._3, that._4) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity7.plus(that: Arity3) = Arity10(this._1, this._2, this._3, this._4, this._5, this._6, this._7, that._1, that._2, that._3) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity8.plus(that: Arity2) = Arity10(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, that._1, that._2) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity9.plus(that: Arity1) = Arity10(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, that._1) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity1.plus(that: Arity10) = Arity11(this._1, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity2.plus(that: Arity9) = Arity11(this._1, this._2, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity3.plus(that: Arity8) = Arity11(this._1, this._2, this._3, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity4.plus(that: Arity7) = Arity11(this._1, this._2, this._3, this._4, that._1, that._2, that._3, that._4, that._5, that._6, that._7) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity5.plus(that: Arity6) = Arity11(this._1, this._2, this._3, this._4, this._5, that._1, that._2, that._3, that._4, that._5, that._6) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity6.plus(that: Arity5) = Arity11(this._1, this._2, this._3, this._4, this._5, this._6, that._1, that._2, that._3, that._4, that._5) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity7.plus(that: Arity4) = Arity11(this._1, this._2, this._3, this._4, this._5, this._6, this._7, that._1, that._2, that._3, that._4) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity8.plus(that: Arity3) = Arity11(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, that._1, that._2, that._3) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity9.plus(that: Arity2) = Arity11(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, that._1, that._2) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity10.plus(that: Arity1) = Arity11(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, that._1) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity1.plus(that: Arity11) = Arity12(this._1, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity2.plus(that: Arity10) = Arity12(this._1, this._2, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity3.plus(that: Arity9) = Arity12(this._1, this._2, this._3, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity4.plus(that: Arity8) = Arity12(this._1, this._2, this._3, this._4, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity5.plus(that: Arity7) = Arity12(this._1, this._2, this._3, this._4, this._5, that._1, that._2, that._3, that._4, that._5, that._6, that._7) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity6.plus(that: Arity6) = Arity12(this._1, this._2, this._3, this._4, this._5, this._6, that._1, that._2, that._3, that._4, that._5, that._6) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity7.plus(that: Arity5) = Arity12(this._1, this._2, this._3, this._4, this._5, this._6, this._7, that._1, that._2, that._3, that._4, that._5) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity8.plus(that: Arity4) = Arity12(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, that._1, that._2, that._3, that._4) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity9.plus(that: Arity3) = Arity12(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, that._1, that._2, that._3) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity10.plus(that: Arity2) = Arity12(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, that._1, that._2) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity11.plus(that: Arity1) = Arity12(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, that._1) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity1.plus(that: Arity12) = Arity13(this._1, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity2.plus(that: Arity11) = Arity13(this._1, this._2, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity3.plus(that: Arity10) = Arity13(this._1, this._2, this._3, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity4.plus(that: Arity9) = Arity13(this._1, this._2, this._3, this._4, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity5.plus(that: Arity8) = Arity13(this._1, this._2, this._3, this._4, this._5, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity6.plus(that: Arity7) = Arity13(this._1, this._2, this._3, this._4, this._5, this._6, that._1, that._2, that._3, that._4, that._5, that._6, that._7) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity7.plus(that: Arity6) = Arity13(this._1, this._2, this._3, this._4, this._5, this._6, this._7, that._1, that._2, that._3, that._4, that._5, that._6) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity8.plus(that: Arity5) = Arity13(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, that._1, that._2, that._3, that._4, that._5) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity9.plus(that: Arity4) = Arity13(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, that._1, that._2, that._3, that._4) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity10.plus(that: Arity3) = Arity13(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, that._1, that._2, that._3) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity11.plus(that: Arity2) = Arity13(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, that._1, that._2) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity12.plus(that: Arity1) = 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, that._1) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity1.plus(that: Arity13) = Arity14(this._1, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity2.plus(that: Arity12) = Arity14(this._1, this._2, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity3.plus(that: Arity11) = Arity14(this._1, this._2, this._3, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity4.plus(that: Arity10) = Arity14(this._1, this._2, this._3, this._4, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity5.plus(that: Arity9) = Arity14(this._1, this._2, this._3, this._4, this._5, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity6.plus(that: Arity8) = Arity14(this._1, this._2, this._3, this._4, this._5, this._6, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity7.plus(that: Arity7) = Arity14(this._1, this._2, this._3, this._4, this._5, this._6, this._7, that._1, that._2, that._3, that._4, that._5, that._6, that._7) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity8.plus(that: Arity6) = Arity14(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, that._1, that._2, that._3, that._4, that._5, that._6) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity9.plus(that: Arity5) = Arity14(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, that._1, that._2, that._3, that._4, that._5) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity10.plus(that: Arity4) = Arity14(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, that._1, that._2, that._3, that._4) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity11.plus(that: Arity3) = Arity14(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, that._1, that._2, that._3) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity12.plus(that: Arity2) = 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, that._1, that._2) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity13.plus(that: Arity1) = 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, that._1) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity1.plus(that: Arity14) = Arity15(this._1, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity2.plus(that: Arity13) = Arity15(this._1, this._2, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity3.plus(that: Arity12) = Arity15(this._1, this._2, this._3, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity4.plus(that: Arity11) = Arity15(this._1, this._2, this._3, this._4, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity5.plus(that: Arity10) = Arity15(this._1, this._2, this._3, this._4, this._5, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity6.plus(that: Arity9) = Arity15(this._1, this._2, this._3, this._4, this._5, this._6, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity7.plus(that: Arity8) = Arity15(this._1, this._2, this._3, this._4, this._5, this._6, this._7, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity8.plus(that: Arity7) = Arity15(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, that._1, that._2, that._3, that._4, that._5, that._6, that._7) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity9.plus(that: Arity6) = Arity15(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, that._1, that._2, that._3, that._4, that._5, that._6) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity10.plus(that: Arity5) = Arity15(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, that._1, that._2, that._3, that._4, that._5) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity11.plus(that: Arity4) = Arity15(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, that._1, that._2, that._3, that._4) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity12.plus(that: Arity3) = 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, that._1, that._2, that._3) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity13.plus(that: Arity2) = 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, that._1, that._2) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity14.plus(that: Arity1) = 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, that._1) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity1.plus(that: Arity15) = Arity16(this._1, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity2.plus(that: Arity14) = Arity16(this._1, this._2, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity3.plus(that: Arity13) = Arity16(this._1, this._2, this._3, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity4.plus(that: Arity12) = Arity16(this._1, this._2, this._3, this._4, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity5.plus(that: Arity11) = Arity16(this._1, this._2, this._3, this._4, this._5, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity6.plus(that: Arity10) = Arity16(this._1, this._2, this._3, this._4, this._5, this._6, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity7.plus(that: Arity9) = Arity16(this._1, this._2, this._3, this._4, this._5, this._6, this._7, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity8.plus(that: Arity8) = Arity16(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity9.plus(that: Arity7) = Arity16(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, that._1, that._2, that._3, that._4, that._5, that._6, that._7) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity10.plus(that: Arity6) = Arity16(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, that._1, that._2, that._3, that._4, that._5, that._6) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity11.plus(that: Arity5) = Arity16(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, that._1, that._2, that._3, that._4, that._5) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity12.plus(that: Arity4) = 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, that._1, that._2, that._3, that._4) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity13.plus(that: Arity3) = 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, that._1, that._2, that._3) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity14.plus(that: Arity2) = 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, that._1, that._2) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity15.plus(that: Arity1) = 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, that._1) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity1.plus(that: Arity16) = Arity17(this._1, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity2.plus(that: Arity15) = Arity17(this._1, this._2, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity3.plus(that: Arity14) = Arity17(this._1, this._2, this._3, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity4.plus(that: Arity13) = Arity17(this._1, this._2, this._3, this._4, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity5.plus(that: Arity12) = Arity17(this._1, this._2, this._3, this._4, this._5, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity6.plus(that: Arity11) = Arity17(this._1, this._2, this._3, this._4, this._5, this._6, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity7.plus(that: Arity10) = Arity17(this._1, this._2, this._3, this._4, this._5, this._6, this._7, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity8.plus(that: Arity9) = Arity17(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity9.plus(that: Arity8) = Arity17(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity10.plus(that: Arity7) = Arity17(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, that._1, that._2, that._3, that._4, that._5, that._6, that._7) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity11.plus(that: Arity6) = Arity17(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, that._1, that._2, that._3, that._4, that._5, that._6) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity12.plus(that: Arity5) = 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, that._1, that._2, that._3, that._4, that._5) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity13.plus(that: Arity4) = 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, that._1, that._2, that._3, that._4) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity14.plus(that: Arity3) = 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, that._1, that._2, that._3) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity15.plus(that: Arity2) = 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, that._1, that._2) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity16.plus(that: Arity1) = 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, that._1) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity1.plus(that: Arity17) = Arity18(this._1, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity2.plus(that: Arity16) = Arity18(this._1, this._2, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity3.plus(that: Arity15) = Arity18(this._1, this._2, this._3, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity4.plus(that: Arity14) = Arity18(this._1, this._2, this._3, this._4, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity5.plus(that: Arity13) = Arity18(this._1, this._2, this._3, this._4, this._5, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity6.plus(that: Arity12) = Arity18(this._1, this._2, this._3, this._4, this._5, this._6, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity7.plus(that: Arity11) = Arity18(this._1, this._2, this._3, this._4, this._5, this._6, this._7, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity8.plus(that: Arity10) = Arity18(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity9.plus(that: Arity9) = Arity18(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity10.plus(that: Arity8) = Arity18(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity11.plus(that: Arity7) = Arity18(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, that._1, that._2, that._3, that._4, that._5, that._6, that._7) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity12.plus(that: Arity6) = 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, that._1, that._2, that._3, that._4, that._5, that._6) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity13.plus(that: Arity5) = 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, that._1, that._2, that._3, that._4, that._5) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity14.plus(that: Arity4) = 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, that._1, that._2, that._3, that._4) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity15.plus(that: Arity3) = 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, that._1, that._2, that._3) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity16.plus(that: Arity2) = 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, that._1, that._2) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity17.plus(that: Arity1) = 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, that._1) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity1.plus(that: Arity18) = Arity19(this._1, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity2.plus(that: Arity17) = Arity19(this._1, this._2, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity3.plus(that: Arity16) = Arity19(this._1, this._2, this._3, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity4.plus(that: Arity15) = Arity19(this._1, this._2, this._3, this._4, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity5.plus(that: Arity14) = Arity19(this._1, this._2, this._3, this._4, this._5, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity6.plus(that: Arity13) = Arity19(this._1, this._2, this._3, this._4, this._5, this._6, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity7.plus(that: Arity12) = Arity19(this._1, this._2, this._3, this._4, this._5, this._6, this._7, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity8.plus(that: Arity11) = Arity19(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity9.plus(that: Arity10) = Arity19(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity10.plus(that: Arity9) = Arity19(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity11.plus(that: Arity8) = Arity19(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity12.plus(that: Arity7) = 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, that._1, that._2, that._3, that._4, that._5, that._6, that._7) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity13.plus(that: Arity6) = 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, that._1, that._2, that._3, that._4, that._5, that._6) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity14.plus(that: Arity5) = 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, that._1, that._2, that._3, that._4, that._5) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity15.plus(that: Arity4) = 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, that._1, that._2, that._3, that._4) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity16.plus(that: Arity3) = 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, that._1, that._2, that._3) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity17.plus(that: Arity2) = 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, that._1, that._2) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity18.plus(that: Arity1) = 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, that._1) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity1.plus(that: Arity19) = Arity20(this._1, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18, that._19) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity2.plus(that: Arity18) = Arity20(this._1, this._2, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity3.plus(that: Arity17) = Arity20(this._1, this._2, this._3, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity4.plus(that: Arity16) = Arity20(this._1, this._2, this._3, this._4, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity5.plus(that: Arity15) = Arity20(this._1, this._2, this._3, this._4, this._5, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity6.plus(that: Arity14) = Arity20(this._1, this._2, this._3, this._4, this._5, this._6, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity7.plus(that: Arity13) = Arity20(this._1, this._2, this._3, this._4, this._5, this._6, this._7, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity8.plus(that: Arity12) = Arity20(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity9.plus(that: Arity11) = Arity20(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity10.plus(that: Arity10) = Arity20(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity11.plus(that: Arity9) = Arity20(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity12.plus(that: Arity8) = 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, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity13.plus(that: Arity7) = 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, that._1, that._2, that._3, that._4, that._5, that._6, that._7) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity14.plus(that: Arity6) = 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, that._1, that._2, that._3, that._4, that._5, that._6) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity15.plus(that: Arity5) = 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, that._1, that._2, that._3, that._4, that._5) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity16.plus(that: Arity4) = 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, that._1, that._2, that._3, that._4) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity17.plus(that: Arity3) = 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, that._1, that._2, that._3) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity18.plus(that: Arity2) = 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, that._1, that._2) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity19.plus(that: Arity1) = 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, that._1) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity1.plus(that: Arity20) = Arity21(this._1, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18, that._19, that._20) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity2.plus(that: Arity19) = Arity21(this._1, this._2, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18, that._19) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity3.plus(that: Arity18) = Arity21(this._1, this._2, this._3, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity4.plus(that: Arity17) = Arity21(this._1, this._2, this._3, this._4, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity5.plus(that: Arity16) = Arity21(this._1, this._2, this._3, this._4, this._5, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity6.plus(that: Arity15) = Arity21(this._1, this._2, this._3, this._4, this._5, this._6, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity7.plus(that: Arity14) = Arity21(this._1, this._2, this._3, this._4, this._5, this._6, this._7, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity8.plus(that: Arity13) = Arity21(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity9.plus(that: Arity12) = Arity21(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity10.plus(that: Arity11) = Arity21(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity11.plus(that: Arity10) = Arity21(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity12.plus(that: Arity9) = 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, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity13.plus(that: Arity8) = 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, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity14.plus(that: Arity7) = 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, that._1, that._2, that._3, that._4, that._5, that._6, that._7) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity15.plus(that: Arity6) = 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, that._1, that._2, that._3, that._4, that._5, that._6) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity16.plus(that: Arity5) = 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, that._1, that._2, that._3, that._4, that._5) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity17.plus(that: Arity4) = 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, that._1, that._2, that._3, that._4) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity18.plus(that: Arity3) = 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, that._1, that._2, that._3) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity19.plus(that: Arity2) = 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, that._1, that._2) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity20.plus(that: Arity1) = 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, that._1) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity1.plus(that: Arity21) = Arity22(this._1, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18, that._19, that._20, that._21) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity2.plus(that: Arity20) = Arity22(this._1, this._2, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18, that._19, that._20) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity3.plus(that: Arity19) = Arity22(this._1, this._2, this._3, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18, that._19) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity4.plus(that: Arity18) = Arity22(this._1, this._2, this._3, this._4, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity5.plus(that: Arity17) = Arity22(this._1, this._2, this._3, this._4, this._5, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity6.plus(that: Arity16) = Arity22(this._1, this._2, this._3, this._4, this._5, this._6, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity7.plus(that: Arity15) = Arity22(this._1, this._2, this._3, this._4, this._5, this._6, this._7, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity8.plus(that: Arity14) = Arity22(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity9.plus(that: Arity13) = Arity22(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity10.plus(that: Arity12) = Arity22(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity11.plus(that: Arity11) = Arity22(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity12.plus(that: Arity10) = 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, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity13.plus(that: Arity9) = 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, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity14.plus(that: Arity8) = 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, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity15.plus(that: Arity7) = 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, that._1, that._2, that._3, that._4, that._5, that._6, that._7) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity16.plus(that: Arity6) = 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, that._1, that._2, that._3, that._4, that._5, that._6) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity17.plus(that: Arity5) = 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, that._1, that._2, that._3, that._4, that._5) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity18.plus(that: Arity4) = 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, that._1, that._2, that._3, that._4) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity19.plus(that: Arity3) = 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, that._1, that._2, that._3) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity20.plus(that: Arity2) = 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, that._1, that._2) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity21.plus(that: Arity1) = 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, that._1) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity1.plus(that: Arity22) = Arity23(this._1, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18, that._19, that._20, that._21, that._22) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity2.plus(that: Arity21) = Arity23(this._1, this._2, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18, that._19, that._20, that._21) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity3.plus(that: Arity20) = Arity23(this._1, this._2, this._3, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18, that._19, that._20) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity4.plus(that: Arity19) = Arity23(this._1, this._2, this._3, this._4, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18, that._19) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity5.plus(that: Arity18) = Arity23(this._1, this._2, this._3, this._4, this._5, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity6.plus(that: Arity17) = Arity23(this._1, this._2, this._3, this._4, this._5, this._6, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity7.plus(that: Arity16) = Arity23(this._1, this._2, this._3, this._4, this._5, this._6, this._7, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity8.plus(that: Arity15) = Arity23(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity9.plus(that: Arity14) = Arity23(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity10.plus(that: Arity13) = Arity23(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity11.plus(that: Arity12) = Arity23(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity12.plus(that: Arity11) = Arity23(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, this._12, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity13.plus(that: Arity10) = Arity23(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, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity14.plus(that: Arity9) = Arity23(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, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity15.plus(that: Arity8) = Arity23(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, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity16.plus(that: Arity7) = Arity23(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, that._1, that._2, that._3, that._4, that._5, that._6, that._7) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity17.plus(that: Arity6) = Arity23(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, that._1, that._2, that._3, that._4, that._5, that._6) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity18.plus(that: Arity5) = Arity23(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, that._1, that._2, that._3, that._4, that._5) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity19.plus(that: Arity4) = Arity23(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, that._1, that._2, that._3, that._4) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity20.plus(that: Arity3) = Arity23(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, that._1, that._2, that._3) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity21.plus(that: Arity2) = Arity23(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, that._1, that._2) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity22.plus(that: Arity1) = Arity23(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, that._1) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity1.plus(that: Arity23) = Arity24(this._1, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18, that._19, that._20, that._21, that._22, that._23) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity2.plus(that: Arity22) = Arity24(this._1, this._2, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18, that._19, that._20, that._21, that._22) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity3.plus(that: Arity21) = Arity24(this._1, this._2, this._3, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18, that._19, that._20, that._21) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity4.plus(that: Arity20) = Arity24(this._1, this._2, this._3, this._4, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18, that._19, that._20) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity5.plus(that: Arity19) = Arity24(this._1, this._2, this._3, this._4, this._5, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18, that._19) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity6.plus(that: Arity18) = Arity24(this._1, this._2, this._3, this._4, this._5, this._6, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity7.plus(that: Arity17) = Arity24(this._1, this._2, this._3, this._4, this._5, this._6, this._7, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity8.plus(that: Arity16) = Arity24(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity9.plus(that: Arity15) = Arity24(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity10.plus(that: Arity14) = Arity24(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity11.plus(that: Arity13) = Arity24(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity12.plus(that: Arity12) = Arity24(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, this._12, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity13.plus(that: Arity11) = Arity24(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, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity14.plus(that: Arity10) = Arity24(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, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity15.plus(that: Arity9) = Arity24(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, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity16.plus(that: Arity8) = Arity24(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, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity17.plus(that: Arity7) = Arity24(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, that._1, that._2, that._3, that._4, that._5, that._6, that._7) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity18.plus(that: Arity6) = Arity24(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, that._1, that._2, that._3, that._4, that._5, that._6) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity19.plus(that: Arity5) = Arity24(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, that._1, that._2, that._3, that._4, that._5) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity20.plus(that: Arity4) = Arity24(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, that._1, that._2, that._3, that._4) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity21.plus(that: Arity3) = Arity24(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, that._1, that._2, that._3) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity22.plus(that: Arity2) = Arity24(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, that._1, that._2) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity23.plus(that: Arity1) = Arity24(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, this._23, that._1) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity1.plus(that: Arity24) = Arity25(this._1, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18, that._19, that._20, that._21, that._22, that._23, that._24) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity2.plus(that: Arity23) = Arity25(this._1, this._2, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18, that._19, that._20, that._21, that._22, that._23) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity3.plus(that: Arity22) = Arity25(this._1, this._2, this._3, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18, that._19, that._20, that._21, that._22) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity4.plus(that: Arity21) = Arity25(this._1, this._2, this._3, this._4, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18, that._19, that._20, that._21) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity5.plus(that: Arity20) = Arity25(this._1, this._2, this._3, this._4, this._5, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18, that._19, that._20) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity6.plus(that: Arity19) = Arity25(this._1, this._2, this._3, this._4, this._5, this._6, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18, that._19) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity7.plus(that: Arity18) = Arity25(this._1, this._2, this._3, this._4, this._5, this._6, this._7, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity8.plus(that: Arity17) = Arity25(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity9.plus(that: Arity16) = Arity25(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity10.plus(that: Arity15) = Arity25(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity11.plus(that: Arity14) = Arity25(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity12.plus(that: Arity13) = Arity25(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, this._12, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity13.plus(that: Arity12) = Arity25(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, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity14.plus(that: Arity11) = Arity25(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, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity15.plus(that: Arity10) = Arity25(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, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity16.plus(that: Arity9) = Arity25(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, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity17.plus(that: Arity8) = Arity25(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, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity18.plus(that: Arity7) = Arity25(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, that._1, that._2, that._3, that._4, that._5, that._6, that._7) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity19.plus(that: Arity6) = Arity25(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, that._1, that._2, that._3, that._4, that._5, that._6) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity20.plus(that: Arity5) = Arity25(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, that._1, that._2, that._3, that._4, that._5) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity21.plus(that: Arity4) = Arity25(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, that._1, that._2, that._3, that._4) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity22.plus(that: Arity3) = Arity25(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, that._1, that._2, that._3) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity23.plus(that: Arity2) = Arity25(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, this._23, that._1, that._2) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity24.plus(that: Arity1) = Arity25(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, this._23, this._24, that._1) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity1.plus(that: Arity25) = Arity26(this._1, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18, that._19, that._20, that._21, that._22, that._23, that._24, that._25) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity2.plus(that: Arity24) = Arity26(this._1, this._2, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18, that._19, that._20, that._21, that._22, that._23, that._24) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity3.plus(that: Arity23) = Arity26(this._1, this._2, this._3, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18, that._19, that._20, that._21, that._22, that._23) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity4.plus(that: Arity22) = Arity26(this._1, this._2, this._3, this._4, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18, that._19, that._20, that._21, that._22) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity5.plus(that: Arity21) = Arity26(this._1, this._2, this._3, this._4, this._5, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18, that._19, that._20, that._21) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity6.plus(that: Arity20) = Arity26(this._1, this._2, this._3, this._4, this._5, this._6, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18, that._19, that._20) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity7.plus(that: Arity19) = Arity26(this._1, this._2, this._3, this._4, this._5, this._6, this._7, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18, that._19) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity8.plus(that: Arity18) = Arity26(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17, that._18) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity9.plus(that: Arity17) = Arity26(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16, that._17) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity10.plus(that: Arity16) = Arity26(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15, that._16) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity11.plus(that: Arity15) = Arity26(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14, that._15) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity12.plus(that: Arity14) = Arity26(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, this._12, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13, that._14) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity13.plus(that: Arity13) = Arity26(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, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12, that._13) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity14.plus(that: Arity12) = Arity26(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, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11, that._12) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity15.plus(that: Arity11) = Arity26(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, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10, that._11) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity16.plus(that: Arity10) = Arity26(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, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9, that._10) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity17.plus(that: Arity9) = Arity26(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, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8, that._9) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity18.plus(that: Arity8) = Arity26(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, that._1, that._2, that._3, that._4, that._5, that._6, that._7, that._8) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity19.plus(that: Arity7) = Arity26(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, that._1, that._2, that._3, that._4, that._5, that._6, that._7) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity20.plus(that: Arity6) = Arity26(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, that._1, that._2, that._3, that._4, that._5, that._6) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity21.plus(that: Arity5) = Arity26(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, that._1, that._2, that._3, that._4, that._5) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity22.plus(that: Arity4) = Arity26(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, that._1, that._2, that._3, that._4) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity23.plus(that: Arity3) = Arity26(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, this._23, that._1, that._2, that._3) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity24.plus(that: Arity2) = Arity26(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, this._23, this._24, that._1, that._2) +@Deprecated("Use Scala tuples instead.") infix operator fun Arity25.plus(that: Arity1) = Arity26(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, this._23, this._24, this._25, that._1) diff --git a/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Column.kt b/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Column.kt new file mode 100644 index 00000000..72e8a9b7 --- /dev/null +++ b/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Column.kt @@ -0,0 +1,432 @@ +/*- + * =LICENSE= + * Kotlin Spark API: API for Spark 3.0+ (Scala 2.12) + * ---------- + * Copyright (C) 2019 - 2021 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= + */ + +/** + * This file contains all Column helper functions. + * This includes easier Column creation and operator functions. + */ + +package org.jetbrains.kotlinx.spark.api + +import org.apache.spark.sql.Column +import org.apache.spark.sql.Dataset +import org.apache.spark.sql.TypedColumn +import org.apache.spark.sql.functions +import kotlin.reflect.KProperty1 + +/** + * Selects column based on the column name and returns it as a [Column]. + * + * @note The column name can also reference to a nested column like `a.b`. + */ +operator fun Dataset.invoke(colName: String): Column = col(colName) + +/** + * Helper function to quickly get a [TypedColumn] (or [Column]) from a dataset in a refactor-safe manner. + * ```kotlin + * val dataset: Dataset = ... + * val columnA: TypedColumn = dataset.col(YourClass::a) + * ``` + * @see invoke + */ +@Suppress("UNCHECKED_CAST") +inline fun Dataset.col(column: KProperty1): TypedColumn = + col(column.name).`as`() as TypedColumn + + +/** + * Helper function to quickly get a [TypedColumn] (or [Column]) from a dataset in a refactor-safe manner. + * ```kotlin + * val dataset: Dataset = ... + * val columnA: TypedColumn = dataset(YourClass::a) + * ``` + * @see col + */ +inline operator fun Dataset.invoke(column: KProperty1): TypedColumn = col(column) + + +@Suppress("FunctionName") +@Deprecated( + message = "Changed to \"`===`\" to better reflect Scala API.", + replaceWith = ReplaceWith("this `===` c"), + level = DeprecationLevel.ERROR, +) +infix fun Column.`==`(c: Column): Column = `$eq$eq$eq`(c) + +/** + * Unary minus, i.e. negate the expression. + * ``` + * // Scala: select the amount column and negates all values. + * df.select( -df("amount") ) + * + * // Kotlin: + * import org.jetbrains.kotlinx.spark.api.* + * df.select( -df("amount") ) + * + * // Java: + * import static org.apache.spark.sql.functions.*; + * df.select( negate(col("amount") ); + * ``` + */ +operator fun Column.unaryMinus(): Column = `unary_$minus`() + +/** + * Inversion of boolean expression, i.e. NOT. + * ``` + * // Scala: select rows that are not active (isActive === false) + * df.filter( !df("isActive") ) + * + * // Kotlin: + * import org.jetbrains.kotlinx.spark.api.* + * df.filter( !df("amount") ) + * + * // Java: + * import static org.apache.spark.sql.functions.*; + * df.filter( not(df.col("isActive")) ); + * ``` + */ +operator fun Column.not(): Column = `unary_$bang`() + +/** + * Equality test. + * ``` + * // Scala: + * df.filter( df("colA") === df("colB") ) + * + * // Kotlin: + * import org.jetbrains.kotlinx.spark.api.* + * df.filter( df("colA") eq df("colB") ) + * // or + * df.filter( df("colA") `===` df("colB") ) + * + * // Java + * import static org.apache.spark.sql.functions.*; + * df.filter( col("colA").equalTo(col("colB")) ); + * ``` + */ +infix fun Column.eq(other: Any): Column = `$eq$eq$eq`(other) + +/** + * Equality test. + * ``` + * // Scala: + * df.filter( df("colA") === df("colB") ) + * + * // Kotlin: + * import org.jetbrains.kotlinx.spark.api.* + * df.filter( df("colA") eq df("colB") ) + * // or + * df.filter( df("colA") `===` df("colB") ) + * + * // Java + * import static org.apache.spark.sql.functions.*; + * df.filter( col("colA").equalTo(col("colB")) ); + * ``` + */ +infix fun Column.`===`(other: Any): Column = `$eq$eq$eq`(other) + +/** + * Inequality test. + * ``` + * // Scala: + * df.select( df("colA") =!= df("colB") ) + * df.select( !(df("colA") === df("colB")) ) + * + * // Kotlin: + * import org.jetbrains.kotlinx.spark.api.* + * df.select( df("colA") neq df("colB") ) + * df.select( !(df("colA") eq df("colB")) ) + * // or + * df.select( df("colA") `=!=` df("colB") ) + * df.select( !(df("colA") `===` df("colB")) ) + * + * // Java: + * import static org.apache.spark.sql.functions.*; + * df.select( col("colA").notEqual(col("colB")) ); + * ``` + */ +infix fun Column.neq(other: Any): Column = `$eq$bang$eq`(other) + +/** + * Inequality test. + * ``` + * // Scala: + * df.select( df("colA") =!= df("colB") ) + * df.select( !(df("colA") === df("colB")) ) + * + * // Kotlin: + * import org.jetbrains.kotlinx.spark.api.* + * df.select( df("colA") neq df("colB") ) + * df.select( !(df("colA") eq df("colB")) ) + * // or + * df.select( df("colA") `=!=` df("colB") ) + * df.select( !(df("colA") `===` df("colB")) ) + * + * // Java: + * import static org.apache.spark.sql.functions.*; + * df.select( col("colA").notEqual(col("colB")) ); + * ``` + */ +infix fun Column.`=!=`(other: Any): Column = `$eq$bang$eq`(other) + +/** + * Greater than. + * ``` + * // Scala: The following selects people older than 21. + * people.select( people("age") > 21 ) + * + * // Kotlin: + * import org.jetbrains.kotlinx.spark.api.* + * people.select( people("age") gt 21 ) + * + * // Java: + * import static org.apache.spark.sql.functions.*; + * people.select( people.col("age").gt(21) ); + * ``` + */ +infix fun Column.gt(other: Any): Column = `$greater`(other) + +/** + * Less than. + * ``` + * // Scala: The following selects people younger than 21. + * people.select( people("age") < 21 ) + * + * // Kotlin: + * import org.jetbrains.kotlinx.spark.api.* + * people.select( people("age") lt 21 ) + * + * // Java: + * import static org.apache.spark.sql.functions.*; + * people.select( people.col("age").lt(21) ); + * ``` + */ +infix fun Column.lt(other: Any): Column = `$less`(other) + +/** + * Less than or equal to. + * ``` + * // Scala: The following selects people age 21 or younger than 21. + * people.select( people("age") <= 21 ) + * + * // Kotlin: + * import org.jetbrains.kotlinx.spark.api.* + * people.select( people("age") leq 21 ) + * + * // Java: + * import static org.apache.spark.sql.functions.*; + * people.select( people.col("age").leq(21) ); + * ``` + */ +infix fun Column.leq(other: Any): Column = `$less$eq`(other) + +/** + * Greater than or equal to an expression. + * ``` + * // Scala: The following selects people age 21 or older than 21. + * people.select( people("age") >= 21 ) + * + * // Kotlin: + * import org.jetbrains.kotlinx.spark.api.* + * people.select( people("age") geq 21 ) + * + * // Java: + * import static org.apache.spark.sql.functions.*; + * people.select( people.col("age").geq(21) ); + * ``` + */ +infix fun Column.geq(other: Any): Column = `$greater$eq`(other) + +/** + * True if the current column is in the given [range]. + * ``` + * // Scala: + * df.where( df("colA").between(1, 5) ) + * + * // Kotlin: + * import org.jetbrains.kotlinx.spark.api.* + * df.where( df("colA") inRangeOf 1..5 ) + * + * // Java: + * import static org.apache.spark.sql.functions.*; + * df.where( df.col("colA").between(1, 5) ); + * ``` + */ +infix fun Column.inRangeOf(range: ClosedRange<*>): Column = between(range.start, range.endInclusive) + +/** + * Boolean OR. + * ``` + * // Scala: The following selects people that are in school or employed. + * people.filter( people("inSchool") || people("isEmployed") ) + * + * // Kotlin: + * import org.jetbrains.kotlinx.spark.api.* + * people.filter( people("inSchool") or people("isEmployed") ) + * + * // Java: + * import static org.apache.spark.sql.functions.*; + * people.filter( people.col("inSchool").or(people.col("isEmployed")) ); + * ``` + */ +infix fun Column.or(other: Any): Column = `$bar$bar`(other) + +/** + * Boolean AND. + * ``` + * // Scala: The following selects people that are in school and employed at the same time. + * people.select( people("inSchool") && people("isEmployed") ) + * + * // Kotlin: + * import org.jetbrains.kotlinx.spark.api.* + * people.select( people("inSchool") and people("isEmployed") ) + * // or + * people.select( people("inSchool") `&&` people("isEmployed") ) + * + * // Java: + * import static org.apache.spark.sql.functions.*; + * people.select( people.col("inSchool").and(people.col("isEmployed")) ); + * ``` + */ +infix fun Column.and(other: Any): Column = `$amp$amp`(other) + +/** + * Boolean AND. + * ``` + * // Scala: The following selects people that are in school and employed at the same time. + * people.select( people("inSchool") && people("isEmployed") ) + * + * // Kotlin: + * import org.jetbrains.kotlinx.spark.api.* + * people.select( people("inSchool") and people("isEmployed") ) + * // or + * people.select( people("inSchool") `&&` people("isEmployed") ) + * + * // Java: + * import static org.apache.spark.sql.functions.*; + * people.select( people.col("inSchool").and(people.col("isEmployed")) ); + * ``` + */ +infix fun Column.`&&`(other: Any): Column = `$amp$amp`(other) + +/** + * Multiplication of this expression and another expression. + * ``` + * // Scala: The following multiplies a person's height by their weight. + * people.select( people("height") * people("weight") ) + * + * // Kotlin: + * import org.jetbrains.kotlinx.spark.api.* + * people.select( people("height") * people("weight") ) + * + * // Java: + * import static org.apache.spark.sql.functions.*; + * people.select( people.col("height").multiply(people.col("weight")) ); + * ``` + */ +operator fun Column.times(other: Any): Column = `$times`(other) + +/** + * Division this expression by another expression. + * ``` + * // Scala: The following divides a person's height by their weight. + * people.select( people("height") / people("weight") ) + * + * // Kotlin + * import org.jetbrains.kotlinx.spark.api.* + * people.select( people("height") / people("weight") ) + * + * // Java: + * import static org.apache.spark.sql.functions.*; + * people.select( people.col("height").divide(people.col("weight")) ); + * ``` + */ +operator fun Column.div(other: Any): Column = `$div`(other) + +/** + * Modulo (a.k.a. remainder) expression. + * ``` + * // Scala: + * df.where( df("colA") % 2 === 0 ) + * + * // Kotlin: + * import org.jetbrains.kotlinx.spark.api.* + * df.where( df("colA") % 2 eq 0 ) + * + * // Java: + * import static org.apache.spark.sql.functions.*; + * df.where( df.col("colA").mod(2).equalTo(0) ); + * ``` + */ +operator fun Column.rem(other: Any): Column = `$percent`(other) + +/** + * An expression that gets an item at position `ordinal` out of an array, + * or gets a value by key `key` in a `MapType`. + * ``` + * // Scala: + * df.where( df("arrayColumn").getItem(0) === 5 ) + * + * // Kotlin + * import org.jetbrains.kotlinx.spark.api.* + * df.where( df("arrayColumn")[0] eq 5 ) + * + * // Java + * import static org.apache.spark.sql.functions.*; + * df.where( df.col("arrayColumn").getItem(0).equalTo(5) ); + * ``` + */ +operator fun Column.get(key: Any): Column = getItem(key) + +/** + * 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()) + +/** + * Creates a [Column] of literal value. + * + * The passed in object is returned directly if it is already a [Column]. + * If the object is a Scala Symbol, it is converted into a [Column] also. + * Otherwise, a new [Column] is created to represent the literal value. + * + * This is just a shortcut to the function from [org.apache.spark.sql.functions]. + * For all the functions, simply add `import org.apache.spark.sql.functions.*` to your file. + */ +fun lit(a: Any): Column = functions.lit(a) + +/** + * 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) ) + * ``` + */ +@Suppress("UNCHECKED_CAST") +inline fun col(column: KProperty1): TypedColumn = + functions.col(column.name).`as`() as TypedColumn diff --git a/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt b/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt new file mode 100644 index 00000000..f8d90fa3 --- /dev/null +++ b/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt @@ -0,0 +1,491 @@ +/*- + * =LICENSE= + * Kotlin Spark API: API for Spark 3.0+ (Scala 2.12) + * ---------- + * Copyright (C) 2019 - 2021 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= + */ + +/** + * This files contains conversions of Iterators, Collections, Tuples, etc. between the Scala- + * and Kotlin/Java variants. + */ + +@file:Suppress("NOTHING_TO_INLINE", "RemoveExplicitTypeArguments", "unused") + +package org.jetbrains.kotlinx.spark.api + +import org.apache.spark.api.java.Optional +import scala.* +import scala.collection.JavaConverters +import java.util.* +import java.util.Enumeration +import java.util.concurrent.ConcurrentMap +import scala.collection.Iterable as ScalaIterable +import scala.collection.Iterator as ScalaIterator +import scala.collection.Map as ScalaMap +import scala.collection.Seq as ScalaSeq +import scala.collection.Set as ScalaSet +import scala.collection.concurrent.Map as ScalaConcurrentMap +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?.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?.toOptional(): Optional = Optional.ofNullable(this) + +/** Converts Java [Optional] to Scala [Option]. */ +fun Optional.toOption(): Option = Option.apply(getOrNull()) + +/** + * @see JavaConverters.asScalaIterator for more information. + */ +fun Iterator.asScalaIterator(): ScalaIterator = JavaConverters.asScalaIterator(this) + +/** + * @see JavaConverters.enumerationAsScalaIterator for more information. + */ +fun Enumeration.asScalaIterator(): ScalaIterator = JavaConverters.enumerationAsScalaIterator(this) + +/** + * @see JavaConverters.iterableAsScalaIterable for more information. + */ +fun Iterable.asScalaIterable(): ScalaIterable = JavaConverters.iterableAsScalaIterable(this) + +/** + * @see JavaConverters.collectionAsScalaIterable for more information. + */ +fun Collection.asScalaIterable(): ScalaIterable = JavaConverters.collectionAsScalaIterable(this) + +/** + * @see JavaConverters.asScalaBuffer for more information. + */ +fun MutableList.asScalaMutableBuffer(): ScalaMutableBuffer = JavaConverters.asScalaBuffer(this) + +/** + * @see JavaConverters.asScalaSet for more information. + */ +fun MutableSet.asScalaMutableSet(): ScalaMutableSet = JavaConverters.asScalaSet(this) + +/** + * @see JavaConverters.mapAsScalaMap for more information. + */ +fun MutableMap.asScalaMutableMap(): ScalaMutableMap = JavaConverters.mapAsScalaMap(this) + +/** + * @see JavaConverters.dictionaryAsScalaMap for more information. + */ +fun Map.asScalaMap(): ScalaMap = JavaConverters.mapAsScalaMap(this) + +/** + * @see JavaConverters.mapAsScalaConcurrentMap for more information. + */ +fun ConcurrentMap.asScalaConcurrentMap(): ScalaConcurrentMap = + JavaConverters.mapAsScalaConcurrentMap(this) + +/** + * @see JavaConverters.dictionaryAsScalaMap for more information. + */ +fun Dictionary.asScalaMap(): ScalaMutableMap = JavaConverters.dictionaryAsScalaMap(this) + +/** + * @see JavaConverters.propertiesAsScalaMap for more information. + */ +fun Properties.asScalaMap(): ScalaMutableMap = JavaConverters.propertiesAsScalaMap(this) + + +/** + * @see JavaConverters.asJavaIterator for more information. + */ +fun ScalaIterator.asKotlinIterator(): Iterator = JavaConverters.asJavaIterator(this) + +/** + * @see JavaConverters.asJavaEnumeration for more information. + */ +fun ScalaIterator.asKotlinEnumeration(): Enumeration = JavaConverters.asJavaEnumeration(this) + +/** + * @see JavaConverters.asJavaIterable for more information. + */ +fun ScalaIterable.asKotlinIterable(): Iterable = JavaConverters.asJavaIterable(this) + +/** + * @see JavaConverters.asJavaCollection for more information. + */ +fun ScalaIterable.asKotlinCollection(): Collection = JavaConverters.asJavaCollection(this) + +/** + * @see JavaConverters.bufferAsJavaList for more information. + */ +fun ScalaMutableBuffer.asKotlinMutableList(): MutableList = JavaConverters.bufferAsJavaList(this) + +/** + * @see JavaConverters.mutableSeqAsJavaList for more information. + */ +fun ScalaMutableSeq.asKotlinMutableList(): MutableList = JavaConverters.mutableSeqAsJavaList(this) + +/** + * @see JavaConverters.seqAsJavaList for more information. + */ +fun ScalaSeq.asKotlinList(): List = JavaConverters.seqAsJavaList(this) + +/** + * @see JavaConverters.mutableSetAsJavaSet for more information. + */ +fun ScalaMutableSet.asKotlinMutableSet(): MutableSet = JavaConverters.mutableSetAsJavaSet(this) + +/** + * @see JavaConverters.setAsJavaSet for more information. + */ +fun ScalaSet.asKotlinSet(): Set = JavaConverters.setAsJavaSet(this) + +/** + * @see JavaConverters.mutableMapAsJavaMap for more information. + */ +fun ScalaMutableMap.asKotlinMutableMap(): MutableMap = JavaConverters.mutableMapAsJavaMap(this) + +/** + * @see JavaConverters.asJavaDictionary for more information. + */ +fun ScalaMutableMap.asKotlinDictionary(): Dictionary = JavaConverters.asJavaDictionary(this) + +/** + * @see JavaConverters.mapAsJavaMap for more information. + */ +fun ScalaMap.asKotlinMap(): Map = JavaConverters.mapAsJavaMap(this) + +/** + * @see JavaConverters.mapAsJavaConcurrentMap for more information. + */ +fun ScalaConcurrentMap.asKotlinConcurrentMap(): ConcurrentMap = + JavaConverters.mapAsJavaConcurrentMap(this) + + +/** + * Returns a new [Arity2] based on the arguments in the current [Pair]. + */ +@Deprecated("Use Scala tuples instead.", ReplaceWith("this.toTuple()", "scala.Tuple2")) +fun Pair.toArity(): Arity2 = Arity2(first, second) + +/** + * Returns a new [Pair] based on the arguments in the current [Arity2]. + */ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Arity2.toPair(): Pair = Pair(_1, _2) + +/** + * Returns a new [Arity3] based on the arguments in the current [Triple]. + */ +@Deprecated("Use Scala tuples instead.", ReplaceWith("this.toTuple()", "scala.Tuple3")) +fun Triple.toArity(): Arity3 = Arity3(first, second, third) + +/** + * Returns a new [Triple] based on the arguments in the current [Arity3]. + */ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Arity3.toTriple(): Triple = Triple(_1, _2, _3) + + +/** + * Returns a new Arity1 based on this Tuple1. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Tuple1.toArity(): Arity1 = Arity1(this._1()) + +/** + * Returns a new Arity2 based on this Tuple2. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Tuple2.toArity(): Arity2 = Arity2(this._1(), this._2()) + +/** + * Returns a new Arity3 based on this Tuple3. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Tuple3.toArity(): Arity3 = Arity3(this._1(), this._2(), this._3()) + +/** + * Returns a new Arity4 based on this Tuple4. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Tuple4.toArity(): Arity4 = Arity4(this._1(), this._2(), this._3(), this._4()) + +/** + * Returns a new Arity5 based on this Tuple5. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Tuple5.toArity(): Arity5 = Arity5(this._1(), this._2(), this._3(), this._4(), this._5()) + +/** + * Returns a new Arity6 based on this Tuple6. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Tuple6.toArity(): Arity6 = Arity6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) + +/** + * Returns a new Arity7 based on this Tuple7. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Tuple7.toArity(): Arity7 = Arity7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) + +/** + * Returns a new Arity8 based on this Tuple8. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Tuple8.toArity(): Arity8 = Arity8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) + +/** + * Returns a new Arity9 based on this Tuple9. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Tuple9.toArity(): Arity9 = Arity9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) + +/** + * Returns a new Arity10 based on this Tuple10. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Tuple10.toArity(): Arity10 = Arity10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) + +/** + * Returns a new Arity11 based on this Tuple11. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Tuple11.toArity(): Arity11 = Arity11(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11()) + +/** + * Returns a new Arity12 based on this Tuple12. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Tuple12.toArity(): Arity12 = Arity12(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12()) + +/** + * Returns a new Arity13 based on this Tuple13. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Tuple13.toArity(): Arity13 = Arity13(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13()) + +/** + * Returns a new Arity14 based on this Tuple14. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Tuple14.toArity(): Arity14 = Arity14(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14()) + +/** + * Returns a new Arity15 based on this Tuple15. + **/@Deprecated("Use Scala tuples instead.", ReplaceWith("")) + +fun Tuple15.toArity(): Arity15 = Arity15(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15()) + +/** + * Returns a new Arity16 based on this Tuple16. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Tuple16.toArity(): Arity16 = Arity16(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16()) + +/** + * Returns a new Arity17 based on this Tuple17. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Tuple17.toArity(): Arity17 = Arity17(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17()) + +/** + * Returns a new Arity18 based on this Tuple18. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Tuple18.toArity(): Arity18 = Arity18(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18()) + +/** + * Returns a new Arity19 based on this Tuple19. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Tuple19.toArity(): Arity19 = Arity19(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19()) + +/** + * Returns a new Arity20 based on this Tuple20. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Tuple20.toArity(): Arity20 = Arity20(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20()) + +/** + * Returns a new Arity21 based on this Tuple21. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Tuple21.toArity(): Arity21 = Arity21(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20(), this._21()) + +/** + * Returns a new Arity22 based on this Tuple22. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Tuple22.toArity(): Arity22 = Arity22(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20(), this._21(), this._22()) + +/** + * Returns a new Tuple1 based on this Arity1. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Arity1.toTuple(): Tuple1 = Tuple1(this._1) + +/** + * Returns a new Tuple2 based on this Arity2. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Arity2.toTuple(): Tuple2 = Tuple2(this._1, this._2) + +/** + * Returns a new Tuple3 based on this Arity3. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Arity3.toTuple(): Tuple3 = Tuple3(this._1, this._2, this._3) + +/** + * Returns a new Tuple4 based on this Arity4. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Arity4.toTuple(): Tuple4 = Tuple4(this._1, this._2, this._3, this._4) + +/** + * Returns a new Tuple5 based on this Arity5. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Arity5.toTuple(): Tuple5 = Tuple5(this._1, this._2, this._3, this._4, this._5) + +/** + * Returns a new Tuple6 based on this Arity6. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Arity6.toTuple(): Tuple6 = Tuple6(this._1, this._2, this._3, this._4, this._5, this._6) + +/** + * Returns a new Tuple7 based on this Arity7. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Arity7.toTuple(): Tuple7 = Tuple7(this._1, this._2, this._3, this._4, this._5, this._6, this._7) + +/** + * Returns a new Tuple8 based on this Arity8. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Arity8.toTuple(): Tuple8 = Tuple8(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8) + +/** + * Returns a new Tuple9 based on this Arity9. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Arity9.toTuple(): Tuple9 = Tuple9(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9) + +/** + * Returns a new Tuple10 based on this Arity10. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Arity10.toTuple(): Tuple10 = Tuple10(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10) + +/** + * Returns a new Tuple11 based on this Arity11. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Arity11.toTuple(): Tuple11 = Tuple11(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11) + +/** + * Returns a new Tuple12 based on this Arity12. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Arity12.toTuple(): Tuple12 = Tuple12(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, this._12) + +/** + * Returns a new Tuple13 based on this Arity13. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Arity13.toTuple(): Tuple13 = Tuple13(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, this._12, this._13) + +/** + * Returns a new Tuple14 based on this Arity14. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Arity14.toTuple(): Tuple14 = Tuple14(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, this._12, this._13, this._14) + +/** + * Returns a new Tuple15 based on this Arity15. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Arity15.toTuple(): Tuple15 = Tuple15(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, this._12, this._13, this._14, this._15) + +/** + * Returns a new Tuple16 based on this Arity16. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Arity16.toTuple(): Tuple16 = Tuple16(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, this._12, this._13, this._14, this._15, this._16) + +/** + * Returns a new Tuple17 based on this Arity17. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Arity17.toTuple(): Tuple17 = Tuple17(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, this._12, this._13, this._14, this._15, this._16, this._17) + +/** + * Returns a new Tuple18 based on this Arity18. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Arity18.toTuple(): Tuple18 = Tuple18(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, this._12, this._13, this._14, this._15, this._16, this._17, this._18) + +/** + * Returns a new Tuple19 based on this Arity19. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Arity19.toTuple(): Tuple19 = Tuple19(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, this._12, this._13, this._14, this._15, this._16, this._17, this._18, this._19) + +/** + * Returns a new Tuple20 based on this Arity20. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Arity20.toTuple(): Tuple20 = Tuple20(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, this._12, this._13, this._14, this._15, this._16, this._17, this._18, this._19, this._20) + +/** + * Returns a new Tuple21 based on this Arity21. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Arity21.toTuple(): Tuple21 = Tuple21(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, this._12, this._13, this._14, this._15, this._16, this._17, this._18, this._19, this._20, this._21) + +/** + * Returns a new Tuple22 based on this Arity22. + **/ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +fun Arity22.toTuple(): Tuple22 = Tuple22(this._1, this._2, this._3, this._4, this._5, this._6, this._7, this._8, this._9, this._10, this._11, this._12, this._13, this._14, this._15, this._16, this._17, this._18, this._19, this._20, this._21, this._22) diff --git a/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/DataStreamWriter.kt b/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/DataStreamWriter.kt new file mode 100644 index 00000000..d0c1ece1 --- /dev/null +++ b/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/DataStreamWriter.kt @@ -0,0 +1,42 @@ +/*- + * =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 +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)) diff --git a/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Dataset.kt b/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Dataset.kt new file mode 100644 index 00000000..71abb1ee --- /dev/null +++ b/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Dataset.kt @@ -0,0 +1,475 @@ +/*- + * =LICENSE= + * Kotlin Spark API: API for Spark 3.0+ (Scala 2.12) + * ---------- + * Copyright (C) 2019 - 2021 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= + */ + +/** + * This file contains all Dataset helper functions. + * This includes the creation of Datasets from arrays, lists, and RDDs, + * as well as lots of extension functions which makes working with Datasets from Kotlin + * possible/easier. + */ + +@file:Suppress("unused") + +package org.jetbrains.kotlinx.spark.api + +import org.apache.spark.api.java.JavaRDDLike +import org.apache.spark.api.java.function.FlatMapFunction +import org.apache.spark.api.java.function.ForeachFunction +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.* +import org.jetbrains.kotlinx.spark.extensions.KSparkExtensions +import scala.Tuple2 +import scala.Tuple3 +import scala.Tuple4 +import scala.Tuple5 +import kotlin.reflect.KProperty1 + + +/** + * Utility method to create dataset from list + */ +inline fun SparkSession.toDS(list: List): Dataset = + createDataset(list, encoder()) + +/** + * Utility method to create dataset from *array or vararg arguments + */ +inline fun SparkSession.dsOf(vararg t: T): Dataset = + createDataset(listOf(*t), encoder()) + +/** + * Utility method to create dataset from list + */ +inline fun List.toDS(spark: SparkSession): Dataset = + spark.createDataset(this, encoder()) + +/** + * Utility method to create dataset from RDD + */ +inline fun RDD.toDS(spark: SparkSession): Dataset = + spark.createDataset(this, encoder()) + +/** + * Utility method to create dataset from JavaRDD + */ +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. + */ +inline fun Dataset.map(noinline func: (T) -> R): Dataset = + map(MapFunction(func), encoder()) + +/** + * (Kotlin-specific) + * Returns a new Dataset by first applying a function to all elements of this Dataset, + * and then flattening the results. + */ +inline fun Dataset.flatMap(noinline func: (T) -> Iterator): Dataset = + flatMap(func, encoder()) + +/** + * (Kotlin-specific) + * Returns a new Dataset by flattening. This means that a Dataset of an iterable such as + * `listOf(listOf(1, 2, 3), listOf(4, 5, 6))` will be flattened to a Dataset of `listOf(1, 2, 3, 4, 5, 6)`. + */ +inline fun > Dataset.flatten(): Dataset = + flatMap(FlatMapFunction { it.iterator() }, encoder()) + +/** + * (Kotlin-specific) + * Returns a [KeyValueGroupedDataset] where the data is grouped by the given key [func]. + */ +inline fun Dataset.groupByKey(noinline func: (T) -> R): KeyValueGroupedDataset = + groupByKey(MapFunction(func), encoder()) + +/** + * (Kotlin-specific) + * Returns a new Dataset that contains the result of applying [func] to each partition. + */ +inline fun Dataset.mapPartitions(noinline func: (Iterator) -> Iterator): Dataset = + mapPartitions(func, encoder()) + +/** + * (Kotlin-specific) + * Filters rows to eliminate [null] values. + */ +@Suppress("UNCHECKED_CAST") +fun Dataset.filterNotNull(): Dataset = filter { it != null } as Dataset + + +/** + * (Kotlin-specific) + * Reduces the elements of this Dataset using the specified binary function. The given `func` + * must be commutative and associative or the result may be non-deterministic. + */ +inline fun Dataset.reduceK(noinline func: (T, T) -> T): T = + reduce(ReduceFunction(func)) + +/** + * (Kotlin-specific) + * Maps the Dataset to only retain the "keys" or [Tuple2._1] values. + */ +@JvmName("takeKeysTuple2") +inline fun Dataset>.takeKeys(): Dataset = map { it._1() } + +/** + * (Kotlin-specific) + * Maps the Dataset to only retain the "keys" or [Pair.first] values. + */ +inline fun Dataset>.takeKeys(): Dataset = map { it.first } + +/** + * (Kotlin-specific) + * Maps the Dataset to only retain the "keys" or [Arity2._1] values. + */ +@JvmName("takeKeysArity2") +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +inline fun Dataset>.takeKeys(): Dataset = map { it._1 } + +/** + * (Kotlin-specific) + * Maps the Dataset to only retain the "values" or [Tuple2._2] values. + */ +@JvmName("takeValuesTuple2") +inline fun Dataset>.takeValues(): Dataset = map { it._2() } + +/** + * (Kotlin-specific) + * Maps the Dataset to only retain the "values" or [Pair.second] values. + */ +inline fun Dataset>.takeValues(): Dataset = map { it.second } + +/** + * (Kotlin-specific) + * Maps the Dataset to only retain the "values" or [Arity2._2] values. + */ +@JvmName("takeValuesArity2") +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +inline fun Dataset>.takeValues(): Dataset = map { it._2 } + +/** DEPRECATED: Use [as] or [to] for this. */ +@Deprecated( + message = "Deprecated, since we already have `as`() and to().", + replaceWith = ReplaceWith("this.to()"), + level = DeprecationLevel.ERROR, +) +inline fun Dataset.downcast(): Dataset = `as`(encoder()) + +/** + * (Kotlin-specific) + * Returns a new Dataset where each record has been mapped on to the specified type. The + * method used to map columns depend on the type of [R]: + * - When [R] is a class, fields for the class will be mapped to columns of the same name + * (case sensitivity is determined by [spark.sql.caseSensitive]). + * - When [R] is a tuple, the columns will be mapped by ordinal (i.e. the first column will + * be assigned to `_1`). + * - When [R] is a primitive type (i.e. [String], [Int], etc.), then the first column of the + * `DataFrame` will be used. + * + * If the schema of the Dataset does not match the desired [R] type, you can use [Dataset.select]/[selectTyped] + * along with [Dataset.alias] or [as]/[to] to rearrange or rename as required. + * + * Note that [as]/[to] only changes the view of the data that is passed into typed operations, + * such as [map], and does not eagerly project away any columns that are not present in + * the specified class. + * + * @see to as alias for [as] + */ +inline fun Dataset<*>.`as`(): Dataset = `as`(encoder()) + +/** + * (Kotlin-specific) + * Returns a new Dataset where each record has been mapped on to the specified type. The + * method used to map columns depend on the type of [R]: + * - When [R] is a class, fields for the class will be mapped to columns of the same name + * (case sensitivity is determined by [spark.sql.caseSensitive]). + * - When [R] is a tuple, the columns will be mapped by ordinal (i.e. the first column will + * be assigned to `_1`). + * - When [R] is a primitive type (i.e. [String], [Int], etc.), then the first column of the + * `DataFrame` will be used. + * + * If the schema of the Dataset does not match the desired [R] type, you can use [Dataset.select]/[selectTyped] + * along with [Dataset.alias] or [as]/[to] to rearrange or rename as required. + * + * Note that [as]/[to] only changes the view of the data that is passed into typed operations, + * such as [map], and does not eagerly project away any columns that are not present in + * the specified class. + * + * @see as as alias for [to] + */ +inline fun Dataset<*>.to(): Dataset = `as`(encoder()) + +/** + * (Kotlin-specific) + * Applies a function [func] to all rows. + */ +inline fun Dataset.forEach(noinline func: (T) -> Unit): Unit = foreach(ForeachFunction(func)) + +/** + * (Kotlin-specific) + * Runs [func] on each partition of this Dataset. + */ +inline fun Dataset.forEachPartition(noinline func: (Iterator) -> Unit): Unit = + foreachPartition(ForeachPartitionFunction(func)) + +/** + * It's hard to call `Dataset.debugCodegen` from kotlin, so here is utility for that + */ +fun Dataset.debugCodegen(): Dataset = also { KSparkExtensions.debugCodegen(it) } + +/** + * It's hard to call `Dataset.debug` from kotlin, so here is utility for that + */ +fun Dataset.debug(): Dataset = also { KSparkExtensions.debug(it) } + + +/** + * Alias for [Dataset.joinWith] which passes "left" argument + * and respects the fact that in result of left join right relation is nullable + * + * @receiver left dataset + * @param right right dataset + * @param col join condition + * + * @return dataset of [Tuple2] where right element is forced nullable + */ +inline fun Dataset.leftJoin(right: Dataset, col: Column): Dataset> = + joinWith(right, col, "left") + +/** + * Alias for [Dataset.joinWith] which passes "right" argument + * and respects the fact that in result of right join left relation is nullable + * + * @receiver left dataset + * @param right right dataset + * @param col join condition + * + * @return dataset of [Tuple2] where left element is forced nullable + */ +inline fun Dataset.rightJoin(right: Dataset, col: Column): Dataset> = + joinWith(right, col, "right") + +/** + * Alias for [Dataset.joinWith] which passes "inner" argument + * + * @receiver left dataset + * @param right right dataset + * @param col join condition + * + * @return resulting dataset of [Tuple2] + */ +inline fun Dataset.innerJoin(right: Dataset, col: Column): Dataset> = + joinWith(right, col, "inner") + +/** + * Alias for [Dataset.joinWith] which passes "full" argument + * and respects the fact that in result of join any element of resulting tuple is nullable + * + * @receiver left dataset + * @param right right dataset + * @param col join condition + * + * @return dataset of [Tuple2] where both elements are forced nullable + */ +inline fun Dataset.fullJoin( + right: Dataset, + col: Column, +): Dataset> = joinWith(right, col, "full") + +/** + * Alias for [Dataset.sort] which forces user to provide sorted columns from the source dataset + * + * @receiver source [Dataset] + * @param columns producer of sort columns + * @return sorted [Dataset] + */ +inline fun Dataset.sort(columns: (Dataset) -> Array): Dataset = sort(*columns(this)) + +/** Returns a dataset sorted by the first (`_1`) value of each [Tuple2] inside. */ +@JvmName("sortByTuple2Key") +fun Dataset>.sortByKey(): Dataset> = sort("_1") + +/** Returns a dataset sorted by the second (`_2`) value of each [Tuple2] inside. */ +@JvmName("sortByTuple2Value") +fun Dataset>.sortByValue(): Dataset> = sort("_2") + +/** Returns a dataset sorted by the first (`_1`) value of each [Arity2] inside. */ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +@JvmName("sortByArity2Key") +fun Dataset>.sortByKey(): Dataset> = sort("_1") + +/** Returns a dataset sorted by the second (`_2`) value of each [Arity2] inside. */ +@Deprecated("Use Scala tuples instead.", ReplaceWith("")) +@JvmName("sortByArity2Value") +fun Dataset>.sortByValue(): Dataset> = sort("_2") + +/** Returns a dataset sorted by the first (`first`) value of each [Pair] inside. */ +@JvmName("sortByPairKey") +fun Dataset>.sortByKey(): Dataset> = sort("first") + +/** Returns a dataset sorted by the second (`second`) value of each [Pair] inside. */ +@JvmName("sortByPairValue") +fun Dataset>.sortByValue(): Dataset> = sort("second") + +/** + * This function creates block, where one can call any further computations on already cached dataset + * Data will be unpersisted automatically at the end of computation + * + * it may be useful in many situations, for example, when one needs to write data to several targets + * ```kotlin + * ds.withCached { + * write() + * .also { it.orc("First destination") } + * .also { it.avro("Second destination") } + * } + * ``` + * + * @param blockingUnpersist if execution should be blocked until everything persisted will be deleted + * @param executeOnCached Block which should be executed on cached dataset. + * @return result of block execution for further usage. It may be anything including source or new dataset + */ +inline fun Dataset.withCached( + blockingUnpersist: Boolean = false, + executeOnCached: Dataset.() -> R, +): R { + val cached = this.cache() + return cached.executeOnCached().also { cached.unpersist(blockingUnpersist) } +} + +/** + * Collects the dataset as list where each item has been mapped to type [T]. + */ +inline fun Dataset<*>.toList(): List = to().collectAsList() as List + +/** + * Collects the dataset as Array where each item has been mapped to type [T]. + */ +inline fun Dataset<*>.toArray(): Array = to().collect() as Array + + +/** + * Allows to sort data class dataset on one or more of the properties of the data class. + * ```kotlin + * val sorted: Dataset = unsorted.sort(YourClass::a) + * val sorted2: Dataset = unsorted.sort(YourClass::a, YourClass::b) + * ``` + */ +fun Dataset.sort(col: KProperty1, vararg cols: KProperty1): Dataset = + sort(col.name, *cols.map { it.name }.toTypedArray()) + +/** + * Alternative to [Dataset.show] which returns source dataset. + * Useful for debug purposes when you need to view content of a dataset as an intermediate operation + */ +fun Dataset.showDS(numRows: Int = 20, truncate: Boolean = true): Dataset = apply { show(numRows, truncate) } + +/** + * 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, +): Dataset> = + select( + c1 as TypedColumn, + c2 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, + c3: TypedColumn, +): Dataset> = + select( + c1 as TypedColumn, + c2 as TypedColumn, + c3 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, + c3: TypedColumn, + c4: TypedColumn, +): Dataset> = + select( + c1 as TypedColumn, + c2 as TypedColumn, + c3 as TypedColumn, + c4 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, + c3: TypedColumn, + c4: TypedColumn, + c5: TypedColumn, +): Dataset> = + select( + c1 as TypedColumn, + c2 as TypedColumn, + c3 as TypedColumn, + c4 as TypedColumn, + c5 as TypedColumn, + ) + diff --git a/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Encoding.kt b/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Encoding.kt new file mode 100644 index 00000000..643237ec --- /dev/null +++ b/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Encoding.kt @@ -0,0 +1,341 @@ +/*- + * =LICENSE= + * Kotlin Spark API + * ---------- + * Copyright (C) 2019 - 2020 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= + */ + +/** + * This file contains the encoding logic for the Kotlin Spark API. + * It provides encoders for Spark, based on reflection, for functions that need it. + * Aside from the normal Spark encoders, it also provides encoding for Kotlin data classes, Iterables, + * Products, Arrays, Maps etc. + */ + +@file:Suppress("HasPlatformType", "unused", "FunctionName") + +package org.jetbrains.kotlinx.spark.api + +import org.apache.spark.sql.* +import org.apache.spark.sql.Encoders.* +import org.apache.spark.sql.KotlinReflection.* +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.sql.types.* +import org.apache.spark.unsafe.types.CalendarInterval +import scala.Product +import scala.reflect.ClassTag +import java.beans.PropertyDescriptor +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 java.util.* +import java.util.concurrent.ConcurrentHashMap +import kotlin.Boolean +import kotlin.BooleanArray +import kotlin.Byte +import kotlin.ByteArray +import kotlin.Double +import kotlin.DoubleArray +import kotlin.Float +import kotlin.FloatArray +import kotlin.Int +import kotlin.IntArray +import kotlin.Long +import kotlin.LongArray +import kotlin.Short +import kotlin.ShortArray +import kotlin.String +import kotlin.reflect.KClass +import kotlin.reflect.KType +import kotlin.reflect.full.findAnnotation +import kotlin.reflect.full.isSubclassOf +import kotlin.reflect.full.primaryConstructor +import kotlin.reflect.typeOf + +@JvmField +val ENCODERS: Map, Encoder<*>> = mapOf( + Boolean::class to BOOLEAN(), + Byte::class to BYTE(), + Short::class to SHORT(), + Int::class to INT(), + Long::class to LONG(), + Float::class to FLOAT(), + Double::class to DOUBLE(), + String::class to STRING(), + BigDecimal::class to DECIMAL(), + Date::class to DATE(), + LocalDate::class to LOCALDATE(), // 3.0+ + Timestamp::class to TIMESTAMP(), + Instant::class to INSTANT(), // 3.0+ + ByteArray::class to BINARY(), +// Duration::class to DURATION(), // 3.2+ +// Period::class to PERIOD(), // 3.2+ +) + +private fun checkIfEncoderRequiresNewerVersion(klass: KClass<*>) { + when (klass) { + Duration::class, Period::class -> throw IllegalArgumentException("$klass is supported in Spark 3.2") + } +} + + +private val knownDataTypes: Map, DataType> = mapOf( + Byte::class to DataTypes.ByteType, + Short::class to DataTypes.ShortType, + Int::class to DataTypes.IntegerType, + Long::class to DataTypes.LongType, + Boolean::class to DataTypes.BooleanType, + Float::class to DataTypes.FloatType, + Double::class to DataTypes.DoubleType, + String::class to DataTypes.StringType, + LocalDate::class to DataTypes.DateType, + Date::class to DataTypes.DateType, + Timestamp::class to DataTypes.TimestampType, + Instant::class to DataTypes.TimestampType, + ByteArray::class to DataTypes.BinaryType, + Decimal::class to DecimalType.SYSTEM_DEFAULT(), + BigDecimal::class to DecimalType.SYSTEM_DEFAULT(), + CalendarInterval::class to DataTypes.CalendarIntervalType, +) + +/** + * Main method of API, which gives you seamless integration with Spark: + * It creates encoder for any given supported type T + * + * Supported types are data classes, primitives, and Lists, Maps and Arrays containing them + * + * @param T type, supported by Spark + * @return generated encoder + */ +@OptIn(ExperimentalStdlibApi::class) +inline fun encoder(): Encoder = generateEncoder(typeOf(), T::class) + +/** + * @see encoder + */ +@Suppress("UNCHECKED_CAST") +fun generateEncoder(type: KType, cls: KClass<*>): Encoder { + checkIfEncoderRequiresNewerVersion(cls) + return when { + isSupportedByKotlinClassEncoder(cls) -> kotlinClassEncoder(schema = memoizedSchema(type), kClass = cls) + else -> ENCODERS[cls] as? Encoder? ?: bean(cls.java) + } as Encoder +} + +private fun isSupportedByKotlinClassEncoder(cls: KClass<*>): Boolean = + when { + cls == ByteArray::class -> false // uses binary encoder + cls.isData -> true + cls.isSubclassOf(Map::class) -> true + cls.isSubclassOf(Iterable::class) -> true + cls.isSubclassOf(Product::class) -> true + cls.java.isArray -> true + else -> false + } + + +private fun kotlinClassEncoder(schema: DataType, kClass: KClass<*>): Encoder { + val serializer = + if (schema is DataTypeWithClass) serializerFor(kClass.java, schema) + else serializerForType(getType(kClass.java)) + + val deserializer = + if (schema is DataTypeWithClass) deserializerFor(kClass.java, schema) + else deserializerForType(getType(kClass.java)) + + return ExpressionEncoder(serializer, deserializer, ClassTag.apply(kClass.java)) +} + +/** + * Not meant to be used by the user explicitly. + * + * This function generates the DataType schema for supported classes, including Kotlin data classes, [Map], + * [Iterable], [Product], [Array], and combinations of those. + * + * It's mainly used by [generateEncoder]/[encoder]. + */ +@OptIn(ExperimentalStdlibApi::class) +fun schema(type: KType, map: Map = mapOf()): DataType { + val primitiveSchema = knownDataTypes[type.classifier] + if (primitiveSchema != null) + return KSimpleTypeWrapper( + /* dt = */ primitiveSchema, + /* cls = */ (type.classifier!! as KClass<*>).java, + /* nullable = */ type.isMarkedNullable + ) + + val klass = type.classifier as? KClass<*> ?: throw IllegalArgumentException("Unsupported type $type") + val args = type.arguments + + val types = transitiveMerge( + map, + klass.typeParameters.zip(args).associate { + it.first.name to it.second.type!! + }, + ) + + return when { + klass.isSubclassOf(Enum::class) -> + KSimpleTypeWrapper( + /* dt = */ DataTypes.StringType, + /* cls = */ klass.java, + /* nullable = */ type.isMarkedNullable + ) + + klass.isSubclassOf(Iterable::class) || klass.java.isArray -> { + val listParam = if (klass.java.isArray) { + when (klass) { + IntArray::class -> typeOf() + LongArray::class -> typeOf() + FloatArray::class -> typeOf() + DoubleArray::class -> typeOf() + BooleanArray::class -> typeOf() + ShortArray::class -> typeOf() + /* ByteArray handled by BinaryType */ + else -> types.getValue(klass.typeParameters[0].name) + } + } else types.getValue(klass.typeParameters[0].name) + + val dataType = DataTypes.createArrayType( + /* elementType = */ schema(listParam, types), + /* containsNull = */ listParam.isMarkedNullable + ) + + KComplexTypeWrapper( + /* dt = */ dataType, + /* cls = */ klass.java, + /* nullable = */ type.isMarkedNullable + ) + } + + klass.isSubclassOf(Map::class) -> { + val mapKeyParam = types.getValue(klass.typeParameters[0].name) + val mapValueParam = types.getValue(klass.typeParameters[1].name) + + val dataType = DataTypes.createMapType( + /* keyType = */ schema(mapKeyParam, types), + /* valueType = */ schema(mapValueParam, types), + /* valueContainsNull = */ true + ) + + KComplexTypeWrapper( + /* dt = */ dataType, + /* cls = */ klass.java, + /* nullable = */ type.isMarkedNullable + ) + } + + klass.isData -> { + val structType = StructType( + klass + .primaryConstructor!! + .parameters + .filter { it.findAnnotation() == null } + .map { + val projectedType = types[it.type.toString()] ?: it.type + val propertyDescriptor = PropertyDescriptor( + /* propertyName = */ it.name, + /* beanClass = */ klass.java, + /* readMethodName = */ "is" + it.name?.replaceFirstChar { + if (it.isLowerCase()) it.titlecase(Locale.getDefault()) + else it.toString() + }, + /* writeMethodName = */ null + ) + + KStructField( + /* getterName = */ propertyDescriptor.readMethod.name, + /* delegate = */ StructField( + /* name = */ it.name, + /* dataType = */ schema(projectedType, types), + /* nullable = */ projectedType.isMarkedNullable, + /* metadata = */ Metadata.empty() + ) + ) + } + .toTypedArray() + ) + KDataTypeWrapper(structType, klass.java, true) + } + klass.isSubclassOf(Product::class) -> { + + // create map from T1, T2 to Int, String etc. + val typeMap = klass.constructors.first().typeParameters.map { it.name } + .zip( + type.arguments.map { it.type } + ) + .toMap() + + // collect params by name and actual type + val params = klass.constructors.first().parameters.map { + val typeName = it.type.toString().replace("!", "") + it.name to (typeMap[typeName] ?: it.type) + } + + val structType = DataTypes.createStructType( + params.map { (fieldName, fieldType) -> + val dataType = schema(fieldType, types) + + KStructField( + /* getterName = */ fieldName, + /* delegate = */ StructField( + /* name = */ fieldName, + /* dataType = */ dataType, + /* nullable = */ fieldType.isMarkedNullable, + /* metadata = */Metadata.empty() + ) + ) + }.toTypedArray() + ) + + KComplexTypeWrapper( + /* dt = */ structType, + /* cls = */ klass.java, + /* nullable = */ true + ) + } + + else -> throw IllegalArgumentException("$type is unsupported") + } +} + +/** + * Memoized version of [schema]. This ensures the [DataType] of given `type` only + * has to be calculated once. + */ +private val memoizedSchema: (type: KType) -> DataType = memoize { + schema(it) +} + +private fun transitiveMerge(a: Map, b: Map): Map = + a + b.mapValues { a.getOrDefault(it.value.toString(), it.value) } + +/** Wrapper around function with 1 argument to avoid recalculation when a certain argument is queried again. */ +private class Memoize1(private val function: (T) -> R) : (T) -> R { + private val values = ConcurrentHashMap() + override fun invoke(x: T): R = values.getOrPut(x) { function(x) } +} + +/** Wrapper around function to avoid recalculation when a certain argument is queried again. */ +private fun ((T) -> R).memoized(): (T) -> R = Memoize1(this) + +/** Wrapper around function to avoid recalculation when a certain argument is queried again. */ +private fun memoize(function: (T) -> R): (T) -> R = Memoize1(function) + diff --git a/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/GroupState.kt b/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/GroupState.kt new file mode 100644 index 00000000..e2013783 --- /dev/null +++ b/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/GroupState.kt @@ -0,0 +1,64 @@ +/*- + * =LICENSE= + * Kotlin Spark API: API for Spark 3.0+ (Scala 2.12) + * ---------- + * Copyright (C) 2019 - 2021 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= + */ + +/** + * This file contains some helper functions to more easily work with [GroupState] from Kotlin. + */ + +package org.jetbrains.kotlinx.spark.api + +import org.apache.spark.sql.streaming.GroupState +import kotlin.reflect.KProperty + +/** + * (Kotlin-specific) + * Returns the group state value if it exists, else `null`. + * This is comparable to [GroupState.getOption], but instead utilises Kotlin's nullability features + * to get the same result. + */ +fun GroupState.getOrNull(): S? = if (exists()) get() else null + +/** + * (Kotlin-specific) + * Allows the group state object to be used as a delegate. Will be `null` if it does not exist. + * + * For example: + * ```kotlin + * groupedDataset.mapGroupsWithState(GroupStateTimeout.NoTimeout()) { key, values, state: GroupState -> + * var s by state + * ... + * } + * ``` + */ +operator fun GroupState.getValue(thisRef: Any?, property: KProperty<*>): S? = getOrNull() + +/** + * (Kotlin-specific) + * Allows the group state object to be used as a delegate. Will be `null` if it does not exist. + * + * For example: + * ```kotlin + * groupedDataset.mapGroupsWithState(GroupStateTimeout.NoTimeout()) { key, values, state: GroupState -> + * var s by state + * ... + * } + * ``` + */ +operator fun GroupState.setValue(thisRef: Any?, property: KProperty<*>, value: S?): Unit = update(value) diff --git a/kotlin-spark-api/common/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Iterators.kt b/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Iterators.kt similarity index 57% rename from kotlin-spark-api/common/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Iterators.kt rename to kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Iterators.kt index b93ce377..1be79918 100644 --- a/kotlin-spark-api/common/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Iterators.kt +++ b/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Iterators.kt @@ -17,35 +17,57 @@ * limitations under the License. * =LICENSEEND= */ + +/** + * This file contains several ways to wrap and modify iterators lazily. + * This includes mapping, filtering, and partitioning. + */ + package org.jetbrains.kotlinx.spark.api +/** Partitions the values of the iterator lazily in groups of [size]. */ class PartitioningIterator( private val source: Iterator, private val size: Int, - private val cutIncomplete: Boolean = false + private val cutIncomplete: Boolean = false, ) : AbstractIterator>() { + override fun computeNext() { if (!source.hasNext()) return done() val interimResult = arrayListOf() repeat(size) { - if (source.hasNext()) interimResult.add(source.next()) - else return if (cutIncomplete) done() else setNext(interimResult) + if (source.hasNext()) + interimResult.add(source.next()) + else + return if (cutIncomplete) + done() + else + setNext(interimResult) } setNext(interimResult) } + } +/** Maps the values of the iterator lazily using [func]. */ class MappingIterator( - private val self: Iterator, - private val func: (T) -> R + private val source: Iterator, + private val func: (T) -> R, ) : AbstractIterator() { - override fun computeNext() = if (self.hasNext()) setNext(func(self.next())) else done() + + override fun computeNext(): Unit = + if (source.hasNext()) + setNext(func(source.next())) + else + done() } +/** Filters the values of the iterator lazily using [predicate]. */ class FilteringIterator( private val source: Iterator, - private val predicate: (T) -> Boolean + private val predicate: (T) -> Boolean, ) : AbstractIterator() { + override fun computeNext() { while (source.hasNext()) { val next = source.next() @@ -56,10 +78,15 @@ class FilteringIterator( } done() } + } + +/** Maps the values of the iterator lazily using [func]. */ fun Iterator.map(func: (T) -> R): Iterator = MappingIterator(this, func) -fun Iterator.filter(func: (T) -> Boolean): Iterator = FilteringIterator(this, func) +/** Filters the values of the iterator lazily using [predicate]. */ +fun Iterator.filter(predicate: (T) -> Boolean): Iterator = FilteringIterator(this, predicate) +/** Partitions the values of the iterator lazily in groups of [size]. */ fun Iterator.partition(size: Int): Iterator> = PartitioningIterator(this, size) diff --git a/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/KeyValueGroupedDataset.kt b/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/KeyValueGroupedDataset.kt new file mode 100644 index 00000000..ec840dbe --- /dev/null +++ b/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/KeyValueGroupedDataset.kt @@ -0,0 +1,203 @@ +/*- + * =LICENSE= + * Kotlin Spark API: API for Spark 3.0+ (Scala 2.12) + * ---------- + * Copyright (C) 2019 - 2021 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= + */ + +/** + * This file contains several extension functions to work with [KeyValueGroupedDataset]s more easily + * from Kotlin. This includes automatically providing the right encoders, as well as mapping to `Arities`. + */ + +package org.jetbrains.kotlinx.spark.api + +import org.apache.spark.api.java.function.CoGroupFunction +import org.apache.spark.api.java.function.FlatMapGroupsFunction +import org.apache.spark.api.java.function.FlatMapGroupsWithStateFunction +import org.apache.spark.api.java.function.MapFunction +import org.apache.spark.api.java.function.MapGroupsFunction +import org.apache.spark.api.java.function.MapGroupsWithStateFunction +import org.apache.spark.api.java.function.ReduceFunction +import org.apache.spark.sql.Dataset +import org.apache.spark.sql.Encoder +import org.apache.spark.sql.KeyValueGroupedDataset +import org.apache.spark.sql.streaming.GroupState +import org.apache.spark.sql.streaming.GroupStateTimeout +import org.apache.spark.sql.streaming.OutputMode +import scala.Tuple2 + + +/** + * Returns a new [KeyValueGroupedDataset] where the given function [func] has been applied + * to the data. The grouping key is unchanged by this. + * + * ```kotlin + * // Create values grouped by key from a Dataset> + * ds.groupByKey { it._1 }.mapValues { it._2 } + * ``` + */ +inline fun KeyValueGroupedDataset.mapValues(noinline func: (VALUE) -> R): KeyValueGroupedDataset = + mapValues(MapFunction(func), encoder()) + +/** + * (Kotlin-specific) + * Applies the given function to each group of data. For each unique group, the function will + * be passed the group key and an iterator that contains all the elements in the group. The + * function can return an element of arbitrary type which will be returned as a new [Dataset]. + * + * This function does not support partial aggregation, and as a result requires shuffling all + * the data in the [Dataset]. If an application intends to perform an aggregation over each + * key, it is best to use the reduce function or an + * [org.apache.spark.sql.expressions.Aggregator]. + * + * Internally, the implementation will spill to disk if any given group is too large to fit into + * memory. However, users must take care to avoid materializing the whole iterator for a group + * (for example, by calling [toList]) unless they are sure that this is possible given the memory + * constraints of their cluster. + */ +inline fun KeyValueGroupedDataset.mapGroups(noinline func: (KEY, Iterator) -> R): Dataset = + mapGroups(MapGroupsFunction(func), encoder()) + +/** + * (Kotlin-specific) + * Reduces the elements of each group of data using the specified binary function. + * The given function must be commutative and associative or the result may be non-deterministic. + * + * Note that you need to use [reduceGroupsK] always instead of the Java- or Scala-specific + * [KeyValueGroupedDataset.reduceGroups] to make the compiler work. + */ +inline fun KeyValueGroupedDataset.reduceGroupsK(noinline func: (VALUE, VALUE) -> VALUE): Dataset> = + reduceGroups(ReduceFunction(func)) + +/** + * (Kotlin-specific) + * Applies the given function to each group of data. For each unique group, the function will + * be passed the group key and an iterator that contains all the elements in the group. The + * function can return an iterator containing elements of an arbitrary type which will be returned + * as a new [Dataset]. + * + * This function does not support partial aggregation, and as a result requires shuffling all + * the data in the [Dataset]. If an application intends to perform an aggregation over each + * key, it is best to use the reduce function or an + * [org.apache.spark.sql.expressions.Aggregator]. + * + * Internally, the implementation will spill to disk if any given group is too large to fit into + * memory. However, users must take care to avoid materializing the whole iterator for a group + * (for example, by calling [toList]) unless they are sure that this is possible given the memory + * constraints of their cluster. + */ +inline fun KeyValueGroupedDataset.flatMapGroups( + noinline func: (key: K, values: Iterator) -> Iterator, +): Dataset = flatMapGroups( + FlatMapGroupsFunction(func), + encoder(), +) + + +/** + * (Kotlin-specific) + * Applies the given function to each group of data, while maintaining a user-defined per-group + * state. The result Dataset will represent the objects returned by the function. + * For a static batch Dataset, the function will be invoked once per group. For a streaming + * Dataset, the function will be invoked for each group repeatedly in every trigger, and + * updates to each group's state will be saved across invocations. + * See [org.apache.spark.sql.streaming.GroupState] for more details. + * + * @param S The type of the user-defined state. Must be encodable to Spark SQL types. + * @param U The type of the output objects. Must be encodable to Spark SQL types. + * @param func Function to be called on every group. + * + * See [Encoder] for more details on what types are encodable to Spark SQL. + */ +inline fun KeyValueGroupedDataset.mapGroupsWithState( + noinline func: (key: K, values: Iterator, state: GroupState) -> U, +): Dataset = mapGroupsWithState( + MapGroupsWithStateFunction(func), + encoder(), + encoder(), +) + +/** + * (Kotlin-specific) + * Applies the given function to each group of data, while maintaining a user-defined per-group + * state. The result Dataset will represent the objects returned by the function. + * For a static batch Dataset, the function will be invoked once per group. For a streaming + * Dataset, the function will be invoked for each group repeatedly in every trigger, and + * updates to each group's state will be saved across invocations. + * See [org.apache.spark.sql.streaming.GroupState] for more details. + * + * @param S The type of the user-defined state. Must be encodable to Spark SQL types. + * @param U The type of the output objects. Must be encodable to Spark SQL types. + * @param func Function to be called on every group. + * @param timeoutConf Timeout configuration for groups that do not receive data for a while. + * + * See [Encoder] for more details on what types are encodable to Spark SQL. + */ +inline fun KeyValueGroupedDataset.mapGroupsWithState( + timeoutConf: GroupStateTimeout, + noinline func: (key: K, values: Iterator, state: GroupState) -> U, +): Dataset = mapGroupsWithState( + MapGroupsWithStateFunction(func), + encoder(), + encoder(), + timeoutConf, +) + +/** + * (Kotlin-specific) + * Applies the given function to each group of data, while maintaining a user-defined per-group + * state. The result Dataset will represent the objects returned by the function. + * For a static batch Dataset, the function will be invoked once per group. For a streaming + * Dataset, the function will be invoked for each group repeatedly in every trigger, and + * updates to each group's state will be saved across invocations. + * See [GroupState] for more details. + * + * @param S The type of the user-defined state. Must be encodable to Spark SQL types. + * @param U The type of the output objects. Must be encodable to Spark SQL types. + * @param func Function to be called on every group. + * @param outputMode The output mode of the function. + * @param timeoutConf Timeout configuration for groups that do not receive data for a while. + * + * See [Encoder] for more details on what types are encodable to Spark SQL. + */ +inline fun KeyValueGroupedDataset.flatMapGroupsWithState( + outputMode: OutputMode, + timeoutConf: GroupStateTimeout, + noinline func: (key: K, values: Iterator, state: GroupState) -> Iterator, +): Dataset = flatMapGroupsWithState( + FlatMapGroupsWithStateFunction(func), + outputMode, + encoder(), + encoder(), + timeoutConf, +) + +/** + * (Kotlin-specific) + * Applies the given function to each cogrouped data. For each unique group, the function will + * be passed the grouping key and 2 iterators containing all elements in the group from + * [Dataset] [this] and [other]. The function can return an iterator containing elements of an + * arbitrary type which will be returned as a new [Dataset]. + */ +inline fun KeyValueGroupedDataset.cogroup( + other: KeyValueGroupedDataset, + noinline func: (key: K, left: Iterator, right: Iterator) -> Iterator, +): Dataset = cogroup( + other, + CoGroupFunction(func), + encoder(), +) \ No newline at end of file diff --git a/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt b/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt new file mode 100644 index 00000000..652e52b7 --- /dev/null +++ b/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkSession.kt @@ -0,0 +1,382 @@ +/*- + * =LICENSE= + * Kotlin Spark API + * ---------- + * Copyright (C) 2019 - 2020 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= + */ + +/** + * 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.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.Row +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.api.tuples.* +import org.jetbrains.kotlinx.spark.extensions.KSparkExtensions +import java.io.Serializable + +/** + * This wrapper over [SparkSession] which provides several additional methods to create [org.apache.spark.sql.Dataset]. + * + * @param spark The current [SparkSession] to wrap + */ +class KSparkSession(val spark: SparkSession) { + + /** Lazy instance of [JavaSparkContext] wrapper around [sparkContext]. */ + val sc: JavaSparkContext by lazy { JavaSparkContext(spark.sparkContext) } + + /** Utility method to create dataset from list. */ + inline fun List.toDS(): Dataset = toDS(spark) + + /** Utility method to create dataset from [Array]. */ + inline fun Array.toDS(): Dataset = spark.dsOf(*this) + + /** Utility method to create dataset from vararg arguments. */ + inline fun dsOf(vararg arg: T): Dataset = spark.dsOf(*arg) + + /** Utility method to create dataset from Scala [RDD]. */ + inline fun RDD.toDS(): Dataset = toDS(spark) + + /** 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). + * + * The following example registers a UDF in Kotlin: + * ```Kotlin + * sparkSession.udf.register("myUDF") { arg1: Int, arg2: String -> arg2 + arg1 } + * ``` + * + * @note The user-defined functions must be deterministic. Due to optimization, + * duplicate invocations may be eliminated or the function may even be invoked more times than + * it is present in the query. + */ + 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(@Transient val ssc: JavaStreamingContext) : Serializable { + // Serializable and Transient so that [withSpark] works inside [foreachRDD] and other Spark functions that serialize + + private var runAfterStart: KSparkStreamingSession.() -> Unit = {} + + /** [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 + } + + 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. + * ```kotlin + * withSpark(rdd) { // this: KSparkSession + * + * } + * ``` + */ + fun withSpark(rddForConf: JavaRDDLike<*, *>, func: KSparkSession.() -> T): T = + KSparkSession(getSpark(rddForConf)).func() + + /** + * Helper function to enter Spark scope from [sscForConf] like + * ```kotlin + * withSpark(ssc) { // this: KSparkSession + * + * } + * ``` + */ + fun withSpark(sscForConf: JavaStreamingContext, func: KSparkSession.() -> T): T = + KSparkSession(getSpark(sscForConf)).func() +} + + +/** + * The entry point to programming Spark with the Dataset and DataFrame API. + * + * @see org.apache.spark.sql.SparkSession + */ +typealias SparkSession = org.apache.spark.sql.SparkSession + +/** + * Control our logLevel. This overrides any user-defined log settings. + * @param level The desired log level as [SparkLogLevel]. + */ +fun SparkContext.setLogLevel(level: SparkLogLevel): Unit = setLogLevel(level.name) + +/** Log levels for spark. */ +enum class SparkLogLevel { + ALL, DEBUG, ERROR, FATAL, INFO, OFF, TRACE, WARN +} + +/** + * Returns the Spark context associated with this Spark session. + */ +val SparkSession.sparkContext: SparkContext + get() = KSparkExtensions.sparkContext(this) + +/** + * Wrapper for spark creation which allows setting different spark params. + * + * @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 [KSparkSession] (it means that `this` inside block will point to [KSparkSession]) + */ +@JvmOverloads +inline fun withSpark( + props: Map = emptyMap(), + master: String = SparkConf().get("spark.master", "local[*]"), + appName: String = "Kotlin Spark Sample", + logLevel: SparkLogLevel = ERROR, + func: KSparkSession.() -> Unit, +) { + val builder = SparkSession + .builder() + .master(master) + .appName(appName) + .apply { + props.forEach { + when (val value = it.value) { + is String -> config(it.key, value) + is Boolean -> config(it.key, value) + is Long -> config(it.key, value) + is Double -> config(it.key, value) + else -> throw IllegalArgumentException("Cannot set property ${it.key} because value $value of unsupported type ${value::class}") + } + } + } + withSpark(builder, logLevel, func) + +} + +/** + * Wrapper for spark creation which allows setting different spark params. + * + * @param builder A [SparkSession.Builder] object, configured how you want. + * @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]) + */ + +@JvmOverloads +inline fun withSpark(builder: Builder, logLevel: SparkLogLevel = ERROR, func: KSparkSession.() -> Unit) { + builder + .getOrCreate() + .apply { + KSparkSession(this).apply { + sparkContext.setLogLevel(logLevel) + func() + spark.stop() + } + } +} + +/** + * Wrapper for spark creation which copies params from [sparkConf]. + * + * @param sparkConf Sets a list of config options based on this. + * @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]) + */ +@JvmOverloads +inline fun withSpark(sparkConf: SparkConf, logLevel: SparkLogLevel = ERROR, func: KSparkSession.() -> Unit) { + withSpark( + builder = SparkSession.builder().config(sparkConf), + logLevel = logLevel, + func = 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. 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( + 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, + startStreamingContext: Boolean = true, + func: KSparkStreamingSession.() -> Unit, +) { + + // will only be set when a new context is created + var kSparkStreamingSession: KSparkStreamingSession? = null + + val 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 + } + + val ssc = when { + checkpointPath != null -> + JavaStreamingContext.getOrCreate(checkpointPath, creatingFunc, hadoopConf, createOnError) + + else -> creatingFunc() + } + + if (startStreamingContext) { + 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. + * 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 + */ +inline fun SparkSession.broadcast(value: T): Broadcast = try { + sparkContext.broadcast(value, encoder().clsTag()) +} catch (e: ClassNotFoundException) { + JavaSparkContext(sparkContext).broadcast(value) +} + +/** + * 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 + */ +@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.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/StreamingKeyValues.kt b/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/StreamingKeyValues.kt new file mode 100644 index 00000000..f9044b5b --- /dev/null +++ b/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/StreamingKeyValues.kt @@ -0,0 +1,676 @@ +/*- + * =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("unused") + +package org.jetbrains.kotlinx.spark.api + +import org.apache.spark.HashPartitioner +import org.apache.spark.Partitioner +import org.apache.spark.api.java.JavaPairRDD +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.api.java.Optional +import org.apache.spark.streaming.Duration +import org.apache.spark.streaming.StateSpec +import org.apache.spark.streaming.api.java.JavaDStream +import org.apache.spark.streaming.api.java.JavaMapWithStateDStream +import org.apache.spark.streaming.api.java.JavaPairDStream +import scala.Tuple2 + + +fun JavaDStream>.toPairDStream(): JavaPairDStream = + JavaPairDStream.fromJavaDStream(this) + +fun JavaPairDStream.toTupleDStream(): JavaDStream> = + toJavaDStream() + +fun JavaRDD>.toPairRDD(): JavaPairRDD = + JavaPairRDD.fromJavaRDD(this) + +fun JavaPairRDD.toTupleRDD(): JavaRDD> = + JavaPairRDD.toRDD(this).toJavaRDD() + + +/** + * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to + * generate the RDDs with `numPartitions` partitions. + */ +fun JavaDStream>.groupByKey( + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +): JavaDStream>> = + toPairDStream() + .groupByKey(numPartitions) + .toTupleDStream() + +/** + * 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 JavaDStream>.groupByKey(partitioner: Partitioner): JavaDStream>> = + toPairDStream() + .groupByKey(partitioner) + .toTupleDStream() + +/** + * 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 JavaDStream>.reduceByKey( + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), + reduceFunc: (V, V) -> V, +): JavaDStream> = + toPairDStream() + .reduceByKey(reduceFunc, numPartitions) + .toTupleDStream() + +/** + * 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 JavaDStream>.reduceByKey( + partitioner: Partitioner, + reduceFunc: (V, V) -> V, +): JavaDStream> = + toPairDStream() + .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 + * 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, + partitioner: Partitioner, + mapSideCombine: Boolean = true, +): JavaDStream> = + toPairDStream() + .combineByKey(createCombiner, mergeValue, mergeCombiner, partitioner, mapSideCombine) + .toTupleDStream() + +/** + * 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 JavaDStream>.groupByKeyAndWindow( + windowDuration: Duration, + slideDuration: Duration = dstream().slideDuration(), + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +): JavaDStream>> = + toPairDStream() + .groupByKeyAndWindow(windowDuration, slideDuration, numPartitions) + .toTupleDStream() + +/** + * 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 JavaDStream>.groupByKeyAndWindow( + windowDuration: Duration, + slideDuration: Duration = dstream().slideDuration(), + partitioner: Partitioner, +): JavaDStream>> = + toPairDStream() + .groupByKeyAndWindow(windowDuration, slideDuration, partitioner) + .toTupleDStream() + +/** + * 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 JavaDStream>.reduceByKeyAndWindow( + windowDuration: Duration, + slideDuration: Duration = dstream().slideDuration(), + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), + reduceFunc: (V, V) -> V, +): JavaDStream> = + toPairDStream() + .reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, numPartitions) + .toTupleDStream() + +/** + * 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 JavaDStream>.reduceByKeyAndWindow( + windowDuration: Duration, + slideDuration: Duration = dstream().slideDuration(), + partitioner: Partitioner, + reduceFunc: (V, V) -> V, +): JavaDStream> = + toPairDStream() + .reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, partitioner) + .toTupleDStream() + +/** + * 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 JavaDStream>.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> = + toPairDStream() + .reduceByKeyAndWindow( + /* reduceFunc = */ reduceFunc, + /* invReduceFunc = */ invReduceFunc, + /* windowDuration = */ windowDuration, + /* slideDuration = */ slideDuration, + /* numPartitions = */ numPartitions, + /* filterFunc = */ filterFunc?.let { + { tuple -> + filterFunc(tuple) + } + } + ) + .toTupleDStream() + +/** + * 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 JavaDStream>.reduceByKeyAndWindow( + invReduceFunc: (V, V) -> V, + windowDuration: Duration, + slideDuration: Duration = dstream().slideDuration(), + partitioner: Partitioner, + filterFunc: ((Tuple2) -> Boolean)? = null, + reduceFunc: (V, V) -> V, +): JavaDStream> = + toPairDStream() + .reduceByKeyAndWindow( + /* reduceFunc = */ reduceFunc, + /* invReduceFunc = */ invReduceFunc, + /* windowDuration = */ windowDuration, + /* slideDuration = */ slideDuration, + /* partitioner = */ partitioner, + /* filterFunc = */ filterFunc?.let { + { tuple -> + filterFunc(tuple) + } + } + ) + .toTupleDStream() + +/** + * 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 + * 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 + * 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 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 + */ +fun JavaDStream>.mapWithState( + spec: StateSpec, +): JavaMapWithStateDStream = + 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. + * 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("updateStateByKeyNullable") +fun JavaDStream>.updateStateByKey( + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), + updateFunc: (List, S?) -> S?, +): JavaDStream> = + toPairDStream() + .updateStateByKey( + { list: List, s: Optional -> + updateFunc(list, s.getOrNull()).toOptional() + }, + 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. + * 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. + * 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 + */ +@JvmName("updateStateByKeyNullable") +fun JavaDStream>.updateStateByKey( + partitioner: Partitioner, + updateFunc: (List, S?) -> S?, +): JavaDStream> = + toPairDStream() + .updateStateByKey( + { list: List, s: Optional -> + updateFunc(list, s.getOrNull()).toOptional() + }, + 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 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. + * 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 + */ +@JvmName("updateStateByKeyNullable") +fun JavaDStream>.updateStateByKey( + partitioner: Partitioner, + initialRDD: JavaRDD>, + updateFunc: (List, S?) -> S?, +): JavaDStream> = + toPairDStream() + .updateStateByKey( + { list: List, s: Optional -> + updateFunc(list, s.getOrNull()).toOptional() + }, + partitioner, + initialRDD.toPairRDD(), + ) + .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 + * 'this' DStream without changing the key. + */ +fun JavaDStream>.mapValues( + mapValuesFunc: (V) -> U, +): JavaDStream> = + toPairDStream() + .mapValues(mapValuesFunc) + .toTupleDStream() + +/** + * 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 JavaDStream>.flatMapValues( + flatMapValuesFunc: (V) -> Iterator, +): JavaDStream> = + toPairDStream() + .flatMapValues(flatMapValuesFunc) + .toTupleDStream() + +/** + * 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 JavaDStream>.cogroup( + other: JavaDStream>, + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +): JavaDStream, Iterable>>> = + toPairDStream() + .cogroup( + other.toPairDStream(), + numPartitions, + ) + .toTupleDStream() + + +/** + * 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 JavaDStream>.cogroup( + other: JavaDStream>, + partitioner: Partitioner, +): JavaDStream, Iterable>>> = + toPairDStream() + .cogroup( + other.toPairDStream(), + partitioner, + ) + .toTupleDStream() + +/** + * 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 JavaDStream>.join( + other: JavaDStream>, + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +): JavaDStream>> = + toPairDStream() + .join( + other.toPairDStream(), + numPartitions, + ) + .toTupleDStream() + +/** + * 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 JavaDStream>.join( + other: JavaDStream>, + partitioner: Partitioner, +): JavaDStream>> = + toPairDStream() + .join( + other.toPairDStream(), + partitioner, + ) + .toTupleDStream() + +/** + * 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 JavaDStream>.leftOuterJoin( + other: JavaDStream>, + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +): JavaDStream>>> = + toPairDStream() + .leftOuterJoin( + other.toPairDStream(), + numPartitions, + ) + .toTupleDStream() + +/** + * 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 JavaDStream>.leftOuterJoin( + other: JavaDStream>, + partitioner: Partitioner, +): JavaDStream>>> = + toPairDStream() + .leftOuterJoin( + other.toPairDStream(), + partitioner, + ) + .toTupleDStream() + +/** + * 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 JavaDStream>.rightOuterJoin( + other: JavaDStream>, + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +): JavaDStream, W>>> = + toPairDStream() + .rightOuterJoin( + other.toPairDStream(), + numPartitions, + ) + .toTupleDStream() + +/** + * 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 JavaDStream>.rightOuterJoin( + other: JavaDStream>, + partitioner: Partitioner, +): JavaDStream, W>>> = + toPairDStream() + .rightOuterJoin( + other.toPairDStream(), + partitioner, + ) + .toTupleDStream() + +/** + * 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 JavaDStream>.fullOuterJoin( + other: JavaDStream>, + numPartitions: Int = dstream().ssc().sc().defaultParallelism(), +): JavaDStream, Optional>>> = + toPairDStream() + .fullOuterJoin( + other.toPairDStream(), + numPartitions, + ) + .toTupleDStream() + +/** + * 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 JavaDStream>.fullOuterJoin( + other: JavaDStream>, + partitioner: Partitioner, +): JavaDStream, Optional>>> = + toPairDStream() + .fullOuterJoin( + other.toPairDStream(), + partitioner, + ) + .toTupleDStream() + +/** + * 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 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". + */ +fun JavaDStream>.saveAsNewAPIHadoopFiles( + prefix: String, + suffix: String, +): Unit = toPairDStream().saveAsNewAPIHadoopFiles(prefix, suffix) diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/UDFRegister.kt b/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/UDFRegister.kt similarity index 86% rename from kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/UDFRegister.kt rename to kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/UDFRegister.kt index 6dc19d58..05d72675 100644 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/UDFRegister.kt +++ b/kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/UDFRegister.kt @@ -17,6 +17,11 @@ * limitations under the License. * =LICENSEEND= */ + +/** + * This file contains functions to register UDFs easily from Kotlin. + */ + @file:Suppress("DuplicatedCode") package org.jetbrains.kotlinx.spark.api @@ -32,28 +37,34 @@ import kotlin.reflect.KClass import kotlin.reflect.full.isSubclassOf import kotlin.reflect.typeOf -fun DataType.unWrapper(): DataType { - return when (this) { +/** Unwraps [DataTypeWithClass]. */ +fun DataType.unWrap(): DataType = + when (this) { is DataTypeWithClass -> DataType.fromJson(dt().json()) else -> this } -} /** - * Checks if [this] is of a valid type for an UDF, otherwise it throws a [TypeOfUDFParameterNotSupportedException] + * Checks if [this] is of a valid type for a UDF, otherwise it throws a [TypeOfUDFParameterNotSupportedException] */ @PublishedApi internal fun KClass<*>.checkForValidType(parameterName: String) { - if (this == String::class || isSubclassOf(WrappedArray::class)) return // Most of the time we need strings or WrappedArrays - if (isSubclassOf(Iterable::class) || java.isArray - || isSubclassOf(Map::class) || isSubclassOf(Array::class) - || isSubclassOf(ByteArray::class) || isSubclassOf(CharArray::class) - || isSubclassOf(ShortArray::class) || isSubclassOf(IntArray::class) - || isSubclassOf(LongArray::class) || isSubclassOf(FloatArray::class) - || isSubclassOf(DoubleArray::class) || isSubclassOf(BooleanArray::class) - ) { - throw TypeOfUDFParameterNotSupportedException(this, parameterName) - } + if (this == String::class || isSubclassOf(WrappedArray::class)) + return // Most of the time we need strings or WrappedArrays + + if (isSubclassOf(Iterable::class) + || java.isArray + || isSubclassOf(Map::class) + || isSubclassOf(Array::class) + || isSubclassOf(ByteArray::class) + || isSubclassOf(CharArray::class) + || isSubclassOf(ShortArray::class) + || isSubclassOf(IntArray::class) + || isSubclassOf(LongArray::class) + || isSubclassOf(FloatArray::class) + || isSubclassOf(DoubleArray::class) + || isSubclassOf(BooleanArray::class) + ) throw TypeOfUDFParameterNotSupportedException(this, parameterName) } /** @@ -64,7 +75,7 @@ class TypeOfUDFParameterNotSupportedException(kClass: KClass<*>, parameterName: ) /** - * A wrapper for an UDF with 0 arguments. + * A wrapper for a UDF with 0 arguments. * @property udfName the name of the UDF */ class UDFWrapper0(private val udfName: String) { @@ -77,16 +88,16 @@ class UDFWrapper0(private val udfName: String) { } /** - * Registers the [func] with its [name] in [this] + * Registers the [func] with its [name] in [this]. */ @OptIn(ExperimentalStdlibApi::class) inline fun UDFRegistration.register(name: String, noinline func: () -> R): UDFWrapper0 { - register(name, UDF0(func), schema(typeOf()).unWrapper()) + register(name, UDF0(func), schema(typeOf()).unWrap()) return UDFWrapper0(name) } /** - * A wrapper for an UDF with 1 arguments. + * A wrapper for a UDF with 1 arguments. * @property udfName the name of the UDF */ class UDFWrapper1(private val udfName: String) { @@ -99,17 +110,17 @@ class UDFWrapper1(private val udfName: String) { } /** - * Registers the [func] with its [name] in [this] + * Registers the [func] with its [name] in [this]. */ @OptIn(ExperimentalStdlibApi::class) inline fun UDFRegistration.register(name: String, noinline func: (T0) -> R): UDFWrapper1 { T0::class.checkForValidType("T0") - register(name, UDF1(func), schema(typeOf()).unWrapper()) + register(name, UDF1(func), schema(typeOf()).unWrap()) return UDFWrapper1(name) } /** - * A wrapper for an UDF with 2 arguments. + * A wrapper for a UDF with 2 arguments. * @property udfName the name of the UDF */ class UDFWrapper2(private val udfName: String) { @@ -122,21 +133,21 @@ class UDFWrapper2(private val udfName: String) { } /** - * Registers the [func] with its [name] in [this] + * Registers the [func] with its [name] in [this]. */ @OptIn(ExperimentalStdlibApi::class) inline fun UDFRegistration.register( name: String, - noinline func: (T0, T1) -> R + noinline func: (T0, T1) -> R, ): UDFWrapper2 { T0::class.checkForValidType("T0") T1::class.checkForValidType("T1") - register(name, UDF2(func), schema(typeOf()).unWrapper()) + register(name, UDF2(func), schema(typeOf()).unWrap()) return UDFWrapper2(name) } /** - * A wrapper for an UDF with 3 arguments. + * A wrapper for a UDF with 3 arguments. * @property udfName the name of the UDF */ class UDFWrapper3(private val udfName: String) { @@ -149,22 +160,22 @@ class UDFWrapper3(private val udfName: String) { } /** - * Registers the [func] with its [name] in [this] + * Registers the [func] with its [name] in [this]. */ @OptIn(ExperimentalStdlibApi::class) inline fun UDFRegistration.register( name: String, - noinline func: (T0, T1, T2) -> R + noinline func: (T0, T1, T2) -> R, ): UDFWrapper3 { T0::class.checkForValidType("T0") T1::class.checkForValidType("T1") T2::class.checkForValidType("T2") - register(name, UDF3(func), schema(typeOf()).unWrapper()) + register(name, UDF3(func), schema(typeOf()).unWrap()) return UDFWrapper3(name) } /** - * A wrapper for an UDF with 4 arguments. + * A wrapper for a UDF with 4 arguments. * @property udfName the name of the UDF */ class UDFWrapper4(private val udfName: String) { @@ -177,23 +188,23 @@ class UDFWrapper4(private val udfName: String) { } /** - * Registers the [func] with its [name] in [this] + * Registers the [func] with its [name] in [this]. */ @OptIn(ExperimentalStdlibApi::class) inline fun UDFRegistration.register( name: String, - noinline func: (T0, T1, T2, T3) -> R + noinline func: (T0, T1, T2, T3) -> R, ): UDFWrapper4 { T0::class.checkForValidType("T0") T1::class.checkForValidType("T1") T2::class.checkForValidType("T2") T3::class.checkForValidType("T3") - register(name, UDF4(func), schema(typeOf()).unWrapper()) + register(name, UDF4(func), schema(typeOf()).unWrap()) return UDFWrapper4(name) } /** - * A wrapper for an UDF with 5 arguments. + * A wrapper for a UDF with 5 arguments. * @property udfName the name of the UDF */ class UDFWrapper5(private val udfName: String) { @@ -206,24 +217,24 @@ class UDFWrapper5(private val udfName: String) { } /** - * Registers the [func] with its [name] in [this] + * Registers the [func] with its [name] in [this]. */ @OptIn(ExperimentalStdlibApi::class) inline fun UDFRegistration.register( name: String, - noinline func: (T0, T1, T2, T3, T4) -> R + noinline func: (T0, T1, T2, T3, T4) -> R, ): UDFWrapper5 { T0::class.checkForValidType("T0") T1::class.checkForValidType("T1") T2::class.checkForValidType("T2") T3::class.checkForValidType("T3") T4::class.checkForValidType("T4") - register(name, UDF5(func), schema(typeOf()).unWrapper()) + register(name, UDF5(func), schema(typeOf()).unWrap()) return UDFWrapper5(name) } /** - * A wrapper for an UDF with 6 arguments. + * A wrapper for a UDF with 6 arguments. * @property udfName the name of the UDF */ class UDFWrapper6(private val udfName: String) { @@ -236,19 +247,19 @@ class UDFWrapper6(private val udfName: String) { param2: Column, param3: Column, param4: Column, - param5: Column + param5: Column, ): Column { return functions.callUDF(udfName, param0, param1, param2, param3, param4, param5) } } /** - * Registers the [func] with its [name] in [this] + * Registers the [func] with its [name] in [this]. */ @OptIn(ExperimentalStdlibApi::class) inline fun UDFRegistration.register( name: String, - noinline func: (T0, T1, T2, T3, T4, T5) -> R + noinline func: (T0, T1, T2, T3, T4, T5) -> R, ): UDFWrapper6 { T0::class.checkForValidType("T0") T1::class.checkForValidType("T1") @@ -256,12 +267,12 @@ inline fun ()).unWrapper()) + register(name, UDF6(func), schema(typeOf()).unWrap()) return UDFWrapper6(name) } /** - * A wrapper for an UDF with 7 arguments. + * A wrapper for a UDF with 7 arguments. * @property udfName the name of the UDF */ class UDFWrapper7(private val udfName: String) { @@ -275,19 +286,19 @@ class UDFWrapper7(private val udfName: String) { param3: Column, param4: Column, param5: Column, - param6: Column + param6: Column, ): Column { return functions.callUDF(udfName, param0, param1, param2, param3, param4, param5, param6) } } /** - * Registers the [func] with its [name] in [this] + * Registers the [func] with its [name] in [this]. */ @OptIn(ExperimentalStdlibApi::class) inline fun UDFRegistration.register( name: String, - noinline func: (T0, T1, T2, T3, T4, T5, T6) -> R + noinline func: (T0, T1, T2, T3, T4, T5, T6) -> R, ): UDFWrapper7 { T0::class.checkForValidType("T0") T1::class.checkForValidType("T1") @@ -296,12 +307,12 @@ inline fun ()).unWrapper()) + register(name, UDF7(func), schema(typeOf()).unWrap()) return UDFWrapper7(name) } /** - * A wrapper for an UDF with 8 arguments. + * A wrapper for a UDF with 8 arguments. * @property udfName the name of the UDF */ class UDFWrapper8(private val udfName: String) { @@ -316,19 +327,19 @@ class UDFWrapper8(private val udfName: String) { param4: Column, param5: Column, param6: Column, - param7: Column + param7: Column, ): Column { return functions.callUDF(udfName, param0, param1, param2, param3, param4, param5, param6, param7) } } /** - * Registers the [func] with its [name] in [this] + * Registers the [func] with its [name] in [this]. */ @OptIn(ExperimentalStdlibApi::class) inline fun UDFRegistration.register( name: String, - noinline func: (T0, T1, T2, T3, T4, T5, T6, T7) -> R + noinline func: (T0, T1, T2, T3, T4, T5, T6, T7) -> R, ): UDFWrapper8 { T0::class.checkForValidType("T0") T1::class.checkForValidType("T1") @@ -338,12 +349,12 @@ inline fun ()).unWrapper()) + register(name, UDF8(func), schema(typeOf()).unWrap()) return UDFWrapper8(name) } /** - * A wrapper for an UDF with 9 arguments. + * A wrapper for a UDF with 9 arguments. * @property udfName the name of the UDF */ class UDFWrapper9(private val udfName: String) { @@ -359,19 +370,19 @@ class UDFWrapper9(private val udfName: String) { param5: Column, param6: Column, param7: Column, - param8: Column + param8: Column, ): Column { return functions.callUDF(udfName, param0, param1, param2, param3, param4, param5, param6, param7, param8) } } /** - * Registers the [func] with its [name] in [this] + * Registers the [func] with its [name] in [this]. */ @OptIn(ExperimentalStdlibApi::class) inline fun UDFRegistration.register( name: String, - noinline func: (T0, T1, T2, T3, T4, T5, T6, T7, T8) -> R + noinline func: (T0, T1, T2, T3, T4, T5, T6, T7, T8) -> R, ): UDFWrapper9 { T0::class.checkForValidType("T0") T1::class.checkForValidType("T1") @@ -382,12 +393,12 @@ inline fun ()).unWrapper()) + register(name, UDF9(func), schema(typeOf()).unWrap()) return UDFWrapper9(name) } /** - * A wrapper for an UDF with 10 arguments. + * A wrapper for a UDF with 10 arguments. * @property udfName the name of the UDF */ class UDFWrapper10(private val udfName: String) { @@ -404,7 +415,7 @@ class UDFWrapper10(private val udfName: String) { param6: Column, param7: Column, param8: Column, - param9: Column + param9: Column, ): Column { return functions.callUDF( udfName, @@ -423,12 +434,12 @@ class UDFWrapper10(private val udfName: String) { } /** - * Registers the [func] with its [name] in [this] + * Registers the [func] with its [name] in [this]. */ @OptIn(ExperimentalStdlibApi::class) inline fun UDFRegistration.register( name: String, - noinline func: (T0, T1, T2, T3, T4, T5, T6, T7, T8, T9) -> R + noinline func: (T0, T1, T2, T3, T4, T5, T6, T7, T8, T9) -> R, ): UDFWrapper10 { T0::class.checkForValidType("T0") T1::class.checkForValidType("T1") @@ -440,12 +451,12 @@ inline fun ()).unWrapper()) + register(name, UDF10(func), schema(typeOf()).unWrap()) return UDFWrapper10(name) } /** - * A wrapper for an UDF with 11 arguments. + * A wrapper for a UDF with 11 arguments. * @property udfName the name of the UDF */ class UDFWrapper11(private val udfName: String) { @@ -463,7 +474,7 @@ class UDFWrapper11(private val udfName: String) { param7: Column, param8: Column, param9: Column, - param10: Column + param10: Column, ): Column { return functions.callUDF( udfName, @@ -483,12 +494,12 @@ class UDFWrapper11(private val udfName: String) { } /** - * Registers the [func] with its [name] in [this] + * Registers the [func] with its [name] in [this]. */ @OptIn(ExperimentalStdlibApi::class) inline fun UDFRegistration.register( name: String, - noinline func: (T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10) -> R + noinline func: (T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10) -> R, ): UDFWrapper11 { T0::class.checkForValidType("T0") T1::class.checkForValidType("T1") @@ -501,12 +512,12 @@ inline fun ()).unWrapper()) + register(name, UDF11(func), schema(typeOf()).unWrap()) return UDFWrapper11(name) } /** - * A wrapper for an UDF with 12 arguments. + * A wrapper for a UDF with 12 arguments. * @property udfName the name of the UDF */ class UDFWrapper12(private val udfName: String) { @@ -525,7 +536,7 @@ class UDFWrapper12(private val udfName: String) { param8: Column, param9: Column, param10: Column, - param11: Column + param11: Column, ): Column { return functions.callUDF( udfName, @@ -546,12 +557,12 @@ class UDFWrapper12(private val udfName: String) { } /** - * Registers the [func] with its [name] in [this] + * Registers the [func] with its [name] in [this]. */ @OptIn(ExperimentalStdlibApi::class) inline fun UDFRegistration.register( name: String, - noinline func: (T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11) -> R + noinline func: (T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11) -> R, ): UDFWrapper12 { T0::class.checkForValidType("T0") T1::class.checkForValidType("T1") @@ -565,12 +576,12 @@ inline fun ()).unWrapper()) + register(name, UDF12(func), schema(typeOf()).unWrap()) return UDFWrapper12(name) } /** - * A wrapper for an UDF with 13 arguments. + * A wrapper for a UDF with 13 arguments. * @property udfName the name of the UDF */ class UDFWrapper13(private val udfName: String) { @@ -590,7 +601,7 @@ class UDFWrapper13(private val udfName: String) { param9: Column, param10: Column, param11: Column, - param12: Column + param12: Column, ): Column { return functions.callUDF( udfName, @@ -612,12 +623,12 @@ class UDFWrapper13(private val udfName: String) { } /** - * Registers the [func] with its [name] in [this] + * Registers the [func] with its [name] in [this]. */ @OptIn(ExperimentalStdlibApi::class) inline fun UDFRegistration.register( name: String, - noinline func: (T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12) -> R + noinline func: (T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12) -> R, ): UDFWrapper13 { T0::class.checkForValidType("T0") T1::class.checkForValidType("T1") @@ -632,12 +643,12 @@ inline fun ()).unWrapper()) + register(name, UDF13(func), schema(typeOf()).unWrap()) return UDFWrapper13(name) } /** - * A wrapper for an UDF with 14 arguments. + * A wrapper for a UDF with 14 arguments. * @property udfName the name of the UDF */ class UDFWrapper14(private val udfName: String) { @@ -658,7 +669,7 @@ class UDFWrapper14(private val udfName: String) { param10: Column, param11: Column, param12: Column, - param13: Column + param13: Column, ): Column { return functions.callUDF( udfName, @@ -681,12 +692,12 @@ class UDFWrapper14(private val udfName: String) { } /** - * Registers the [func] with its [name] in [this] + * Registers the [func] with its [name] in [this]. */ @OptIn(ExperimentalStdlibApi::class) inline fun UDFRegistration.register( name: String, - noinline func: (T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13) -> R + noinline func: (T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13) -> R, ): UDFWrapper14 { T0::class.checkForValidType("T0") T1::class.checkForValidType("T1") @@ -702,12 +713,12 @@ inline fun ()).unWrapper()) + register(name, UDF14(func), schema(typeOf()).unWrap()) return UDFWrapper14(name) } /** - * A wrapper for an UDF with 15 arguments. + * A wrapper for a UDF with 15 arguments. * @property udfName the name of the UDF */ class UDFWrapper15(private val udfName: String) { @@ -729,7 +740,7 @@ class UDFWrapper15(private val udfName: String) { param11: Column, param12: Column, param13: Column, - param14: Column + param14: Column, ): Column { return functions.callUDF( udfName, @@ -753,12 +764,12 @@ class UDFWrapper15(private val udfName: String) { } /** - * Registers the [func] with its [name] in [this] + * Registers the [func] with its [name] in [this]. */ @OptIn(ExperimentalStdlibApi::class) inline fun UDFRegistration.register( name: String, - noinline func: (T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14) -> R + noinline func: (T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14) -> R, ): UDFWrapper15 { T0::class.checkForValidType("T0") T1::class.checkForValidType("T1") @@ -775,12 +786,12 @@ inline fun ()).unWrapper()) + register(name, UDF15(func), schema(typeOf()).unWrap()) return UDFWrapper15(name) } /** - * A wrapper for an UDF with 16 arguments. + * A wrapper for a UDF with 16 arguments. * @property udfName the name of the UDF */ class UDFWrapper16(private val udfName: String) { @@ -803,7 +814,7 @@ class UDFWrapper16(private val udfName: String) { param12: Column, param13: Column, param14: Column, - param15: Column + param15: Column, ): Column { return functions.callUDF( udfName, @@ -828,12 +839,12 @@ class UDFWrapper16(private val udfName: String) { } /** - * Registers the [func] with its [name] in [this] + * Registers the [func] with its [name] in [this]. */ @OptIn(ExperimentalStdlibApi::class) inline fun UDFRegistration.register( name: String, - noinline func: (T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15) -> R + noinline func: (T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15) -> R, ): UDFWrapper16 { T0::class.checkForValidType("T0") T1::class.checkForValidType("T1") @@ -851,12 +862,12 @@ inline fun ()).unWrapper()) + register(name, UDF16(func), schema(typeOf()).unWrap()) return UDFWrapper16(name) } /** - * A wrapper for an UDF with 17 arguments. + * A wrapper for a UDF with 17 arguments. * @property udfName the name of the UDF */ class UDFWrapper17(private val udfName: String) { @@ -880,7 +891,7 @@ class UDFWrapper17(private val udfName: String) { param13: Column, param14: Column, param15: Column, - param16: Column + param16: Column, ): Column { return functions.callUDF( udfName, @@ -906,12 +917,12 @@ class UDFWrapper17(private val udfName: String) { } /** - * Registers the [func] with its [name] in [this] + * Registers the [func] with its [name] in [this]. */ @OptIn(ExperimentalStdlibApi::class) inline fun UDFRegistration.register( name: String, - noinline func: (T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16) -> R + noinline func: (T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16) -> R, ): UDFWrapper17 { T0::class.checkForValidType("T0") T1::class.checkForValidType("T1") @@ -930,12 +941,12 @@ inline fun ()).unWrapper()) + register(name, UDF17(func), schema(typeOf()).unWrap()) return UDFWrapper17(name) } /** - * A wrapper for an UDF with 18 arguments. + * A wrapper for a UDF with 18 arguments. * @property udfName the name of the UDF */ class UDFWrapper18(private val udfName: String) { @@ -960,7 +971,7 @@ class UDFWrapper18(private val udfName: String) { param14: Column, param15: Column, param16: Column, - param17: Column + param17: Column, ): Column { return functions.callUDF( udfName, @@ -987,12 +998,12 @@ class UDFWrapper18(private val udfName: String) { } /** - * Registers the [func] with its [name] in [this] + * Registers the [func] with its [name] in [this]. */ @OptIn(ExperimentalStdlibApi::class) inline fun UDFRegistration.register( name: String, - noinline func: (T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17) -> R + noinline func: (T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17) -> R, ): UDFWrapper18 { T0::class.checkForValidType("T0") T1::class.checkForValidType("T1") @@ -1012,12 +1023,12 @@ inline fun ()).unWrapper()) + register(name, UDF18(func), schema(typeOf()).unWrap()) return UDFWrapper18(name) } /** - * A wrapper for an UDF with 19 arguments. + * A wrapper for a UDF with 19 arguments. * @property udfName the name of the UDF */ class UDFWrapper19(private val udfName: String) { @@ -1043,7 +1054,7 @@ class UDFWrapper19(private val udfName: String) { param15: Column, param16: Column, param17: Column, - param18: Column + param18: Column, ): Column { return functions.callUDF( udfName, @@ -1071,12 +1082,12 @@ class UDFWrapper19(private val udfName: String) { } /** - * Registers the [func] with its [name] in [this] + * Registers the [func] with its [name] in [this]. */ @OptIn(ExperimentalStdlibApi::class) inline fun UDFRegistration.register( name: String, - noinline func: (T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18) -> R + noinline func: (T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18) -> R, ): UDFWrapper19 { T0::class.checkForValidType("T0") T1::class.checkForValidType("T1") @@ -1097,12 +1108,12 @@ inline fun ()).unWrapper()) + register(name, UDF19(func), schema(typeOf()).unWrap()) return UDFWrapper19(name) } /** - * A wrapper for an UDF with 20 arguments. + * A wrapper for a UDF with 20 arguments. * @property udfName the name of the UDF */ class UDFWrapper20(private val udfName: String) { @@ -1129,7 +1140,7 @@ class UDFWrapper20(private val udfName: String) { param16: Column, param17: Column, param18: Column, - param19: Column + param19: Column, ): Column { return functions.callUDF( udfName, @@ -1158,12 +1169,12 @@ class UDFWrapper20(private val udfName: String) { } /** - * Registers the [func] with its [name] in [this] + * Registers the [func] with its [name] in [this]. */ @OptIn(ExperimentalStdlibApi::class) inline fun UDFRegistration.register( name: String, - noinline func: (T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19) -> R + noinline func: (T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19) -> R, ): UDFWrapper20 { T0::class.checkForValidType("T0") T1::class.checkForValidType("T1") @@ -1185,12 +1196,12 @@ inline fun ()).unWrapper()) + register(name, UDF20(func), schema(typeOf()).unWrap()) return UDFWrapper20(name) } /** - * A wrapper for an UDF with 21 arguments. + * A wrapper for a UDF with 21 arguments. * @property udfName the name of the UDF */ class UDFWrapper21(private val udfName: String) { @@ -1218,7 +1229,7 @@ class UDFWrapper21(private val udfName: String) { param17: Column, param18: Column, param19: Column, - param20: Column + param20: Column, ): Column { return functions.callUDF( udfName, @@ -1248,12 +1259,12 @@ class UDFWrapper21(private val udfName: String) { } /** - * Registers the [func] with its [name] in [this] + * Registers the [func] with its [name] in [this]. */ @OptIn(ExperimentalStdlibApi::class) inline fun UDFRegistration.register( name: String, - noinline func: (T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20) -> R + noinline func: (T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20) -> R, ): UDFWrapper21 { T0::class.checkForValidType("T0") T1::class.checkForValidType("T1") @@ -1276,12 +1287,12 @@ inline fun ()).unWrapper()) + register(name, UDF21(func), schema(typeOf()).unWrap()) return UDFWrapper21(name) } /** - * A wrapper for an UDF with 22 arguments. + * A wrapper for a UDF with 22 arguments. * @property udfName the name of the UDF */ class UDFWrapper22(private val udfName: String) { @@ -1310,7 +1321,7 @@ class UDFWrapper22(private val udfName: String) { param18: Column, param19: Column, param20: Column, - param21: Column + param21: Column, ): Column { return functions.callUDF( udfName, @@ -1341,12 +1352,12 @@ class UDFWrapper22(private val udfName: String) { } /** - * Registers the [func] with its [name] in [this] + * Registers the [func] with its [name] in [this]. */ @OptIn(ExperimentalStdlibApi::class) inline fun UDFRegistration.register( name: String, - noinline func: (T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21) -> R + noinline func: (T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21) -> R, ): UDFWrapper22 { T0::class.checkForValidType("T0") T1::class.checkForValidType("T1") @@ -1370,6 +1381,6 @@ inline fun ()).unWrapper()) + register(name, UDF22(func), schema(typeOf()).unWrap()) return UDFWrapper22(name) } 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 new file mode 100644 index 00000000..e95a65f7 --- /dev/null +++ b/kotlin-spark-api/3.0/src/test/kotlin/org/jetbrains/kotlinx/spark/api/ApiTest.kt @@ -0,0 +1,100 @@ +package org.jetbrains.kotlinx.spark.api/*- + * =LICENSE= + * Kotlin Spark API + * ---------- + * Copyright (C) 2019 - 2020 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= + */ +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.shouldBe +import scala.collection.Seq +import java.io.Serializable +import kotlin.collections.Iterator +import scala.collection.Iterator as ScalaIterator +import scala.collection.Map as ScalaMap +import scala.collection.mutable.Map as ScalaMutableMap + +class ApiTest : ShouldSpec({ + + context("miscellaneous integration tests") { + withSpark(props = mapOf("spark.sql.codegen.comments" to true)) { + + @OptIn(ExperimentalStdlibApi::class) + should("broadcast variables") { + val largeList = (1..15).map { SomeClass(a = (it..15).toList().toIntArray(), b = it) } + val broadcast = spark.broadcast(largeList) + val broadcast2 = spark.broadcast(arrayOf(doubleArrayOf(1.0, 2.0, 3.0, 4.0))) + + val result: List = listOf(1, 2, 3, 4, 5) + .toDS() + .mapPartitions { iterator -> + val receivedBroadcast = broadcast.value + val receivedBroadcast2 = broadcast2.value + + buildList { + iterator.forEach { + this.add(it + receivedBroadcast[it].b * receivedBroadcast2[0][0]) + } + }.iterator() + } + .collectAsList() + + expect(result).contains.inOrder.only.values(3.0, 5.0, 7.0, 9.0, 11.0) + } + + should("Handle JavaConversions in Kotlin") { + // Test the iterator conversion + val scalaIterator: ScalaIterator = listOf("test1", "test2").iterator().asScalaIterator() + scalaIterator.next() shouldBe "test1" + + val kotlinIterator: Iterator = scalaIterator.asKotlinIterator() + kotlinIterator.next() shouldBe "test2" + + + val scalaMap: ScalaMap = mapOf(1 to "a", 2 to "b").asScalaMap() + scalaMap.get(1).get() shouldBe "a" + scalaMap.get(2).get() shouldBe "b" + + val kotlinMap: Map = scalaMap.asKotlinMap() + kotlinMap[1] shouldBe "a" + kotlinMap[2] shouldBe "b" + + + val scalaMutableMap: ScalaMutableMap = mutableMapOf(1 to "a").asScalaMutableMap() + scalaMutableMap.get(1).get() shouldBe "a" + + scalaMutableMap.put(2, "b") + + val kotlinMutableMap: MutableMap = scalaMutableMap.asKotlinMutableMap() + kotlinMutableMap[1] shouldBe "a" + kotlinMutableMap[2] shouldBe "b" + + val scalaSeq: Seq = listOf("a", "b").iterator().asScalaIterator().toSeq() + scalaSeq.take(1).toList().last() shouldBe "a" + scalaSeq.take(2).toList().last() shouldBe "b" + + val kotlinList: List = scalaSeq.asKotlinList() + kotlinList.first() shouldBe "a" + kotlinList.last() shouldBe "b" + } + } + } +}) + + +// (data) class must be Serializable to be broadcast +data class SomeClass(val a: IntArray, val b: Int) : Serializable diff --git a/kotlin-spark-api/3.0/src/test/kotlin/org/jetbrains/kotlinx/spark/api/DatasetFunctionTest.kt b/kotlin-spark-api/3.0/src/test/kotlin/org/jetbrains/kotlinx/spark/api/DatasetFunctionTest.kt new file mode 100644 index 00000000..26dcceaf --- /dev/null +++ b/kotlin-spark-api/3.0/src/test/kotlin/org/jetbrains/kotlinx/spark/api/DatasetFunctionTest.kt @@ -0,0 +1,455 @@ +/*- + * =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.* +import ch.tutteli.atrium.api.verbs.expect +import io.kotest.core.spec.style.ShouldSpec +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.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.jetbrains.kotlinx.spark.api.tuples.* +import scala.Tuple2 +import scala.Tuple3 +import scala.Tuple4 +import scala.Tuple5 +import java.io.Serializable + +class DatasetFunctionTest : ShouldSpec({ + + context("dataset extensions") { + withSpark(props = mapOf("spark.sql.codegen.comments" to true)) { + + should("handle cached operations") { + val result = dsOf(1, 2, 3, 4, 5) + .map { it X (it + 2) } + .withCached { + expect(collectAsList()).contains.inAnyOrder.only.values( + 1 X 3, + 2 X 4, + 3 X 5, + 4 X 6, + 5 X 7, + ) + + val next = filter { it._1 % 2 == 0 } + expect(next.collectAsList()).contains.inAnyOrder.only.values(2 X 4, 4 X 6) + next + } + .map { it: Tuple2 -> + it + (it._1 + it._2) * 2 + } + .collectAsList() + expect(result).contains.inOrder.only.values(2 X 4 X 12, 4 X 6 X 20) + } + + should("handle join operations") { + data class Left(val id: Int, val name: String) + + data class Right(val id: Int, val value: Int) + + val first = dsOf(Left(1, "a"), Left(2, "b")) + val second = dsOf(Right(1, 100), Right(3, 300)) + val result = first + .leftJoin(second, first.col("id") eq second.col("id")) + .map { it._1.id X it._1.name X it._2?.value } + .collectAsList() + expect(result).contains.inOrder.only.values(t(1, "a", 100), t(2, "b", null)) + } + + should("handle map operations") { + val result = dsOf(listOf(1, 2, 3, 4), listOf(3, 4, 5, 6)) + .flatMap { it.iterator() } + .map { it + 4 } + .filter { it < 10 } + .collectAsList() + expect(result).contains.inAnyOrder.only.values(5, 6, 7, 8, 7, 8, 9) + } + + should("Allow simple forEachPartition in datasets") { + val dataset = dsOf( + SomeClass(intArrayOf(1, 2, 3), 1), + SomeClass(intArrayOf(4, 3, 2), 1), + ) + dataset.forEachPartition { + it.forEach { + it.b shouldBe 1 + } + } + } + + should("Have easier access to keys and values for key/value datasets") { + val dataset: Dataset = dsOf( + SomeClass(intArrayOf(1, 2, 3), 1), + SomeClass(intArrayOf(4, 3, 2), 1), + ) + .groupByKey { it.b } + .reduceGroupsK { a, b -> SomeClass(a.a + b.a, a.b) } + .takeValues() + + dataset.count() shouldBe 1 + } + + should("Be able to sort datasets with property reference") { + val dataset: Dataset = dsOf( + SomeClass(intArrayOf(1, 2, 3), 2), + SomeClass(intArrayOf(4, 3, 2), 1), + ) + dataset.sort(SomeClass::b) + dataset.takeAsList(1).first().b shouldBe 2 + + dataset.sort(SomeClass::a, SomeClass::b) + dataset.takeAsList(1).first().b shouldBe 2 + } + + should("Have Kotlin ready functions in place of overload ambiguity") { + val dataset: Tuple2 = dsOf( + SomeClass(intArrayOf(1, 2, 3), 1), + SomeClass(intArrayOf(4, 3, 2), 1), + ) + .groupByKey { it: SomeClass -> it.b } + .reduceGroupsK { v1: SomeClass, v2: SomeClass -> v1 } + .filter { it: Tuple2 -> true } // not sure why this does work, but reduce doesn't + .reduceK { v1: Tuple2, v2: Tuple2 -> v1 } + + dataset._2.a shouldBe intArrayOf(1, 2, 3) + } + } + } + + context("grouped dataset extensions") { + withSpark(props = mapOf("spark.sql.codegen.comments" to true)) { + + should("perform flat map on grouped datasets") { + val groupedDataset = listOf(t(1, "a"), t(1, "b"), t(2, "c")) + .toDS() + .groupByKey { it._1 } + + val flatMapped = groupedDataset.flatMapGroups { key, values -> + val collected = values.asSequence().toList() + + if (collected.size > 1) collected.iterator() + else emptyList>().iterator() + } + + flatMapped.count() shouldBe 2 + } + + should("perform map group with state and timeout conf on grouped datasets") { + val groupedDataset = listOf(t(1, "a"), t(1, "b"), t(2, "c")) + .toDS() + .groupByKey { it._1 } + + val mappedWithStateTimeoutConf = + groupedDataset.mapGroupsWithState(GroupStateTimeout.NoTimeout()) { key, values, state: GroupState -> + var s by state + val collected = values.asSequence().toList() + + s = key + s shouldBe key + + s!! X collected.map { it._2 } + } + + mappedWithStateTimeoutConf.count() shouldBe 2 + } + + should("perform map group with state on grouped datasets") { + val groupedDataset = listOf(t(1, "a"), t(1, "b"), t(2, "c")) + .toDS() + .groupByKey { it._1 } + + val mappedWithState = groupedDataset.mapGroupsWithState { key, values, state: GroupState -> + var s by state + val collected = values.asSequence().toList() + + s = key + s shouldBe key + + s!! X collected.map { it._2 } + } + + mappedWithState.count() shouldBe 2 + } + + should("perform flat map group with state on grouped datasets") { + val groupedDataset = listOf(t(1, "a"), t(1, "b"), t(2, "c")) + .toDS() + .groupByKey { it._1 } + + val flatMappedWithState = groupedDataset.mapGroupsWithState { key, values, state: GroupState -> + var s by state + val collected = values.asSequence().toList() + + s = key + s shouldBe key + + if (collected.size > 1) collected.iterator() + else emptyList>().iterator() + } + + flatMappedWithState.count() shouldBe 2 + } + + should("be able to cogroup grouped datasets") { + val groupedDataset1 = listOf(1 X "a", 1 X "b", 2 X "c") + .toDS() + .groupByKey { it._1 } + + val groupedDataset2 = listOf(1 X "d", 5 X "e", 3 X "f") + .toDS() + .groupByKey { it._1 } + + val cogrouped = groupedDataset1.cogroup(groupedDataset2) { key, left, right -> + listOf( + key to (left.asSequence() + right.asSequence()).map { it._2 }.toList() + ).iterator() + } + + cogrouped.count() shouldBe 4 + } + } + } + + context("RDD conversions") { + withSpark(props = mapOf("spark.sql.codegen.comments" to true)) { + + should("Convert Scala RDD to Dataset") { + val rdd0: RDD = sc.parallelize( + listOf(1, 2, 3, 4, 5, 6) + ).rdd() + val dataset0: Dataset = rdd0.toDS() + + dataset0.toList() shouldBe listOf(1, 2, 3, 4, 5, 6) + } + + should("Convert a JavaRDD to a Dataset") { + val rdd1: JavaRDD = sc.parallelize( + listOf(1, 2, 3, 4, 5, 6) + ) + val dataset1: Dataset = rdd1.toDS() + + dataset1.toList() shouldBe listOf(1, 2, 3, 4, 5, 6) + } + + should("Convert JavaDoubleRDD to Dataset") { + + // JavaDoubleRDD + val rdd2: JavaDoubleRDD = sc.parallelizeDoubles( + listOf(1.0, 2.0, 3.0, 4.0, 5.0, 6.0) + ) + val dataset2: Dataset = rdd2.toDS() + + dataset2.toList() shouldBe listOf(1.0, 2.0, 3.0, 4.0, 5.0, 6.0) + } + + should("Convert JavaPairRDD to Dataset") { + val rdd3: JavaPairRDD = sc.parallelizePairs( + listOf(t(1, 1.0), t(2, 2.0), t(3, 3.0)) + ) + val dataset3: Dataset> = rdd3.toDS() + + dataset3.toList>() shouldBe listOf(t(1, 1.0), t(2, 2.0), t(3, 3.0)) + } + + should("Convert Kotlin Serializable data class RDD to Dataset") { + val rdd4 = sc.parallelize( + listOf(SomeClass(intArrayOf(1, 2), 0)) + ) + val dataset4 = rdd4.toDS() + + dataset4.toList().first().let { (a, b) -> + a contentEquals intArrayOf(1, 2) shouldBe true + b shouldBe 0 + } + } + + should("Convert Tuple RDD to Dataset") { + val rdd5 = sc.parallelize( + listOf(t(1.0, 4)) + ) + val dataset5 = rdd5.toDS() + + dataset5.toList>() shouldBe listOf(t(1.0, 4)) + } + + should("Convert List RDD to Dataset") { + val rdd6 = sc.parallelize( + listOf(listOf(1, 2, 3), listOf(4, 5, 6)) + ) + val dataset6 = rdd6.toDS() + + dataset6.toList>() shouldBe listOf(listOf(1, 2, 3), listOf(4, 5, 6)) + } + + should("Sort Tuple2 Dataset") { + val list = listOf( + t(1, 6), + t(2, 5), + t(3, 4), + ) + val dataset = list.toDS() + + dataset.sortByKey().collectAsList() shouldBe list.sortedBy { it._1 } + dataset.sortByValue().collectAsList() shouldBe list.sortedBy { it._2 } + } + + should("Sort Pair Dataset") { + val list = listOf( + Pair(1, 6), + Pair(2, 5), + Pair(3, 4), + ) + val dataset = list.toDS() + + dataset.sortByKey().collectAsList() shouldBe list.sortedBy { it.first } + dataset.sortByValue().collectAsList() shouldBe list.sortedBy { it.second } + } + } + } + + context("Column functions") { + withSpark(props = mapOf("spark.sql.codegen.comments" to true)) { + + @Suppress("UNCHECKED_CAST") + should("support dataset select") { + val dataset = dsOf( + SomeClass(intArrayOf(1, 2, 3), 3), + SomeClass(intArrayOf(1, 2, 4), 5), + ) + + val newDS1WithAs: Dataset = dataset.selectTyped( + functions.col("a").`as`(), + ) + newDS1WithAs.collectAsList() + + 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.collectAsList() + + val newDS3: Dataset> = dataset.selectTyped( + col(SomeClass::a), + col(SomeClass::b), + col(SomeClass::b), + ) + newDS3.collectAsList() + + val newDS4: Dataset> = dataset.selectTyped( + col(SomeClass::a), + col(SomeClass::b), + col(SomeClass::b), + col(SomeClass::b), + ) + newDS4.collectAsList() + + val newDS5: Dataset> = dataset.selectTyped( + col(SomeClass::a), + col(SomeClass::b), + col(SomeClass::b), + col(SomeClass::b), + col(SomeClass::b), + ) + newDS5.collectAsList() + } + + should("Access columns using invoke on datasets") { + val dataset = dsOf( + SomeClass(intArrayOf(1, 2, 3), 4), + SomeClass(intArrayOf(4, 3, 2), 1), + ) + + dataset.col("a") shouldBe dataset("a") + } + + should("Use infix- and operator funs on columns") { + val dataset = dsOf( + SomeOtherClass(intArrayOf(1, 2, 3), 4, true), + SomeOtherClass(intArrayOf(4, 3, 2), 1, true), + ) + + (dataset("a") == dataset("a")) shouldBe dataset("a").equals(dataset("a")) + (dataset("a") != dataset("a")) shouldBe !dataset("a").equals(dataset("a")) + (dataset("a") eq dataset("a")) shouldBe dataset("a").equalTo(dataset("a")) + dataset("a").equalTo(dataset("a")) shouldBe (dataset("a") `===` dataset("a")) + (dataset("a") neq dataset("a")) shouldBe dataset("a").notEqual(dataset("a")) + dataset("a").notEqual(dataset("a")) shouldBe (dataset("a") `=!=` dataset("a")) + !(dataset("a") eq dataset("a")) shouldBe dataset("a").notEqual(dataset("a")) + dataset("a").notEqual(dataset("a")) shouldBe (!(dataset("a") `===` dataset("a"))) + -dataset("b") shouldBe functions.negate(dataset("b")) + !dataset("c") shouldBe functions.not(dataset("c")) + dataset("b") gt 3 shouldBe dataset("b").gt(3) + dataset("b") lt 3 shouldBe dataset("b").lt(3) + dataset("b") leq 3 shouldBe dataset("b").leq(3) + dataset("b") geq 3 shouldBe dataset("b").geq(3) + dataset("b") inRangeOf 0..2 shouldBe dataset("b").between(0, 2) + dataset("c") or dataset("c") shouldBe dataset("c").or(dataset("c")) + dataset("c") and dataset("c") shouldBe dataset("c").and(dataset("c")) + dataset("c").and(dataset("c")) shouldBe (dataset("c") `&&` dataset("c")) + dataset("b") + dataset("b") shouldBe dataset("b").plus(dataset("b")) + dataset("b") - dataset("b") shouldBe dataset("b").minus(dataset("b")) + dataset("b") * dataset("b") shouldBe dataset("b").multiply(dataset("b")) + dataset("b") / dataset("b") shouldBe dataset("b").divide(dataset("b")) + dataset("b") % dataset("b") shouldBe dataset("b").mod(dataset("b")) + dataset("b")[0] shouldBe dataset("b").getItem(0) + } + + should("Handle TypedColumns") { + val dataset = dsOf( + SomeOtherClass(intArrayOf(1, 2, 3), 4, true), + SomeOtherClass(intArrayOf(4, 3, 2), 1, true), + ) + + // walking over all column creation methods + val b: Dataset> = dataset.select( + dataset.col(SomeOtherClass::b), + dataset(SomeOtherClass::a), + col(SomeOtherClass::c), + ) + b.collectAsList() + } + + should("Handle some where queries using column operator functions") { + val dataset = dsOf( + SomeOtherClass(intArrayOf(1, 2, 3), 4, true), + SomeOtherClass(intArrayOf(4, 3, 2), 1, true), + ) + dataset.collectAsList() + + val column = functions.col("b").`as`() + + val b = dataset.where(column gt 3 and col(SomeOtherClass::c)) + + b.count() shouldBe 1 + } + + + } + } +}) + +data class SomeOtherClass(val a: IntArray, val b: Int, val c: Boolean) : Serializable diff --git a/kotlin-spark-api/3.0/src/test/kotlin/org/jetbrains/kotlinx/spark/api/EncodingTest.kt b/kotlin-spark-api/3.0/src/test/kotlin/org/jetbrains/kotlinx/spark/api/EncodingTest.kt new file mode 100644 index 00000000..9d08e1fd --- /dev/null +++ b/kotlin-spark-api/3.0/src/test/kotlin/org/jetbrains/kotlinx/spark/api/EncodingTest.kt @@ -0,0 +1,528 @@ +/*- + * =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.* +import ch.tutteli.atrium.api.verbs.expect +import io.kotest.assertions.throwables.shouldThrow +import io.kotest.core.spec.style.ShouldSpec +import io.kotest.matchers.shouldBe +import org.apache.spark.sql.Dataset +import org.apache.spark.sql.types.Decimal +import org.apache.spark.unsafe.types.CalendarInterval +import org.jetbrains.kotlinx.spark.api.tuples.component1 +import org.jetbrains.kotlinx.spark.api.tuples.component2 +import org.jetbrains.kotlinx.spark.api.tuples.component3 +import org.jetbrains.kotlinx.spark.api.tuples.t +import org.jetbrains.kotlinx.spark.extensions.DemoCaseClass +import scala.Product +import scala.Some +import scala.Tuple2 +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 + +class EncodingTest : ShouldSpec({ + + context("encoders") { + withSpark(props = mapOf("spark.sql.codegen.comments" to true)) { + + should("handle LocalDate Datasets") { + val dates = listOf(LocalDate.now(), LocalDate.now()) + val dataset: Dataset = dates.toDS() + dataset.collectAsList() shouldBe dates + } + + should("handle Instant Datasets") { + val instants = listOf(Instant.now(), Instant.now()) + val dataset: Dataset = instants.toDS() + dataset.collectAsList().let { (first, second) -> + val (a, b) = instants + a.compareTo(first) shouldBe 0 + b.compareTo(second) shouldBe 0 + } + } + + should("handle Timestamp Datasets") { + val timeStamps = listOf(Timestamp(0L), Timestamp(1L)) + val dataset = timeStamps.toDS() + dataset.collectAsList() shouldBe timeStamps + } + + should("handle Duration Datasets") { + shouldThrow { + val dataset = dsOf(Duration.ZERO) + dataset.collectAsList() shouldBe listOf(Duration.ZERO) + } + } + + should("handle Period Datasets") { + shouldThrow { + val periods = listOf(Period.ZERO, Period.ofDays(2)) + val dataset = periods.toDS() + + dataset.show(false) + + dataset.collectAsList().let { + it[0] shouldBe Period.ZERO + + // NOTE Spark truncates java.time.Period to months. + it[1] shouldBe Period.ofDays(0) + } + } + } + + should("handle binary datasets") { + val byteArray = "Hello there".encodeToByteArray() + val dataset = dsOf(byteArray) + dataset.collectAsList() shouldBe listOf(byteArray) + } + + should("handle BigDecimal datasets") { + val decimals = listOf(BigDecimal.ONE, BigDecimal.TEN) + val dataset = decimals.toDS() + dataset.collectAsList().let { (one, ten) -> + one.compareTo(BigDecimal.ONE) shouldBe 0 + ten.compareTo(BigDecimal.TEN) shouldBe 0 + } + } + + should("handle nullable datasets") { + val ints = listOf(1, 2, 3, null) + val dataset = ints.toDS() + dataset.collectAsList() shouldBe ints + } + } + } + context("known dataTypes") { + withSpark(props = mapOf("spark.sql.codegen.comments" to true)) { + + should("be able to serialize Instant") { + val instantPair = Instant.now() to Instant.now() + val dataset = dsOf(instantPair) + dataset.collectAsList().single().let { (first, second) -> + val (a, b) = instantPair + a.compareTo(first) shouldBe 0 + b.compareTo(second) shouldBe 0 + } + } + + should("be able to serialize Date") { + val datePair = Date.valueOf("2020-02-10") to 5 + val dataset: Dataset> = dsOf(datePair) + dataset.collectAsList() shouldBe listOf(datePair) + } + + should("be able to serialize Timestamp") { + val timestampPair = Timestamp(0L) to 2 + val dataset = dsOf(timestampPair) + dataset.collectAsList() shouldBe listOf(timestampPair) + } + + should("be able to serialize binary") { + val byteArrayTriple = t("Hello there".encodeToByteArray(), 1, intArrayOf(1, 2, 3)) + val dataset = dsOf(byteArrayTriple) + + val (a, b, c) = dataset.collectAsList().single() + a contentEquals "Hello there".encodeToByteArray() shouldBe true + b shouldBe 1 + c contentEquals intArrayOf(1, 2, 3) shouldBe true + } + + should("be able to serialize Decimal") { + val decimalPair = t(Decimal().set(50), 12) + val dataset = dsOf(decimalPair) + dataset.collectAsList() shouldBe listOf(decimalPair) + } + + should("be able to serialize BigDecimal") { + val decimalPair = t(BigDecimal.TEN, 12) + val dataset = dsOf(decimalPair) + val (a, b) = dataset.collectAsList().single() + a.compareTo(BigDecimal.TEN) shouldBe 0 + b shouldBe 12 + } + + should("be able to serialize CalendarInterval") { + val calendarIntervalPair = CalendarInterval(1, 0, 0L) to 2 + val dataset = dsOf(calendarIntervalPair) + dataset.collectAsList() shouldBe listOf(calendarIntervalPair) + } + + should("Be able to serialize Scala Tuples including data classes") { + val dataset = dsOf( + t("a", t("a", 1, LonLat(1.0, 1.0))), + t("b", t("b", 2, LonLat(1.0, 2.0))), + ) + dataset.show() + val asList = dataset.takeAsList(2) + asList.first() shouldBe t("a", t("a", 1, LonLat(1.0, 1.0))) + } + + should("Be able to serialize data classes with tuples") { + val dataset = dsOf( + DataClassWithTuple(t(5L, "test", t(""))), + DataClassWithTuple(t(6L, "tessst", t(""))), + ) + + dataset.show() + val asList = dataset.takeAsList(2) + asList.first().tuple shouldBe t(5L, "test", t("")) + } + } + } + + context("schema") { + withSpark(props = mapOf("spark.sql.codegen.comments" to true)) { + + should("handle Scala Case class datasets") { + val caseClasses = listOf( + DemoCaseClass(1, "1"), + DemoCaseClass(2, "2"), + DemoCaseClass(3, "3"), + ) + val dataset = caseClasses.toDS() + dataset.show() + dataset.collectAsList() shouldBe caseClasses + } + + should("handle Scala Case class with data class datasets") { + val caseClasses = listOf( + DemoCaseClass(1, "1" to 1L), + DemoCaseClass(2, "2" to 2L), + DemoCaseClass(3, "3" to 3L), + ) + val dataset = caseClasses.toDS() + dataset.show() + dataset.collectAsList() shouldBe caseClasses + } + + should("handle data class with Scala Case class datasets") { + val caseClasses = listOf( + 1 to DemoCaseClass(1, "1"), + 2 to DemoCaseClass(2, "2"), + 3 to DemoCaseClass(3, "3"), + ) + val dataset = caseClasses.toDS() + dataset.show() + dataset.collectAsList() shouldBe caseClasses + } + + should("handle data class with Scala Case class & deeper datasets") { + val caseClasses = listOf( + 1 to DemoCaseClass(1, "1" to DemoCaseClass(1, 1.0)), + 2 to DemoCaseClass(2, "2" to DemoCaseClass(2, 2.0)), + 3 to DemoCaseClass(3, "3" to DemoCaseClass(3, 3.0)), + ) + val dataset = caseClasses.toDS() + dataset.show() + dataset.collectAsList() shouldBe caseClasses + } + + + xshould("handle Scala Option datasets") { + val caseClasses = listOf(Some(1), Some(2), Some(3)) + val dataset = caseClasses.toDS() + dataset.show() + dataset.collectAsList() shouldBe caseClasses + } + + xshould("handle Scala Option Option datasets") { + val caseClasses = listOf( + Some(Some(1)), + Some(Some(2)), + Some(Some(3)), + ) + val dataset = caseClasses.toDS() + dataset.collectAsList() shouldBe caseClasses + } + + xshould("handle data class Scala Option datasets") { + val caseClasses = listOf( + Some(1) to Some(2), + Some(3) to Some(4), + Some(5) to Some(6), + ) + val dataset = caseClasses.toDS() + dataset.collectAsList() shouldBe caseClasses + } + + xshould("handle Scala Option data class datasets") { + val caseClasses = listOf( + Some(1 to 2), + Some(3 to 4), + Some(5 to 6), + ) + val dataset = caseClasses.toDS() + dataset.collectAsList() shouldBe caseClasses + } + + should("collect data classes with doubles correctly") { + val ll1 = LonLat(1.0, 2.0) + val ll2 = LonLat(3.0, 4.0) + val lonlats = dsOf(ll1, ll2).collectAsList() + expect(lonlats).contains.inAnyOrder.only.values(ll1.copy(), ll2.copy()) + } + + should("contain all generic primitives with complex schema") { + val primitives = t(1, 1.0, 1.toFloat(), 1.toByte(), LocalDate.now(), true) + val primitives2 = t(2, 2.0, 2.toFloat(), 2.toByte(), LocalDate.now().plusDays(1), false) + val tuples = dsOf(primitives, primitives2).collectAsList() + expect(tuples).contains.inAnyOrder.only.values(primitives, primitives2) + } + + should("contain all generic primitives with complex nullable schema") { + val primitives = t(1, 1.0, 1.toFloat(), 1.toByte(), LocalDate.now(), true) + val nulls = t(null, null, null, null, null, null) + val tuples = dsOf(primitives, nulls).collectAsList() + expect(tuples).contains.inAnyOrder.only.values(primitives, nulls) + } + + should("Be able to serialize lists of data classes") { + val dataset = dsOf( + listOf(SomeClass(intArrayOf(1, 2, 3), 4)), + listOf(SomeClass(intArrayOf(3, 2, 1), 0)), + ) + + val (first, second) = dataset.collectAsList() + + first.single().let { (a, b) -> + a.contentEquals(intArrayOf(1, 2, 3)) shouldBe true + b shouldBe 4 + } + second.single().let { (a, b) -> + a.contentEquals(intArrayOf(3, 2, 1)) shouldBe true + b shouldBe 0 + } + } + + should("Be able to serialize arrays of data classes") { + val dataset = dsOf( + arrayOf(SomeClass(intArrayOf(1, 2, 3), 4)), + arrayOf(SomeClass(intArrayOf(3, 2, 1), 0)), + ) + + val (first, second) = dataset.collectAsList() + + first.single().let { (a, b) -> + a.contentEquals(intArrayOf(1, 2, 3)) shouldBe true + b shouldBe 4 + } + second.single().let { (a, b) -> + a.contentEquals(intArrayOf(3, 2, 1)) shouldBe true + b shouldBe 0 + } + } + + should("Be able to serialize lists of tuples") { + val dataset = dsOf( + listOf(Tuple2(intArrayOf(1, 2, 3), 4)), + listOf(Tuple2(intArrayOf(3, 2, 1), 0)), + ) + + val (first, second) = dataset.collectAsList() + + first.single().let { + it._1().contentEquals(intArrayOf(1, 2, 3)) shouldBe true + it._2() shouldBe 4 + } + second.single().let { + it._1().contentEquals(intArrayOf(3, 2, 1)) shouldBe true + it._2() shouldBe 0 + } + } + + should("Generate encoder correctly with complex enum data class") { + val dataset: Dataset = + dsOf( + ComplexEnumDataClass( + int = 1, + string = "string", + strings = listOf("1", "2"), + someEnum = SomeEnum.A, + someOtherEnum = SomeOtherEnum.C, + someEnums = listOf(SomeEnum.A, SomeEnum.B), + someOtherEnums = listOf(SomeOtherEnum.C, SomeOtherEnum.D), + someEnumArray = arrayOf(SomeEnum.A, SomeEnum.B), + someOtherArray = arrayOf(SomeOtherEnum.C, SomeOtherEnum.D), + enumMap = mapOf(SomeEnum.A to SomeOtherEnum.C), + ) + ) + + dataset.show(false) + val first = dataset.takeAsList(1).first() + + first.int shouldBe 1 + first.string shouldBe "string" + first.strings shouldBe listOf("1", "2") + first.someEnum shouldBe SomeEnum.A + first.someOtherEnum shouldBe SomeOtherEnum.C + first.someEnums shouldBe listOf(SomeEnum.A, SomeEnum.B) + first.someOtherEnums shouldBe listOf(SomeOtherEnum.C, SomeOtherEnum.D) + first.someEnumArray shouldBe arrayOf(SomeEnum.A, SomeEnum.B) + first.someOtherArray shouldBe arrayOf(SomeOtherEnum.C, SomeOtherEnum.D) + first.enumMap shouldBe mapOf(SomeEnum.A to SomeOtherEnum.C) + } + + should("work with lists of maps") { + val result = dsOf( + listOf(mapOf("a" to "b", "x" to "y")), + listOf(mapOf("a" to "b", "x" to "y")), + listOf(mapOf("a" to "b", "x" to "y")) + ) + .showDS() + .map { it.last() } + .map { it["x"] } + .filterNotNull() + .distinct() + .collectAsList() + expect(result).contains.inOrder.only.value("y") + } + + should("work with lists of lists") { + val result = dsOf( + listOf(listOf(1, 2, 3)), + listOf(listOf(1, 2, 3)), + listOf(listOf(1, 2, 3)) + ) + .map { it.last() } + .map { it.first() } + .reduceK { a, b -> a + b } + expect(result).toBe(3) + } + + should("Generate schema correctly with nullalble list and map") { + val schema = encoder().schema() + schema.fields().forEach { + it.nullable() shouldBe true + } + } + + should("handle strings converted to lists") { + data class Movie(val id: Long, val genres: String) + data class MovieExpanded(val id: Long, val genres: List) + + val comedies = listOf(Movie(1, "Comedy|Romance"), Movie(2, "Horror|Action")).toDS() + .map { MovieExpanded(it.id, it.genres.split("|").toList()) } + .filter { it.genres.contains("Comedy") } + .collectAsList() + expect(comedies).contains.inAnyOrder.only.values( + MovieExpanded( + 1, + listOf("Comedy", "Romance") + ) + ) + } + + should("handle strings converted to arrays") { + + data class Movie(val id: Long, val genres: String) + + data class MovieExpanded(val id: Long, val genres: Array) { + override fun equals(other: Any?): Boolean { + if (this === other) return true + if (javaClass != other?.javaClass) return false + other as MovieExpanded + return if (id != other.id) false else genres.contentEquals(other.genres) + } + + override fun hashCode(): Int { + var result = id.hashCode() + result = 31 * result + genres.contentHashCode() + return result + } + } + + val comedies = listOf(Movie(1, "Comedy|Romance"), Movie(2, "Horror|Action")).toDS() + .map { MovieExpanded(it.id, it.genres.split("|").toTypedArray()) } + .filter { it.genres.contains("Comedy") } + .collectAsList() + + expect(comedies).contains.inAnyOrder.only.values( + MovieExpanded( + 1, + arrayOf("Comedy", "Romance") + ) + ) + } + + should("handle arrays of generics") { + data class Test(val id: Long, val data: Array>) + + val result = listOf(Test(1, arrayOf(5.1 to 6, 6.1 to 7))) + .toDS() + .map { it.id to it.data.firstOrNull { liEl -> liEl.first < 6 } } + .map { it.second } + .collectAsList() + expect(result).contains.inOrder.only.values(5.1 to 6) + } + + should("handle lists of generics") { + data class Test(val id: Long, val data: List>) + + val result = listOf(Test(1, listOf(5.1 to 6, 6.1 to 7))) + .toDS() + .map { it.id to it.data.firstOrNull { liEl -> liEl.first < 6 } } + .map { it.second } + .collectAsList() + expect(result).contains.inOrder.only.values(5.1 to 6) + } + + should("!handle primitive arrays") { + val result = listOf(arrayOf(1, 2, 3, 4)) + .toDS() + .map { it.map { ai -> ai + 1 } } + .collectAsList() + .flatten() + expect(result).contains.inOrder.only.values(2, 3, 4, 5) + } + } + } +}) + +data class DataClassWithTuple(val tuple: T) + +data class LonLat(val lon: Double, val lat: Double) + +enum class SomeEnum { A, B } + +enum class SomeOtherEnum(val value: Int) { C(1), D(2) } + +data class ComplexEnumDataClass( + val int: Int, + val string: String, + val strings: List, + val someEnum: SomeEnum, + val someOtherEnum: SomeOtherEnum, + val someEnums: List, + val someOtherEnums: List, + val someEnumArray: Array, + val someOtherArray: Array, + val enumMap: Map, +) + +data class NullFieldAbleDataClass( + val optionList: List?, + val optionMap: Map?, +) diff --git a/kotlin-spark-api/3.0/src/test/kotlin/org/jetbrains/kotlinx/spark/api/KafkaStreamingTest.kt b/kotlin-spark-api/3.0/src/test/kotlin/org/jetbrains/kotlinx/spark/api/KafkaStreamingTest.kt new file mode 100644 index 00000000..fd0b12f7 --- /dev/null +++ b/kotlin-spark-api/3.0/src/test/kotlin/org/jetbrains/kotlinx/spark/api/KafkaStreamingTest.kt @@ -0,0 +1,130 @@ +/*- + * =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.Tag +import io.kotest.core.extensions.install +import io.kotest.core.spec.style.FunSpec +import io.kotest.extensions.testcontainers.TestContainerExtension +import io.kotest.extensions.testcontainers.kafka.createStringStringConsumer +import io.kotest.extensions.testcontainers.kafka.createStringStringProducer +import io.kotest.matchers.collections.shouldContain +import io.kotest.matchers.collections.shouldContainAll +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 org.testcontainers.containers.KafkaContainer +import org.testcontainers.utility.DockerImageName +import scala.Tuple3 +import java.io.Serializable +import java.time.Duration + +object Kafka : Tag() + +class KafkaStreamingTest : FunSpec() { + init { + + tags(Kafka) + + val kafka = install( + TestContainerExtension(KafkaContainer(DockerImageName.parse("confluentinc/cp-kafka:7.0.1"))) + ) { + withEmbeddedZookeeper() + withEnv("KAFKA_AUTO_CREATE_TOPICS_ENABLE", "true") + } + println(kafka.bootstrapServers) + test("Streaming should support kafka") { + val topic1 = "test1" + val topic2 = "test2" + + 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, + ) + ) + val data = arrayListOf>>() + + withSparkStreaming( + batchDuration = Durations.milliseconds(1000), + appName = "KotlinDirectKafkaWordCount", + timeout = 10_000L, + master = "local" + ) { + + setRunAfterStart { + val producer = autoClose(kafka.createStringStringProducer()) + producer.send(ProducerRecord(topic1, "Hello this is a test test test")) + producer.send(ProducerRecord(topic2, "This is also also a test test something")) + } + + val kafkaParams: Map = mapOf( + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG to "${kafka.host}:${kafka.getMappedPort(KafkaContainer.KAFKA_PORT)}", + 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.PreferBrokers(), + ConsumerStrategies.Subscribe(setOf(topic1, topic2), kafkaParams), + ) + + // Get the lines, split them into words, count the words and print + + val wordCounts = messages + .map { it.topic() X it.value() } + .flatMapValues { it.split(" ").iterator() } + .map { t(it, 1) } + .reduceByKey { a: Int, b: Int -> a + b } + .map { (tup, counter) -> tup + counter } + + + wordCounts.foreachRDD { rdd, _ -> + data.add(rdd.collect()) + } + } + + val resultList = resultLists.flatMap { (topic, tuples) -> + tuples.map { it.prependedBy(topic) } + } + data.flatten() shouldContainAll resultList + } + } +} \ No newline at end of file diff --git a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/ProjectConfig.kt b/kotlin-spark-api/3.0/src/test/kotlin/org/jetbrains/kotlinx/spark/api/ProjectConfig.kt similarity index 85% rename from kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/ProjectConfig.kt rename to kotlin-spark-api/3.0/src/test/kotlin/org/jetbrains/kotlinx/spark/api/ProjectConfig.kt index 8516ae62..63b1251c 100644 --- a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/ProjectConfig.kt +++ b/kotlin-spark-api/3.0/src/test/kotlin/org/jetbrains/kotlinx/spark/api/ProjectConfig.kt @@ -7,9 +7,9 @@ * 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. @@ -20,9 +20,8 @@ package org.jetbrains.kotlinx.spark.api import io.kotest.core.config.AbstractProjectConfig -import io.kotest.extensions.allure.AllureTestReporter @Suppress("unused") object ProjectConfig : AbstractProjectConfig() { - override fun listeners() = super.listeners() + AllureTestReporter(true) + } diff --git a/kotlin-spark-api/3.0/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt b/kotlin-spark-api/3.0/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt new file mode 100644 index 00000000..9719e8fc --- /dev/null +++ b/kotlin-spark-api/3.0/src/test/kotlin/org/jetbrains/kotlinx/spark/api/StreamingTest.kt @@ -0,0 +1,216 @@ +/*- + * =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.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.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.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 + + +class StreamingTest : ShouldSpec({ + + context("streaming") { + + should("stream") { + val input = listOf("aaa", "bbb", "aaa", "ccc") + val counter = Counter(0) + + withSparkStreaming(Duration(10), timeout = 1000) { + + val (counterBroadcast, queue) = withSpark(ssc) { + spark.broadcast(counter) X LinkedList(listOf(sc.parallelize(input))) + } + + val inputStream = ssc.queueStream(queue) + + inputStream.foreachRDD { rdd, _ -> + withSpark(rdd) { + rdd.toDS().forEach { + it shouldBeIn input + counterBroadcast.value.value++ + } + } + } + } + + counter.value shouldBe input.size + } + + should("Work with checkpointpath") { + val emptyDir = createTempDir() + val testDirectory = createTempDir() + val corruptedCheckpointDir = createCorruptedCheckpoint() + + val batchDuration = Durations.seconds(1) + val timeout = Durations.seconds(1).milliseconds() + + + val newContextCreated = AtomicBoolean(false) + + val creatingFun: KSparkStreamingSession.() -> Unit = { + println("created new context") + newContextCreated.set(true) + + // closing statement + ssc.textFileStream(testDirectory.absolutePath).foreachRDD { rdd, _ -> rdd.count() } + } + + // fill emptyDir with checkpoint + newContextCreated.set(false) + withSparkStreaming( + batchDuration = batchDuration, + checkpointPath = emptyDir.absolutePath, + props = mapOf("newContext" to true), + timeout = timeout, + func = creatingFun, + ) + newContextCreated.get() shouldBe true + + // check that creatingFun isn't executed when checkpoint is present + newContextCreated.set(false) + withSparkStreaming( + batchDuration = batchDuration, + checkpointPath = emptyDir.absolutePath, + props = mapOf("newContext" to true), + timeout = timeout, + func = creatingFun, + ) + newContextCreated.get() shouldBe false + + // check that creatingFun is not executed when createOnError = false using corrupted checkpoint + newContextCreated.set(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) + withSparkStreaming( + batchDuration = batchDuration, + checkpointPath = corruptedCheckpointDir, + props = mapOf("newContext" to true), + timeout = timeout, + func = creatingFun, + createOnError = true, + ) + 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)) + } + } +}) + +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?).toOption()).nonEmpty()) + return checkpointDirectory +} + + +class Counter(@Volatile var value: Int) : Serializable + +class Result(@Volatile var list: List> = listOf()) : Serializable \ No newline at end of file diff --git a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/TypeInferenceTest.kt b/kotlin-spark-api/3.0/src/test/kotlin/org/jetbrains/kotlinx/spark/api/TypeInferenceTest.kt similarity index 64% rename from kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/TypeInferenceTest.kt rename to kotlin-spark-api/3.0/src/test/kotlin/org/jetbrains/kotlinx/spark/api/TypeInferenceTest.kt index 8950ec80..2d481e79 100644 --- a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/TypeInferenceTest.kt +++ b/kotlin-spark-api/3.0/src/test/kotlin/org/jetbrains/kotlinx/spark/api/TypeInferenceTest.kt @@ -41,21 +41,21 @@ class TypeInferenceTest : ShouldSpec({ val struct = Struct.fromJson(schema(typeOf>>()).prettyJson())!! should("contain correct typings") { expect(struct.fields).notToBeNull().contains.inAnyOrder.only.entries( - hasField("first", "string"), - hasStruct("second", - hasField("vala", "integer"), - hasStruct("tripl1", - hasField("first", "integer"), - hasStruct("second", - hasField("vala2", "long"), - hasStruct("para2", - hasField("first", "long"), - hasField("second", "string") - ) - ), - hasField("third", "integer") + hasField("first", "string"), + hasStruct("second", + hasField("vala", "integer"), + hasStruct("tripl1", + hasField("first", "integer"), + hasStruct("second", + hasField("vala2", "long"), + hasStruct("para2", + hasField("first", "long"), + hasField("second", "string") ) + ), + hasField("third", "integer") ) + ) ) } } @@ -67,23 +67,23 @@ class TypeInferenceTest : ShouldSpec({ val struct = Struct.fromJson(schema(typeOf>>()).prettyJson())!! should("contain correct typings") { expect(struct.fields).notToBeNull().contains.inAnyOrder.only.entries( - hasField("first", "string"), - hasStruct("second", - hasField("vala", "integer"), - hasStruct("tripl1", - hasField("first", "integer"), - hasStruct("second", - hasField("vala2", "long"), - hasStruct("para2", - hasField("first", "long"), - hasStruct("second", - hasField("vala3", "double") - ) - ) - ), - hasField("third", "integer") + hasField("first", "string"), + hasStruct("second", + hasField("vala", "integer"), + hasStruct("tripl1", + hasField("first", "integer"), + hasStruct("second", + hasField("vala2", "long"), + hasStruct("para2", + hasField("first", "long"), + hasStruct("second", + hasField("vala3", "double") + ) ) + ), + hasField("third", "integer") ) + ) ) } } @@ -93,9 +93,9 @@ class TypeInferenceTest : ShouldSpec({ val struct = Struct.fromJson(schema(typeOf()).prettyJson())!! should("return correct types too") { expect(struct.fields).notToBeNull().contains.inAnyOrder.only.entries( - hasField("a", "string"), - hasField("b", "integer"), - hasField("c", "double") + hasField("a", "string"), + hasField("b", "integer"), + hasField("c", "double") ) } } @@ -115,8 +115,8 @@ class TypeInferenceTest : ShouldSpec({ isOfType("array") feature { f(it::elementType) }.notToBeNull().isA { feature { f(it.value::fields) }.notToBeNull().contains.inAnyOrder.only.entries( - hasField("first", "integer"), - hasField("second", "long") + hasField("first", "integer"), + hasField("second", "long") ) } } @@ -131,7 +131,7 @@ class TypeInferenceTest : ShouldSpec({ isOfType("array") feature { f(it::elementType) }.notToBeNull().isA { feature { f(it.value::fields) }.notToBeNull().contains.inAnyOrder.only.entries( - hasField("e", "string") + hasField("e", "string") ) } } @@ -174,8 +174,8 @@ class TypeInferenceTest : ShouldSpec({ val struct = Struct.fromJson(schema(typeOf()).prettyJson())!! should("Not change order of fields") { expect(struct.fields).notToBeNull().containsExactly( - hasField("lon", "double"), - hasField("lat", "double") + hasField("lon", "double"), + hasField("lat", "double") ) } } @@ -186,38 +186,39 @@ class TypeInferenceTest : ShouldSpec({ should("show that list is nullable and element is not") { expect(struct) - .feature("some", { fields }) { - notToBeNull().contains.inOrder.only.entry { - this - .feature("field name", { name }) { toBe("optionList") } - .feature("optionList is nullable", { nullable }) { toBe(true) } - .feature("optionList", { type }) { - this - .isA() - .feature("element type of optionList", { value.elementType }) { toBe(SimpleElement("integer")) } - .feature("optionList contains null", { value.containsNull }) { toBe(false) } - .feature("optionList type", { value }) { isOfType("array") } - } - } + .feature("some", { fields }) { + notToBeNull().contains.inOrder.only.entry { + this + .feature("field name", { name }) { toBe("optionList") } + .feature("optionList is nullable", { nullable }) { toBe(true) } + .feature("optionList", { type }) { + this + .isA() + .feature("element type of optionList", + { value.elementType }) { toBe(SimpleElement("integer")) } + .feature("optionList contains null", { value.containsNull }) { toBe(false) } + .feature("optionList type", { value }) { isOfType("array") } + } } + } } should("generate valid serializer schema") { expect(encoder().schema()) { this - .feature("data type", { this.fields()?.toList() }) { - this.notToBeNull().contains.inOrder.only.entry { - this - .feature("element name", { name() }) { toBe("optionList") } - .feature("field type", { dataType() }, { - this - .isA() - .feature("element type", { elementType() }) { isA() } - .feature("element nullable", { containsNull() }) { toBe(expected = false) } - }) - .feature("optionList nullable", { nullable() }) { toBe(true) } - } + .feature("data type", { this.fields()?.toList() }) { + this.notToBeNull().contains.inOrder.only.entry { + this + .feature("element name", { name() }) { toBe("optionList") } + .feature("field type", { dataType() }, { + this + .isA() + .feature("element type", { elementType() }) { isA() } + .feature("element nullable", { containsNull() }) { toBe(expected = false) } + }) + .feature("optionList nullable", { nullable() }) { toBe(true) } } + } } } } @@ -229,15 +230,15 @@ private fun Expect.isOfType(type: String) { } private fun hasStruct( - name: String, - expectedField: Expect.() -> Unit, - vararg expectedFields: Expect.() -> Unit, + name: String, + expectedField: Expect.() -> Unit, + vararg expectedFields: Expect.() -> Unit, ): Expect.() -> Unit { return { feature { f(it::name) }.toBe(name) feature { f(it::type) }.isA { feature { f(it.value::fields) }.notToBeNull().contains.inAnyOrder.only.entries(expectedField, - *expectedFields) + *expectedFields) } } } diff --git a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/UDFRegisterTest.kt b/kotlin-spark-api/3.0/src/test/kotlin/org/jetbrains/kotlinx/spark/api/UDFRegisterTest.kt similarity index 97% rename from kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/UDFRegisterTest.kt rename to kotlin-spark-api/3.0/src/test/kotlin/org/jetbrains/kotlinx/spark/api/UDFRegisterTest.kt index 1926be42..df3525ef 100644 --- a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/UDFRegisterTest.kt +++ b/kotlin-spark-api/3.0/src/test/kotlin/org/jetbrains/kotlinx/spark/api/UDFRegisterTest.kt @@ -124,8 +124,8 @@ class UDFRegisterTest : ShouldSpec({ should("succeed in dataset") { val dataset: Dataset = listOf( - NormalClass(name="a", age =10), - NormalClass(name="b", age =20) + NormalClass(name = "a", age = 10), + NormalClass(name = "b", age = 20) ).toDS() val udfWrapper = udf.register("nameConcatAge") { name, age -> @@ -150,7 +150,7 @@ class UDFRegisterTest : ShouldSpec({ should("return NormalClass") { listOf("a" to 1, "b" to 2).toDS().toDF().createOrReplaceTempView("test2") udf.register("toNormalClass") { a, b -> - NormalClass(b, a) + NormalClass(b, a) } spark.sql("select toNormalClass(first, second) from test2").show() } @@ -160,6 +160,7 @@ class UDFRegisterTest : ShouldSpec({ } }) + data class NormalClass( val age: Int, val name: String diff --git a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/struct/model/models.kt b/kotlin-spark-api/3.0/src/test/kotlin/org/jetbrains/kotlinx/spark/api/struct/model/models.kt similarity index 100% rename from kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/struct/model/models.kt rename to kotlin-spark-api/3.0/src/test/kotlin/org/jetbrains/kotlinx/spark/api/struct/model/models.kt diff --git a/kotlin-spark-api/3.2/pom_2.12.xml b/kotlin-spark-api/3.2/pom_2.12.xml deleted file mode 100644 index c0eabd06..00000000 --- a/kotlin-spark-api/3.2/pom_2.12.xml +++ /dev/null @@ -1,134 +0,0 @@ - - - - 4.0.0 - - Kotlin Spark API: API for Spark 3.2+ (Scala 2.12) - kotlin-spark-api-3.2 - Kotlin API compatible with spark 3.2.0 Kotlin for Apache Spark - - org.jetbrains.kotlinx.spark - kotlin-spark-api-parent_2.12 - 1.0.3 - ../../pom_2.12.xml - - jar - - - - org.jetbrains.kotlin - kotlin-stdlib-jdk8 - - - org.jetbrains.kotlin - kotlin-reflect - - - org.jetbrains.kotlinx.spark - core-3.2_${scala.compat.version} - - - org.jetbrains.kotlinx.spark - kotlin-spark-api-common - - - - - org.apache.spark - spark-sql_${scala.compat.version} - ${spark3.version} - provided - - - - - io.kotest - kotest-runner-junit5-jvm - ${kotest.version} - test - - - io.kotest.extensions - kotest-extensions-allure - ${kotest-extension-allure.version} - test - - - com.beust - klaxon - ${klaxon.version} - test - - - ch.tutteli.atrium - atrium-fluent-en_GB - ${atrium.version} - test - - - - - src/main/kotlin - src/test/kotlin - target/${scala.compat.version} - - - org.jetbrains.kotlin - kotlin-maven-plugin - - - compile - - compile - - - - test-compile - - test-compile - - - - - - org.apache.maven.plugins - maven-surefire-plugin - - - org.jetbrains.dokka - dokka-maven-plugin - ${dokka.version} - - 8 - - - - dokka - - dokka - - pre-site - - - javadocjar - - javadocJar - - pre-integration-test - - - - - io.qameta.allure - allure-maven - - ${project.basedir}/allure-results/${scala.compat.version} - - - - org.jacoco - jacoco-maven-plugin - - - - diff --git a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/ApiV1.kt b/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/ApiV1.kt deleted file mode 100644 index 32935f40..00000000 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/ApiV1.kt +++ /dev/null @@ -1,1027 +0,0 @@ -/*- - * =LICENSE= - * Kotlin Spark API - * ---------- - * Copyright (C) 2019 - 2020 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("HasPlatformType", "unused", "FunctionName") - -package org.jetbrains.kotlinx.spark.api - -import org.apache.spark.SparkContext -import org.apache.spark.api.java.JavaSparkContext -import org.apache.spark.api.java.function.* -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.streaming.GroupState -import org.apache.spark.sql.streaming.GroupStateTimeout -import org.apache.spark.sql.streaming.OutputMode -import org.apache.spark.sql.types.* -import org.jetbrains.kotlinx.spark.extensions.KSparkExtensions -import scala.Product -import scala.Tuple2 -import scala.reflect.ClassTag -import java.beans.PropertyDescriptor -import java.math.BigDecimal -import java.sql.Date -import java.sql.Timestamp -import java.time.Instant -import java.time.LocalDate -import java.util.* -import java.util.concurrent.ConcurrentHashMap -import kotlin.Any -import kotlin.Array -import kotlin.Boolean -import kotlin.BooleanArray -import kotlin.Byte -import kotlin.ByteArray -import kotlin.Deprecated -import kotlin.DeprecationLevel -import kotlin.Double -import kotlin.DoubleArray -import kotlin.ExperimentalStdlibApi -import kotlin.Float -import kotlin.FloatArray -import kotlin.IllegalArgumentException -import kotlin.Int -import kotlin.IntArray -import kotlin.Long -import kotlin.LongArray -import kotlin.OptIn -import kotlin.Pair -import kotlin.ReplaceWith -import kotlin.Short -import kotlin.ShortArray -import kotlin.String -import kotlin.Suppress -import kotlin.Triple -import kotlin.Unit -import kotlin.also -import kotlin.apply -import kotlin.invoke -import kotlin.reflect.* -import kotlin.reflect.full.findAnnotation -import kotlin.reflect.full.isSubclassOf -import kotlin.reflect.full.isSubtypeOf -import kotlin.reflect.full.primaryConstructor -import kotlin.to - -@JvmField -val ENCODERS = mapOf, Encoder<*>>( - Boolean::class to BOOLEAN(), - Byte::class to BYTE(), - Short::class to SHORT(), - Int::class to INT(), - Long::class to LONG(), - Float::class to FLOAT(), - Double::class to DOUBLE(), - String::class to STRING(), - BigDecimal::class to DECIMAL(), - Date::class to DATE(), - LocalDate::class to LOCALDATE(), // 3.0 only - Timestamp::class to TIMESTAMP(), - Instant::class to INSTANT(), // 3.0 only - ByteArray::class to BINARY() -) - - -/** - * 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 - */ -inline fun SparkSession.broadcast(value: T): Broadcast = try { - sparkContext.broadcast(value, encoder().clsTag()) -} catch (e: ClassNotFoundException) { - JavaSparkContext(sparkContext).broadcast(value) -} - -/** - * 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 - */ -@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) -} - -/** - * Utility method to create dataset from list - */ -inline fun SparkSession.toDS(list: List): Dataset = - createDataset(list, encoder()) - -/** - * Utility method to create dataset from list - */ -inline fun SparkSession.dsOf(vararg t: T): Dataset = - createDataset(listOf(*t), encoder()) - -/** - * Utility method to create dataset from list - */ -inline fun List.toDS(spark: SparkSession): Dataset = - spark.createDataset(this, encoder()) - -/** - * Main method of API, which gives you seamless integration with Spark: - * It creates encoder for any given supported type T - * - * Supported types are data classes, primitives, and Lists, Maps and Arrays containing them - * - * @param T type, supported by Spark - * @return generated encoder - */ -@OptIn(ExperimentalStdlibApi::class) -inline fun encoder(): Encoder = generateEncoder(typeOf(), T::class) - -fun generateEncoder(type: KType, cls: KClass<*>): Encoder { - @Suppress("UNCHECKED_CAST") - return when { - isSupportedClass(cls) -> kotlinClassEncoder(memoizedSchema(type), cls) - else -> ENCODERS[cls] as? Encoder? ?: bean(cls.java) - } as Encoder -} - -private fun isSupportedClass(cls: KClass<*>): Boolean = cls.isData - || cls.isSubclassOf(Map::class) - || cls.isSubclassOf(Iterable::class) - || cls.isSubclassOf(Product::class) - || cls.java.isArray - -private fun kotlinClassEncoder(schema: DataType, kClass: KClass<*>): Encoder { - return ExpressionEncoder( - if (schema is DataTypeWithClass) KotlinReflection.serializerFor( - kClass.java, - schema - ) else KotlinReflection.serializerForType(KotlinReflection.getType(kClass.java)), - if (schema is DataTypeWithClass) KotlinReflection.deserializerFor( - kClass.java, - schema - ) else KotlinReflection.deserializerForType(KotlinReflection.getType(kClass.java)), - ClassTag.apply(kClass.java) - ) -} - -inline fun Dataset.map(noinline func: (T) -> R): Dataset = - map(MapFunction(func), encoder()) - -inline fun Dataset.flatMap(noinline func: (T) -> Iterator): Dataset = - flatMap(func, encoder()) - -inline fun > Dataset.flatten(): Dataset = - flatMap(FlatMapFunction { it.iterator() }, encoder()) - -inline fun Dataset.groupByKey(noinline func: (T) -> R): KeyValueGroupedDataset = - groupByKey(MapFunction(func), encoder()) - -inline fun Dataset.mapPartitions(noinline func: (Iterator) -> Iterator): Dataset = - mapPartitions(func, encoder()) - -@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()) - -inline fun KeyValueGroupedDataset.mapGroups(noinline func: (KEY, Iterator) -> R): Dataset = - mapGroups(MapGroupsFunction(func), encoder()) - -inline fun KeyValueGroupedDataset.reduceGroupsK(noinline func: (VALUE, VALUE) -> VALUE): Dataset> = - reduceGroups(ReduceFunction(func)) - .map { t -> t._1 to t._2 } - -/** - * (Kotlin-specific) - * Reduces the elements of this Dataset using the specified binary function. The given `func` - * must be commutative and associative or the result may be non-deterministic. - */ -inline fun Dataset.reduceK(noinline func: (T, T) -> T): T = - reduce(ReduceFunction(func)) - -@JvmName("takeKeysTuple2") -inline fun Dataset>.takeKeys(): Dataset = map { it._1() } - -inline fun Dataset>.takeKeys(): Dataset = map { it.first } - -@JvmName("takeKeysArity2") -inline fun Dataset>.takeKeys(): Dataset = map { it._1 } - -@JvmName("takeValuesTuple2") -inline fun Dataset>.takeValues(): Dataset = map { it._2() } - -inline fun Dataset>.takeValues(): Dataset = map { it.second } - -@JvmName("takeValuesArity2") -inline fun Dataset>.takeValues(): Dataset = map { it._2 } - - -inline fun KeyValueGroupedDataset.flatMapGroups( - noinline func: (key: K, values: Iterator) -> Iterator, -): Dataset = flatMapGroups( - FlatMapGroupsFunction(func), - encoder() -) - -fun GroupState.getOrNull(): S? = if (exists()) get() else null - -operator fun GroupState.getValue(thisRef: Any?, property: KProperty<*>): S? = getOrNull() -operator fun GroupState.setValue(thisRef: Any?, property: KProperty<*>, value: S?): Unit = update(value) - - -inline fun KeyValueGroupedDataset.mapGroupsWithState( - noinline func: (key: K, values: Iterator, state: GroupState) -> U, -): Dataset = mapGroupsWithState( - MapGroupsWithStateFunction(func), - encoder(), - encoder() -) - -inline fun KeyValueGroupedDataset.mapGroupsWithState( - timeoutConf: GroupStateTimeout, - noinline func: (key: K, values: Iterator, state: GroupState) -> U, -): Dataset = mapGroupsWithState( - MapGroupsWithStateFunction(func), - encoder(), - encoder(), - timeoutConf -) - -inline fun KeyValueGroupedDataset.flatMapGroupsWithState( - outputMode: OutputMode, - timeoutConf: GroupStateTimeout, - noinline func: (key: K, values: Iterator, state: GroupState) -> Iterator, -): Dataset = flatMapGroupsWithState( - FlatMapGroupsWithStateFunction(func), - outputMode, - encoder(), - encoder(), - timeoutConf -) - -inline fun KeyValueGroupedDataset.cogroup( - other: KeyValueGroupedDataset, - noinline func: (key: K, left: Iterator, right: Iterator) -> Iterator, -): Dataset = cogroup( - other, - CoGroupFunction(func), - encoder() -) - -inline fun Dataset.downcast(): Dataset = `as`(encoder()) -inline fun Dataset<*>.`as`(): Dataset = `as`(encoder()) -inline fun Dataset<*>.to(): Dataset = `as`(encoder()) - -inline fun Dataset.forEach(noinline func: (T) -> Unit) = foreach(ForeachFunction(func)) - -inline fun Dataset.forEachPartition(noinline func: (Iterator) -> Unit) = - foreachPartition(ForeachPartitionFunction(func)) - -/** - * It's hard to call `Dataset.debugCodegen` from kotlin, so here is utility for that - */ -fun Dataset.debugCodegen() = also { KSparkExtensions.debugCodegen(it) } - -val SparkSession.sparkContext - get() = KSparkExtensions.sparkContext(this) - -/** - * It's hard to call `Dataset.debug` from kotlin, so here is utility for that - */ -fun Dataset.debug() = also { KSparkExtensions.debug(it) } - -@Suppress("FunctionName") -@Deprecated("Changed to \"`===`\" to better reflect Scala API.", ReplaceWith("this `===` c")) -infix fun Column.`==`(c: Column) = `$eq$eq$eq`(c) - -/** - * Unary minus, i.e. negate the expression. - * ``` - * // Scala: select the amount column and negates all values. - * df.select( -df("amount") ) - * - * // Kotlin: - * import org.jetbrains.kotlinx.spark.api.* - * df.select( -df("amount") ) - * - * // Java: - * import static org.apache.spark.sql.functions.*; - * df.select( negate(col("amount") ); - * ``` - */ -operator fun Column.unaryMinus(): Column = `unary_$minus`() - -/** - * Inversion of boolean expression, i.e. NOT. - * ``` - * // Scala: select rows that are not active (isActive === false) - * df.filter( !df("isActive") ) - * - * // Kotlin: - * import org.jetbrains.kotlinx.spark.api.* - * df.select( !df("amount") ) - * - * // Java: - * import static org.apache.spark.sql.functions.*; - * df.filter( not(df.col("isActive")) ); - * ``` - */ -operator fun Column.not(): Column = `unary_$bang`() - -/** - * Equality test. - * ``` - * // Scala: - * df.filter( df("colA") === df("colB") ) - * - * // Kotlin: - * import org.jetbrains.kotlinx.spark.api.* - * df.filter( df("colA") eq df("colB") ) - * // or - * df.filter( df("colA") `===` df("colB") ) - * - * // Java - * import static org.apache.spark.sql.functions.*; - * df.filter( col("colA").equalTo(col("colB")) ); - * ``` - */ -infix fun Column.eq(other: Any): Column = `$eq$eq$eq`(other) - -/** - * Equality test. - * ``` - * // Scala: - * df.filter( df("colA") === df("colB") ) - * - * // Kotlin: - * import org.jetbrains.kotlinx.spark.api.* - * df.filter( df("colA") eq df("colB") ) - * // or - * df.filter( df("colA") `===` df("colB") ) - * - * // Java - * import static org.apache.spark.sql.functions.*; - * df.filter( col("colA").equalTo(col("colB")) ); - * ``` - */ -infix fun Column.`===`(other: Any): Column = `$eq$eq$eq`(other) - -/** - * Inequality test. - * ``` - * // Scala: - * df.select( df("colA") =!= df("colB") ) - * df.select( !(df("colA") === df("colB")) ) - * - * // Kotlin: - * import org.jetbrains.kotlinx.spark.api.* - * df.select( df("colA") neq df("colB") ) - * df.select( !(df("colA") eq df("colB")) ) - * // or - * df.select( df("colA") `=!=` df("colB") ) - * df.select( !(df("colA") `===` df("colB")) ) - * - * // Java: - * import static org.apache.spark.sql.functions.*; - * df.filter( col("colA").notEqual(col("colB")) ); - * ``` - */ -infix fun Column.neq(other: Any): Column = `$eq$bang$eq`(other) - -/** - * Inequality test. - * ``` - * // Scala: - * df.select( df("colA") =!= df("colB") ) - * df.select( !(df("colA") === df("colB")) ) - * - * // Kotlin: - * import org.jetbrains.kotlinx.spark.api.* - * df.select( df("colA") neq df("colB") ) - * df.select( !(df("colA") eq df("colB")) ) - * // or - * df.select( df("colA") `=!=` df("colB") ) - * df.select( !(df("colA") `===` df("colB")) ) - * - * // Java: - * import static org.apache.spark.sql.functions.*; - * df.filter( col("colA").notEqual(col("colB")) ); - * ``` - */ -infix fun Column.`=!=`(other: Any): Column = `$eq$bang$eq`(other) - -/** - * Greater than. - * ``` - * // Scala: The following selects people older than 21. - * people.select( people("age") > 21 ) - * - * // Kotlin: - * import org.jetbrains.kotlinx.spark.api.* - * people.select( people("age") gt 21 ) - * - * // Java: - * import static org.apache.spark.sql.functions.*; - * people.select( people.col("age").gt(21) ); - * ``` - */ -infix fun Column.gt(other: Any): Column = `$greater`(other) - -/** - * Less than. - * ``` - * // Scala: The following selects people younger than 21. - * people.select( people("age") < 21 ) - * - * // Kotlin: - * import org.jetbrains.kotlinx.spark.api.* - * people.select( people("age") lt 21 ) - * - * // Java: - * import static org.apache.spark.sql.functions.*; - * people.select( people.col("age").lt(21) ); - * ``` - */ -infix fun Column.lt(other: Any): Column = `$less`(other) - -/** - * Less than or equal to. - * ``` - * // Scala: The following selects people age 21 or younger than 21. - * people.select( people("age") <= 21 ) - * - * // Kotlin: - * import org.jetbrains.kotlinx.spark.api.* - * people.select( people("age") leq 21 ) - * - * // Java: - * import static org.apache.spark.sql.functions.*; - * people.select( people.col("age").leq(21) ); - * ``` - */ -infix fun Column.leq(other: Any): Column = `$less$eq`(other) - -/** - * Greater than or equal to an expression. - * ``` - * // Scala: The following selects people age 21 or older than 21. - * people.select( people("age") >= 21 ) - * - * // Kotlin: - * import org.jetbrains.kotlinx.spark.api.* - * people.select( people("age") geq 21 ) - * - * // Java: - * import static org.apache.spark.sql.functions.*; - * people.select( people.col("age").geq(21) ); - * ``` - */ -infix fun Column.geq(other: Any): Column = `$greater$eq`(other) - -/** - * True if the current column is in the given [range]. - * ``` - * // Scala: - * df.where( df("colA").between(1, 5) ) - * - * // Kotlin: - * import org.jetbrains.kotlinx.spark.api.* - * df.where( df("colA") inRangeOf 1..5 ) - * - * // Java: - * import static org.apache.spark.sql.functions.*; - * df.where( df.col("colA").between(1, 5) ); - * ``` - */ -infix fun Column.inRangeOf(range: ClosedRange<*>): Column = between(range.start, range.endInclusive) - -/** - * Boolean OR. - * ``` - * // Scala: The following selects people that are in school or employed. - * people.filter( people("inSchool") || people("isEmployed") ) - * - * // Kotlin: - * import org.jetbrains.kotlinx.spark.api.* - * people.filter( people("inSchool") or people("isEmployed") ) - * - * // Java: - * import static org.apache.spark.sql.functions.*; - * people.filter( people.col("inSchool").or(people.col("isEmployed")) ); - * ``` - */ -infix fun Column.or(other: Any): Column = `$bar$bar`(other) - -/** - * Boolean AND. - * ``` - * // Scala: The following selects people that are in school and employed at the same time. - * people.select( people("inSchool") && people("isEmployed") ) - * - * // Kotlin: - * import org.jetbrains.kotlinx.spark.api.* - * people.filter( people("inSchool") and people("isEmployed") ) - * // or - * people.filter( people("inSchool") `&&` people("isEmployed") ) - * - * // Java: - * import static org.apache.spark.sql.functions.*; - * people.select( people.col("inSchool").and(people.col("isEmployed")) ); - * ``` - */ -infix fun Column.and(other: Any): Column = `$amp$amp`(other) - -/** - * Boolean AND. - * ``` - * // Scala: The following selects people that are in school and employed at the same time. - * people.select( people("inSchool") && people("isEmployed") ) - * - * // Kotlin: - * import org.jetbrains.kotlinx.spark.api.* - * people.filter( people("inSchool") and people("isEmployed") ) - * // or - * people.filter( people("inSchool") `&&` people("isEmployed") ) - * - * // Java: - * import static org.apache.spark.sql.functions.*; - * people.select( people.col("inSchool").and(people.col("isEmployed")) ); - * ``` - */ -infix fun Column.`&&`(other: Any): Column = `$amp$amp`(other) - -/** - * Multiplication of this expression and another expression. - * ``` - * // Scala: The following multiplies a person's height by their weight. - * people.select( people("height") * people("weight") ) - * - * // Kotlin: - * import org.jetbrains.kotlinx.spark.api.* - * people.select( people("height") * people("weight") ) - * - * // Java: - * import static org.apache.spark.sql.functions.*; - * people.select( people.col("height").multiply(people.col("weight")) ); - * ``` - */ -operator fun Column.times(other: Any): Column = `$times`(other) - -/** - * Division this expression by another expression. - * ``` - * // Scala: The following divides a person's height by their weight. - * people.select( people("height") / people("weight") ) - * - * // Kotlin - * import org.jetbrains.kotlinx.spark.api.* - * people.select( people("height") / people("weight") ) - * - * // Java: - * import static org.apache.spark.sql.functions.*; - * people.select( people.col("height").divide(people.col("weight")) ); - * ``` - */ -operator fun Column.div(other: Any): Column = `$div`(other) - -/** - * Modulo (a.k.a. remainder) expression. - * ``` - * // Scala: - * df.where( df("colA") % 2 === 0 ) - * - * // Kotlin: - * import org.jetbrains.kotlinx.spark.api.* - * df.where( df("colA") % 2 eq 0 ) - * - * // Java: - * import static org.apache.spark.sql.functions.*; - * df.where( df.col("colA").mod(2).equalTo(0) ); - * ``` - */ -operator fun Column.rem(other: Any): Column = `$percent`(other) - -/** - * An expression that gets an item at position `ordinal` out of an array, - * or gets a value by key `key` in a `MapType`. - * ``` - * // Scala: - * df.where( df("arrayColumn").getItem(0) === 5 ) - * - * // Kotlin - * import org.jetbrains.kotlinx.spark.api.* - * df.where( df("arrayColumn")[0] eq 5 ) - * - * // Java - * import static org.apache.spark.sql.functions.*; - * df.where( df.col("arrayColumn").getItem(0).equalTo(5) ); - * ``` - */ -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 - * 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 - * - * @receiver left dataset - * @param right right dataset - * @param col join condition - * - * @return dataset of pairs where right element is forced nullable - */ -inline fun Dataset.leftJoin(right: Dataset, col: Column): Dataset> { - return joinWith(right, col, "left").map { it._1 to it._2 } -} - -/** - * Alias for [Dataset.joinWith] which passes "right" argument - * and respects the fact that in result of right join left relation is nullable - * - * @receiver left dataset - * @param right right dataset - * @param col join condition - * - * @return dataset of [Pair] where left element is forced nullable - */ -inline fun Dataset.rightJoin(right: Dataset, col: Column): Dataset> { - return joinWith(right, col, "right").map { it._1 to it._2 } -} - -/** - * Alias for [Dataset.joinWith] which passes "inner" argument - * - * @receiver left dataset - * @param right right dataset - * @param col join condition - * - * @return resulting dataset of [Pair] - */ -inline fun Dataset.innerJoin(right: Dataset, col: Column): Dataset> { - return joinWith(right, col, "inner").map { it._1 to it._2 } -} - -/** - * Alias for [Dataset.joinWith] which passes "full" argument - * and respects the fact that in result of join any element of resulting tuple is nullable - * - * @receiver left dataset - * @param right right dataset - * @param col join condition - * - * @return dataset of [Pair] where both elements are forced nullable - */ -inline fun Dataset.fullJoin( - right: Dataset, - col: Column, -): Dataset> { - return joinWith(right, col, "full").map { it._1 to it._2 } -} - -/** - * Alias for [Dataset.sort] which forces user to provide sorted columns from the source dataset - * - * @receiver source [Dataset] - * @param columns producer of sort columns - * @return sorted [Dataset] - */ -inline fun Dataset.sort(columns: (Dataset) -> Array) = sort(*columns(this)) - -/** - * This function creates block, where one can call any further computations on already cached dataset - * Data will be unpersisted automatically at the end of computation - * - * it may be useful in many situations, for example, when one needs to write data to several targets - * ```kotlin - * ds.withCached { - * write() - * .also { it.orc("First destination") } - * .also { it.avro("Second destination") } - * } - * ``` - * - * @param blockingUnpersist if execution should be blocked until everything persisted will be deleted - * @param executeOnCached Block which should be executed on cached dataset. - * @return result of block execution for further usage. It may be anything including source or new dataset - */ -inline fun Dataset.withCached( - blockingUnpersist: Boolean = false, - executeOnCached: Dataset.() -> R, -): R { - val cached = this.cache() - return cached.executeOnCached().also { cached.unpersist(blockingUnpersist) } -} - -inline fun Dataset.toList() = KSparkExtensions.collectAsList(to()) -inline fun Dataset<*>.toArray(): Array = to().collect() as Array - -/** - * Selects column based on the column name and returns it as a [Column]. - * - * @note The column name can also reference to a nested column like `a.b`. - */ -operator fun Dataset.invoke(colName: String): Column = col(colName) - -/** - * Helper function to quickly get a [TypedColumn] (or [Column]) from a dataset in a refactor-safe manner. - * ```kotlin - * val dataset: Dataset = ... - * val columnA: TypedColumn = dataset.col(YourClass::a) - * ``` - * @see invoke - */ - -@Suppress("UNCHECKED_CAST") -inline fun Dataset.col(column: KProperty1): TypedColumn = - col(column.name).`as`() as TypedColumn - -/** - * 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) ) - * ``` - */ -@Suppress("UNCHECKED_CAST") -inline fun col(column: KProperty1): TypedColumn = - functions.col(column.name).`as`() as TypedColumn - -/** - * Helper function to quickly get a [TypedColumn] (or [Column]) from a dataset in a refactor-safe manner. - * ```kotlin - * val dataset: Dataset = ... - * val columnA: TypedColumn = dataset(YourClass::a) - * ``` - * @see col - */ -inline operator fun Dataset.invoke(column: KProperty1): TypedColumn = col(column) - -/** - * Allows to sort data class dataset on one or more of the properties of the data class. - * ```kotlin - * val sorted: Dataset = unsorted.sort(YourClass::a) - * val sorted2: Dataset = unsorted.sort(YourClass::a, YourClass::b) - * ``` - */ -fun Dataset.sort(col: KProperty1, vararg cols: KProperty1): Dataset = - sort(col.name, *cols.map { it.name }.toTypedArray()) - -/** - * Alternative to [Dataset.show] which returns source dataset. - * Useful for debug purposes when you need to view content of a dataset as an intermediate operation - */ -fun Dataset.showDS(numRows: Int = 20, truncate: Boolean = true) = apply { show(numRows, truncate) } - -/** - * 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, -): Dataset> = - 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, -): Dataset> = - 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, -): Dataset> = - 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, -): Dataset> = - 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) -fun schema(type: KType, map: Map = mapOf()): DataType { - val primitiveSchema = knownDataTypes[type.classifier] - if (primitiveSchema != null) return KSimpleTypeWrapper( - primitiveSchema, - (type.classifier!! as KClass<*>).java, - type.isMarkedNullable - ) - val klass = type.classifier as? KClass<*> ?: throw IllegalArgumentException("Unsupported type $type") - val args = type.arguments - - val types = transitiveMerge(map, klass.typeParameters.zip(args).map { - it.first.name to it.second.type!! - }.toMap()) - return when { - klass.isSubclassOf(Enum::class) -> { - KSimpleTypeWrapper(DataTypes.StringType, klass.java, type.isMarkedNullable) - } - klass.isSubclassOf(Iterable::class) || klass.java.isArray -> { - val listParam = if (klass.java.isArray) { - when (klass) { - IntArray::class -> typeOf() - LongArray::class -> typeOf() - FloatArray::class -> typeOf() - DoubleArray::class -> typeOf() - BooleanArray::class -> typeOf() - ShortArray::class -> typeOf() - ByteArray::class -> typeOf() - else -> types.getValue(klass.typeParameters[0].name) - } - } else types.getValue(klass.typeParameters[0].name) - KComplexTypeWrapper( - DataTypes.createArrayType(schema(listParam, types), listParam.isMarkedNullable), - klass.java, - type.isMarkedNullable - ) - } - klass.isSubclassOf(Map::class) -> { - val mapKeyParam = types.getValue(klass.typeParameters[0].name) - val mapValueParam = types.getValue(klass.typeParameters[1].name) - KComplexTypeWrapper( - DataTypes.createMapType( - schema(mapKeyParam, types), - schema(mapValueParam, types), - true - ), - klass.java, - type.isMarkedNullable - ) - } - klass.isData -> { - val structType = StructType( - klass - .primaryConstructor!! - .parameters - .filter { it.findAnnotation() == null } - .map { - val projectedType = types[it.type.toString()] ?: it.type - val propertyDescriptor = PropertyDescriptor( - it.name, - klass.java, - "is" + it.name?.replaceFirstChar { if (it.isLowerCase()) it.titlecase(Locale.getDefault()) else it.toString() }, - null - ) - KStructField( - propertyDescriptor.readMethod.name, - StructField( - it.name, - schema(projectedType, types), - projectedType.isMarkedNullable, - Metadata.empty() - ) - ) - } - .toTypedArray() - ) - KDataTypeWrapper(structType, klass.java, true) - } - klass.isSubclassOf(Product::class) -> { - val params = type.arguments.mapIndexed { i, it -> - "_${i + 1}" to it.type!! - } - - val structType = DataTypes.createStructType( - params.map { (fieldName, fieldType) -> - val dataType = schema(fieldType, types) - KStructField( - fieldName, - StructField(fieldName, dataType, fieldType.isMarkedNullable, Metadata.empty()) - ) - }.toTypedArray() - ) - - KComplexTypeWrapper(structType, klass.java, true) - } - else -> throw IllegalArgumentException("$type is unsupported") - } -} - -typealias SparkSession = org.apache.spark.sql.SparkSession - -fun SparkContext.setLogLevel(level: SparkLogLevel) = setLogLevel(level.name) - -enum class SparkLogLevel { - ALL, DEBUG, ERROR, FATAL, INFO, OFF, TRACE, WARN -} - -private val knownDataTypes = mapOf( - Byte::class to DataTypes.ByteType, - Short::class to DataTypes.ShortType, - Int::class to DataTypes.IntegerType, - Long::class to DataTypes.LongType, - Boolean::class to DataTypes.BooleanType, - Float::class to DataTypes.FloatType, - Double::class to DataTypes.DoubleType, - String::class to DataTypes.StringType, - LocalDate::class to `DateType$`.`MODULE$`, - Date::class to `DateType$`.`MODULE$`, - Timestamp::class to `TimestampType$`.`MODULE$`, - Instant::class to `TimestampType$`.`MODULE$` -) - -private fun transitiveMerge(a: Map, b: Map): Map { - return a + b.mapValues { - a.getOrDefault(it.value.toString(), it.value) - } -} - -class Memoize1(val f: (T) -> R) : (T) -> R { - private val values = ConcurrentHashMap() - override fun invoke(x: T) = - values.getOrPut(x, { f(x) }) -} - -private fun ((T) -> R).memoize(): (T) -> R = Memoize1(this) - -private val memoizedSchema = { x: KType -> schema(x) }.memoize() 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 deleted file mode 100644 index e49ecf84..00000000 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt +++ /dev/null @@ -1,160 +0,0 @@ -/*- - * =LICENSE= - * Kotlin Spark API: API for Spark 3.0+ (Scala 2.12) - * ---------- - * Copyright (C) 2019 - 2021 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("NOTHING_TO_INLINE", "RemoveExplicitTypeArguments", "unused") - -package org.jetbrains.kotlinx.spark.api - -import scala.collection.JavaConverters -import java.util.* -import java.util.concurrent.ConcurrentMap -import scala.collection.Iterable as ScalaIterable -import scala.collection.Iterator as ScalaIterator -import scala.collection.Map as ScalaMap -import scala.collection.Seq as ScalaSeq -import scala.collection.Set as ScalaSet -import scala.collection.concurrent.Map as ScalaConcurrentMap -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 - -/** - * @see JavaConverters.asScalaIterator for more information. - */ -fun Iterator.asScalaIterator(): ScalaIterator = JavaConverters.asScalaIterator(this) - -/** - * @see JavaConverters.enumerationAsScalaIterator for more information. - */ -fun Enumeration.asScalaIterator(): ScalaIterator = JavaConverters.enumerationAsScalaIterator(this) - -/** - * @see JavaConverters.iterableAsScalaIterable for more information. - */ -fun Iterable.asScalaIterable(): ScalaIterable = JavaConverters.iterableAsScalaIterable(this) - -/** - * @see JavaConverters.collectionAsScalaIterable for more information. - */ -fun Collection.asScalaIterable(): ScalaIterable = JavaConverters.collectionAsScalaIterable(this) - -/** - * @see JavaConverters.asScalaBuffer for more information. - */ -fun MutableList.asScalaMutableBuffer(): ScalaMutableBuffer = JavaConverters.asScalaBuffer(this) - -/** - * @see JavaConverters.asScalaSet for more information. - */ -fun MutableSet.asScalaMutableSet(): ScalaMutableSet = JavaConverters.asScalaSet(this) - -/** - * @see JavaConverters.mapAsScalaMap for more information. - */ -fun MutableMap.asScalaMutableMap(): ScalaMutableMap = JavaConverters.mapAsScalaMap(this) - -/** - * @see JavaConverters.dictionaryAsScalaMap for more information. - */ -fun Map.asScalaMap(): ScalaMap = JavaConverters.mapAsScalaMap(this) - -/** - * @see JavaConverters.mapAsScalaConcurrentMap for more information. - */ -fun ConcurrentMap.asScalaConcurrentMap(): ScalaConcurrentMap = - JavaConverters.mapAsScalaConcurrentMap(this) - -/** - * @see JavaConverters.dictionaryAsScalaMap for more information. - */ -fun Dictionary.asScalaMap(): ScalaMutableMap = JavaConverters.dictionaryAsScalaMap(this) - -/** - * @see JavaConverters.propertiesAsScalaMap for more information. - */ -fun Properties.asScalaMap(): ScalaMutableMap = JavaConverters.propertiesAsScalaMap(this) - - -/** - * @see JavaConverters.asJavaIterator for more information. - */ -fun ScalaIterator.asKotlinIterator(): Iterator = JavaConverters.asJavaIterator(this) - -/** - * @see JavaConverters.asJavaEnumeration for more information. - */ -fun ScalaIterator.asKotlinEnumeration(): Enumeration = JavaConverters.asJavaEnumeration(this) - -/** - * @see JavaConverters.asJavaIterable for more information. - */ -fun ScalaIterable.asKotlinIterable(): Iterable = JavaConverters.asJavaIterable(this) - -/** - * @see JavaConverters.asJavaCollection for more information. - */ -fun ScalaIterable.asKotlinCollection(): Collection = JavaConverters.asJavaCollection(this) - -/** - * @see JavaConverters.bufferAsJavaList for more information. - */ -fun ScalaMutableBuffer.asKotlinMutableList(): MutableList = JavaConverters.bufferAsJavaList(this) - -/** - * @see JavaConverters.mutableSeqAsJavaList for more information. - */ -fun ScalaMutableSeq.asKotlinMutableList(): MutableList = JavaConverters.mutableSeqAsJavaList(this) - -/** - * @see JavaConverters.seqAsJavaList for more information. - */ -fun ScalaSeq.asKotlinList(): List = JavaConverters.seqAsJavaList(this) - -/** - * @see JavaConverters.mutableSetAsJavaSet for more information. - */ -fun ScalaMutableSet.asKotlinMutableSet(): MutableSet = JavaConverters.mutableSetAsJavaSet(this) - -/** - * @see JavaConverters.setAsJavaSet for more information. - */ -fun ScalaSet.asKotlinSet(): Set = JavaConverters.setAsJavaSet(this) - -/** - * @see JavaConverters.mutableMapAsJavaMap for more information. - */ -fun ScalaMutableMap.asKotlinMutableMap(): MutableMap = JavaConverters.mutableMapAsJavaMap(this) - -/** - * @see JavaConverters.asJavaDictionary for more information. - */ -fun ScalaMutableMap.asKotlinDictionary(): Dictionary = JavaConverters.asJavaDictionary(this) - -/** - * @see JavaConverters.mapAsJavaMap for more information. - */ -fun ScalaMap.asKotlinMap(): Map = JavaConverters.mapAsJavaMap(this) - -/** - * @see JavaConverters.mapAsJavaConcurrentMap for more information. - */ -fun ScalaConcurrentMap.asKotlinConcurrentMap(): ConcurrentMap = - JavaConverters.mapAsJavaConcurrentMap(this) - 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 deleted file mode 100644 index 3ef0b177..00000000 --- a/kotlin-spark-api/3.2/src/main/kotlin/org/jetbrains/kotlinx/spark/api/SparkHelper.kt +++ /dev/null @@ -1,83 +0,0 @@ -/*- - * =LICENSE= - * Kotlin Spark API - * ---------- - * Copyright (C) 2019 - 2020 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.sql.SparkSession.Builder -import org.apache.spark.sql.UDFRegistration -import org.jetbrains.kotlinx.spark.api.SparkLogLevel.ERROR - -/** - * Wrapper for spark creation which allows to set different spark params - * - * @param props spark options, value types are runtime-checked for type-correctness - * @param master [SparkSession.Builder.master] - * @param appName [SparkSession.Builder.appName] - * @param func function which will be executed in context of [KSparkSession] (it means that `this` inside block will point to [KSparkSession]) - */ -@JvmOverloads -inline fun withSpark( - props: Map = emptyMap(), - master: String = "local[*]", - appName: String = "Kotlin Spark Sample", - logLevel: SparkLogLevel = ERROR, - func: KSparkSession.() -> Unit, -) { - val builder = SparkSession - .builder() - .master(master) - .appName(appName) - .apply { - props.forEach { - when (val value = it.value) { - is String -> config(it.key, value) - is Boolean -> config(it.key, value) - is Long -> config(it.key, value) - is Double -> config(it.key, value) - else -> throw IllegalArgumentException("Cannot set property ${it.key} because value $value of unsupported type ${value::class}") - } - } - } - withSpark(builder, logLevel, func) - -} - -@JvmOverloads -inline fun withSpark(builder: Builder, logLevel: SparkLogLevel = ERROR, func: KSparkSession.() -> Unit) { - builder - .orCreate - .apply { - KSparkSession(this).apply { - sparkContext.setLogLevel(logLevel) - func() - } - } - .also { it.stop() } -} - -/** - * This wrapper over [SparkSession] which provides several additional methods to create [org.apache.spark.sql.Dataset] - */ -@Suppress("EXPERIMENTAL_FEATURE_WARNING", "unused") -inline class KSparkSession(val spark: SparkSession) { - inline fun List.toDS() = toDS(spark) - inline fun Array.toDS() = spark.dsOf(*this) - inline fun dsOf(vararg arg: T) = spark.dsOf(*arg) - val udf: UDFRegistration get() = spark.udf() -} 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 deleted file mode 100644 index ed784b13..00000000 --- a/kotlin-spark-api/3.2/src/test/kotlin/org/jetbrains/kotlinx/spark/api/ApiTest.kt +++ /dev/null @@ -1,630 +0,0 @@ -package org.jetbrains.kotlinx.spark.api/*- - * =LICENSE= - * Kotlin Spark API - * ---------- - * Copyright (C) 2019 - 2020 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= - */ -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.shouldBe -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 scala.Product -import scala.Tuple1 -import scala.Tuple2 -import scala.Tuple3 -import scala.collection.Seq -import java.io.Serializable -import java.sql.Date -import java.sql.Timestamp -import java.time.Instant -import java.time.LocalDate -import kotlin.collections.Iterator -import scala.collection.Iterator as ScalaIterator -import scala.collection.Map as ScalaMap -import scala.collection.mutable.Map as ScalaMutableMap - -class ApiTest : ShouldSpec({ - context("integration tests") { - withSpark(props = mapOf("spark.sql.codegen.comments" to true)) { - should("collect data classes with doubles correctly") { - val ll1 = LonLat(1.0, 2.0) - val ll2 = LonLat(3.0, 4.0) - val lonlats = dsOf(ll1, ll2).collectAsList() - expect(lonlats).contains.inAnyOrder.only.values(ll1.copy(), ll2.copy()) - } - should("contain all generic primitives with complex schema") { - val primitives = c(1, 1.0, 1.toFloat(), 1.toByte(), LocalDate.now(), true) - val primitives2 = c(2, 2.0, 2.toFloat(), 2.toByte(), LocalDate.now().plusDays(1), false) - val tuples = dsOf(primitives, primitives2).collectAsList() - expect(tuples).contains.inAnyOrder.only.values(primitives, primitives2) - } - should("contain all generic primitives with complex nullable schema") { - val primitives = c(1, 1.0, 1.toFloat(), 1.toByte(), LocalDate.now(), true) - val nulls = c(null, null, null, null, null, null) - val tuples = dsOf(primitives, nulls).collectAsList() - expect(tuples).contains.inAnyOrder.only.values(primitives, nulls) - } - should("handle cached operations") { - val result = dsOf(1, 2, 3, 4, 5) - .map { it to (it + 2) } - .withCached { - expect(collectAsList()).contains.inAnyOrder.only.values( - 1 to 3, - 2 to 4, - 3 to 5, - 4 to 6, - 5 to 7 - ) - - val next = filter { it.first % 2 == 0 } - expect(next.collectAsList()).contains.inAnyOrder.only.values(2 to 4, 4 to 6) - next - } - .map { c(it.first, it.second, (it.first + it.second) * 2) } - .collectAsList() - expect(result).contains.inOrder.only.values(c(2, 4, 12), c(4, 6, 20)) - } - should("handle join operations") { - data class Left(val id: Int, val name: String) - - data class Right(val id: Int, val value: Int) - - val first = dsOf(Left(1, "a"), Left(2, "b")) - val second = dsOf(Right(1, 100), Right(3, 300)) - val result = first - .leftJoin(second, first.col("id").eq(second.col("id"))) - .map { c(it.first.id, it.first.name, it.second?.value) } - .collectAsList() - expect(result).contains.inOrder.only.values(c(1, "a", 100), c(2, "b", null)) - } - should("handle map operations") { - val result = dsOf(listOf(1, 2, 3, 4), listOf(3, 4, 5, 6)) - .flatMap { it.iterator() } - .map { it + 4 } - .filter { it < 10 } - .collectAsList() - expect(result).contains.inAnyOrder.only.values(5, 6, 7, 8, 7, 8, 9) - } - should("handle strings converted to lists") { - data class Movie(val id: Long, val genres: String) - data class MovieExpanded(val id: Long, val genres: List) - - val comedies = listOf(Movie(1, "Comedy|Romance"), Movie(2, "Horror|Action")).toDS() - .map { MovieExpanded(it.id, it.genres.split("|").toList()) } - .filter { it.genres.contains("Comedy") } - .collectAsList() - expect(comedies).contains.inAnyOrder.only.values( - MovieExpanded( - 1, - listOf("Comedy", "Romance") - ) - ) - } - should("handle strings converted to arrays") { - data class Movie(val id: Long, val genres: String) - data class MovieExpanded(val id: Long, val genres: Array) { - override fun equals(other: Any?): Boolean { - if (this === other) return true - if (javaClass != other?.javaClass) return false - other as MovieExpanded - return if (id != other.id) false else genres.contentEquals(other.genres) - } - - override fun hashCode(): Int { - var result = id.hashCode() - result = 31 * result + genres.contentHashCode() - return result - } - } - - val comedies = listOf(Movie(1, "Comedy|Romance"), Movie(2, "Horror|Action")).toDS() - .map { MovieExpanded(it.id, it.genres.split("|").toTypedArray()) } - .filter { it.genres.contains("Comedy") } - .collectAsList() - expect(comedies).contains.inAnyOrder.only.values( - MovieExpanded( - 1, - arrayOf("Comedy", "Romance") - ) - ) - } - should("handle arrays of generics") { - data class Test(val id: Long, val data: Array>) - - val result = listOf(Test(1, arrayOf(5.1 to 6, 6.1 to 7))) - .toDS() - .map { it.id to it.data.firstOrNull { liEl -> liEl.first < 6 } } - .map { it.second } - .collectAsList() - expect(result).contains.inOrder.only.values(5.1 to 6) - } - should("handle lists of generics") { - data class Test(val id: Long, val data: List>) - - val result = listOf(Test(1, listOf(5.1 to 6, 6.1 to 7))) - .toDS() - .map { it.id to it.data.firstOrNull { liEl -> liEl.first < 6 } } - .map { it.second } - .collectAsList() - expect(result).contains.inOrder.only.values(5.1 to 6) - } - should("!handle primitive arrays") { - val result = listOf(arrayOf(1, 2, 3, 4)) - .toDS() - .map { it.map { ai -> ai + 1 } } - .collectAsList() - .flatten() - expect(result).contains.inOrder.only.values(2, 3, 4, 5) - - } - @OptIn(ExperimentalStdlibApi::class) - should("broadcast variables") { - val largeList = (1..15).map { SomeClass(a = (it..15).toList().toIntArray(), b = it) } - val broadcast = spark.broadcast(largeList) - val broadcast2 = spark.broadcast(arrayOf(doubleArrayOf(1.0, 2.0, 3.0, 4.0))) - - val result: List = listOf(1, 2, 3, 4, 5) - .toDS() - .mapPartitions { iterator -> - val receivedBroadcast = broadcast.value - val receivedBroadcast2 = broadcast2.value - - buildList { - iterator.forEach { - this.add(it + receivedBroadcast[it].b * receivedBroadcast2[0][0]) - } - }.iterator() - } - .collectAsList() - - expect(result).contains.inOrder.only.values(3.0, 5.0, 7.0, 9.0, 11.0) - } - should("Handle JavaConversions in Kotlin") { - // Test the iterator conversion - val scalaIterator: ScalaIterator = listOf("test1", "test2").iterator().asScalaIterator() - scalaIterator.next() shouldBe "test1" - - val kotlinIterator: Iterator = scalaIterator.asKotlinIterator() - kotlinIterator.next() shouldBe "test2" - - - val scalaMap: ScalaMap = mapOf(1 to "a", 2 to "b").asScalaMap() - scalaMap.get(1).get() shouldBe "a" - scalaMap.get(2).get() shouldBe "b" - - val kotlinMap: Map = scalaMap.asKotlinMap() - kotlinMap[1] shouldBe "a" - kotlinMap[2] shouldBe "b" - - - val scalaMutableMap: ScalaMutableMap = mutableMapOf(1 to "a").asScalaMutableMap() - scalaMutableMap.get(1).get() shouldBe "a" - - scalaMutableMap.put(2, "b") - - val kotlinMutableMap: MutableMap = scalaMutableMap.asKotlinMutableMap() - kotlinMutableMap[1] shouldBe "a" - kotlinMutableMap[2] shouldBe "b" - - val scalaSeq: Seq = listOf("a", "b").iterator().asScalaIterator().toSeq() - scalaSeq.take(1).toList().last() shouldBe "a" - scalaSeq.take(2).toList().last() shouldBe "b" - - val kotlinList: List = scalaSeq.asKotlinList() - kotlinList.first() shouldBe "a" - kotlinList.last() shouldBe "b" - } - should("perform flat map on grouped datasets") { - val groupedDataset = listOf(1 to "a", 1 to "b", 2 to "c") - .toDS() - .groupByKey { it.first } - - val flatMapped = groupedDataset.flatMapGroups { key, values -> - val collected = values.asSequence().toList() - - if (collected.size > 1) collected.iterator() - else emptyList>().iterator() - } - - flatMapped.count() shouldBe 2 - } - should("perform map group with state and timeout conf on grouped datasets") { - val groupedDataset = listOf(1 to "a", 1 to "b", 2 to "c") - .toDS() - .groupByKey { it.first } - - val mappedWithStateTimeoutConf = - groupedDataset.mapGroupsWithState(GroupStateTimeout.NoTimeout()) { key, values, state: GroupState -> - var s by state - val collected = values.asSequence().toList() - - s = key - s shouldBe key - - s!! to collected.map { it.second } - } - - mappedWithStateTimeoutConf.count() shouldBe 2 - } - should("perform map group with state on grouped datasets") { - val groupedDataset = listOf(1 to "a", 1 to "b", 2 to "c") - .toDS() - .groupByKey { it.first } - - val mappedWithState = groupedDataset.mapGroupsWithState { key, values, state: GroupState -> - var s by state - val collected = values.asSequence().toList() - - s = key - s shouldBe key - - s!! to collected.map { it.second } - } - - mappedWithState.count() shouldBe 2 - } - should("perform flat map group with state on grouped datasets") { - val groupedDataset = listOf(1 to "a", 1 to "b", 2 to "c") - .toDS() - .groupByKey { it.first } - - val flatMappedWithState = groupedDataset.mapGroupsWithState { key, values, state: GroupState -> - var s by state - val collected = values.asSequence().toList() - - s = key - s shouldBe key - - if (collected.size > 1) collected.iterator() - else emptyList>().iterator() - } - - flatMappedWithState.count() shouldBe 2 - } - should("be able to cogroup grouped datasets") { - val groupedDataset1 = listOf(1 to "a", 1 to "b", 2 to "c") - .toDS() - .groupByKey { it.first } - - val groupedDataset2 = listOf(1 to "d", 5 to "e", 3 to "f") - .toDS() - .groupByKey { it.first } - - val cogrouped = groupedDataset1.cogroup(groupedDataset2) { key, left, right -> - listOf( - key to (left.asSequence() + right.asSequence()) - .map { it.second } - .toList() - ).iterator() - } - - cogrouped.count() shouldBe 4 - } - should("handle LocalDate Datasets") { // uses encoder - val dataset: Dataset = dsOf(LocalDate.now(), LocalDate.now()) - dataset.show() - } - should("handle Instant Datasets") { // uses encoder - val dataset: Dataset = dsOf(Instant.now(), Instant.now()) - dataset.show() - } - should("be able to serialize Date") { // uses knownDataTypes - val dataset: Dataset> = dsOf(Date.valueOf("2020-02-10") to 5) - dataset.show() - } - should("handle Timestamp Datasets") { // uses encoder - val dataset = dsOf(Timestamp(0L)) - dataset.show() - } - should("be able to serialize Timestamp") { // uses knownDataTypes - val dataset = dsOf(Timestamp(0L) to 2) - dataset.show() - } - should("Be able to serialize Scala Tuples including data classes") { - val dataset = dsOf( - Tuple2("a", Tuple3("a", 1, LonLat(1.0, 1.0))), - Tuple2("b", Tuple3("b", 2, LonLat(1.0, 2.0))), - ) - dataset.show() - val asList = dataset.takeAsList(2) - asList.first() shouldBe Tuple2("a", Tuple3("a", 1, LonLat(1.0, 1.0))) - } - should("Be able to serialize data classes with tuples") { - val dataset = dsOf( - DataClassWithTuple(Tuple3(5L, "test", Tuple1(""))), - DataClassWithTuple(Tuple3(6L, "tessst", Tuple1(""))), - ) - - dataset.show() - val asList = dataset.takeAsList(2) - asList.first().tuple shouldBe Tuple3(5L, "test", Tuple1("")) - } - @Suppress("UNCHECKED_CAST") - should("support dataset select") { - val dataset = dsOf( - SomeClass(intArrayOf(1, 2, 3), 3), - SomeClass(intArrayOf(1, 2, 4), 5), - ) - - val newDS1WithAs: Dataset = dataset.selectTyped( - col("a").`as`(), - ) - newDS1WithAs.show() - - 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> = dataset.selectTyped( - col(SomeClass::a), - col(SomeClass::b), - col(SomeClass::b), - ) - newDS3.show() - - val newDS4: Dataset> = dataset.selectTyped( - col(SomeClass::a), - col(SomeClass::b), - col(SomeClass::b), - col(SomeClass::b), - ) - newDS4.show() - - val newDS5: Dataset> = dataset.selectTyped( - col(SomeClass::a), - col(SomeClass::b), - col(SomeClass::b), - col(SomeClass::b), - col(SomeClass::b), - ) - newDS5.show() - } - should("Access columns using invoke on datasets") { - val dataset = dsOf( - SomeClass(intArrayOf(1, 2, 3), 4), - SomeClass(intArrayOf(4, 3, 2), 1), - ) - - dataset.col("a") shouldBe dataset("a") - } - should("Use infix- and operator funs on columns") { - val dataset = dsOf( - SomeOtherClass(intArrayOf(1, 2, 3), 4, true), - SomeOtherClass(intArrayOf(4, 3, 2), 1, true), - ) - - (dataset("a") == dataset("a")) shouldBe dataset("a").equals(dataset("a")) - (dataset("a") != dataset("a")) shouldBe !dataset("a").equals(dataset("a")) - (dataset("a") eq dataset("a")) shouldBe dataset("a").equalTo(dataset("a")) - dataset("a").equalTo(dataset("a")) shouldBe (dataset("a") `===` dataset("a")) - (dataset("a") neq dataset("a")) shouldBe dataset("a").notEqual(dataset("a")) - dataset("a").notEqual(dataset("a")) shouldBe (dataset("a") `=!=` dataset("a")) - !(dataset("a") eq dataset("a")) shouldBe dataset("a").notEqual(dataset("a")) - dataset("a").notEqual(dataset("a")) shouldBe (!(dataset("a") `===` dataset("a"))) - -dataset("b") shouldBe negate(dataset("b")) - !dataset("c") shouldBe not(dataset("c")) - dataset("b") gt 3 shouldBe dataset("b").gt(3) - dataset("b") lt 3 shouldBe dataset("b").lt(3) - dataset("b") leq 3 shouldBe dataset("b").leq(3) - dataset("b") geq 3 shouldBe dataset("b").geq(3) - dataset("b") inRangeOf 0..2 shouldBe dataset("b").between(0, 2) - dataset("c") or dataset("c") shouldBe dataset("c").or(dataset("c")) - dataset("c") and dataset("c") shouldBe dataset("c").and(dataset("c")) - dataset("c").and(dataset("c")) shouldBe (dataset("c") `&&` dataset("c")) - dataset("b") + dataset("b") shouldBe dataset("b").plus(dataset("b")) - dataset("b") - dataset("b") shouldBe dataset("b").minus(dataset("b")) - dataset("b") * dataset("b") shouldBe dataset("b").multiply(dataset("b")) - dataset("b") / dataset("b") shouldBe dataset("b").divide(dataset("b")) - dataset("b") % dataset("b") shouldBe dataset("b").mod(dataset("b")) - dataset("b")[0] shouldBe dataset("b").getItem(0) - } - should("Handle TypedColumns") { - val dataset = dsOf( - SomeOtherClass(intArrayOf(1, 2, 3), 4, true), - SomeOtherClass(intArrayOf(4, 3, 2), 1, true), - ) - - // walking over all column creation methods - val b: Dataset> = dataset.select( - dataset.col(SomeOtherClass::b), - dataset(SomeOtherClass::a), - col(SomeOtherClass::c), - ) - b.show() - } - should("Handle some where queries using column operator functions") { - val dataset = dsOf( - SomeOtherClass(intArrayOf(1, 2, 3), 4, true), - SomeOtherClass(intArrayOf(4, 3, 2), 1, true), - ) - dataset.show() - - val column = col("b").`as`() - - val b = dataset.where(column gt 3 and col(SomeOtherClass::c)) - b.show() - - b.count() shouldBe 1 - } - should("Be able to serialize lists of data classes") { - val dataset = dsOf( - listOf(SomeClass(intArrayOf(1, 2, 3), 4)), - listOf(SomeClass(intArrayOf(3, 2, 1), 0)), - ) - dataset.show() - } - should("Be able to serialize arrays of data classes") { - val dataset = dsOf( - arrayOf(SomeClass(intArrayOf(1, 2, 3), 4)), - arrayOf(SomeClass(intArrayOf(3, 2, 1), 0)), - ) - dataset.show() - } - should("Be able to serialize lists of tuples") { - val dataset = dsOf( - listOf(Tuple2(intArrayOf(1, 2, 3), 4)), - listOf(Tuple2(intArrayOf(3, 2, 1), 0)), - ) - dataset.show() - } - should("Allow simple forEachPartition in datasets") { - val dataset = dsOf( - SomeClass(intArrayOf(1, 2, 3), 1), - SomeClass(intArrayOf(4, 3, 2), 1), - ) - dataset.forEachPartition { - it.forEach { - it.b shouldBe 1 - } - } - } - should("Have easier access to keys and values for key/value datasets") { - val dataset: Dataset = dsOf( - SomeClass(intArrayOf(1, 2, 3), 1), - SomeClass(intArrayOf(4, 3, 2), 1), - ) - .groupByKey { it.b } - .reduceGroupsK { a, b -> SomeClass(a.a + b.a, a.b) } - .takeValues() - - dataset.count() shouldBe 1 - } - should("Be able to sort datasets with property reference") { - val dataset: Dataset = dsOf( - SomeClass(intArrayOf(1, 2, 3), 2), - SomeClass(intArrayOf(4, 3, 2), 1), - ) - dataset.sort(SomeClass::b) - dataset.takeAsList(1).first().b shouldBe 2 - - dataset.sort(SomeClass::a, SomeClass::b) - dataset.takeAsList(1).first().b shouldBe 2 - } - should("Have Kotlin ready functions in place of overload ambiguity") { - val dataset: Pair = dsOf( - SomeClass(intArrayOf(1, 2, 3), 1), - SomeClass(intArrayOf(4, 3, 2), 1), - ) - .groupByKey { it: SomeClass -> it.b } - .reduceGroupsK { v1: SomeClass, v2: SomeClass -> v1 } - .filter { it: Pair -> true } // not sure why this does work, but reduce doesn't - .reduceK { v1: Pair, v2: Pair -> v1 } - - dataset.second.a shouldBe intArrayOf(1, 2, 3) - } - should("Generate encoder correctly with complex enum data class") { - val dataset: Dataset = - dsOf( - ComplexEnumDataClass( - 1, - "string", - listOf("1", "2"), - SomeEnum.A, - SomeOtherEnum.C, - listOf(SomeEnum.A, SomeEnum.B), - listOf(SomeOtherEnum.C, SomeOtherEnum.D), - arrayOf(SomeEnum.A, SomeEnum.B), - arrayOf(SomeOtherEnum.C, SomeOtherEnum.D), - mapOf(SomeEnum.A to SomeOtherEnum.C) - ) - ) - - dataset.show(false) - val first = dataset.takeAsList(1).first() - - first.int shouldBe 1 - first.string shouldBe "string" - first.strings shouldBe listOf("1", "2") - first.someEnum shouldBe SomeEnum.A - first.someOtherEnum shouldBe SomeOtherEnum.C - first.someEnums shouldBe listOf(SomeEnum.A, SomeEnum.B) - first.someOtherEnums shouldBe listOf(SomeOtherEnum.C, SomeOtherEnum.D) - first.someEnumArray shouldBe arrayOf(SomeEnum.A, SomeEnum.B) - first.someOtherArray shouldBe arrayOf(SomeOtherEnum.C, SomeOtherEnum.D) - first.enumMap shouldBe mapOf(SomeEnum.A to SomeOtherEnum.C) - } - should("work with lists of maps") { - val result = dsOf( - listOf(mapOf("a" to "b", "x" to "y")), - listOf(mapOf("a" to "b", "x" to "y")), - listOf(mapOf("a" to "b", "x" to "y")) - ) - .showDS() - .map { it.last() } - .map { it["x"] } - .filterNotNull() - .distinct() - .collectAsList() - expect(result).contains.inOrder.only.value("y") - } - should("work with lists of lists") { - val result = dsOf( - listOf(listOf(1, 2, 3)), - listOf(listOf(1, 2, 3)), - listOf(listOf(1, 2, 3)) - ) - .map { it.last() } - .map { it.first() } - .reduceK { a, b -> a + b } - expect(result).toBe(3) - } - should("Generate schema correctly with nullalble list and map") { - val schema = encoder().schema() - schema.fields().forEach { - it.nullable() shouldBe true - } - } - } - } -}) - -data class DataClassWithTuple(val tuple: T) - -data class LonLat(val lon: Double, val lat: Double) - -// (data) class must be Serializable to be broadcast -data class SomeClass(val a: IntArray, val b: Int) : Serializable - -data class SomeOtherClass(val a: IntArray, val b: Int, val c: Boolean) : Serializable - - -enum class SomeEnum { A, B } - -enum class SomeOtherEnum(val value: Int) { C(1), D(2) } - -data class ComplexEnumDataClass( - val int: Int, - val string: String, - val strings: List, - val someEnum: SomeEnum, - val someOtherEnum: SomeOtherEnum, - val someEnums: List, - val someOtherEnums: List, - val someEnumArray: Array, - val someOtherArray: Array, - val enumMap: Map, -) - -data class NullFieldAbleDataClass( - val optionList: List?, - val optionMap: Map? -) \ No newline at end of file diff --git a/kotlin-spark-api/common/pom.xml b/kotlin-spark-api/common/pom.xml deleted file mode 100644 index bdbb3cca..00000000 --- a/kotlin-spark-api/common/pom.xml +++ /dev/null @@ -1,56 +0,0 @@ - - - 4.0.0 - - Kotlin Spark API: Common - kotlin-spark-api-common - Kotlin API for Apache Spark: common parts - - org.jetbrains.kotlinx.spark - kotlin-spark-api-parent - 1.0.3 - ../.. - - - - - org.jetbrains.kotlin - kotlin-stdlib-jdk8 - - - - - src/main/kotlin - src/test/kotlin - - - org.jetbrains.kotlin - kotlin-maven-plugin - - - org.jetbrains.dokka - dokka-maven-plugin - ${dokka.version} - - 8 - - - - dokka - - dokka - - pre-site - - - javadocjar - - javadocJar - - pre-integration-test - - - - - - diff --git a/pom.xml b/pom.xml index c0cd569e..19b3632e 100644 --- a/pom.xml +++ b/pom.xml @@ -2,40 +2,44 @@ 4.0.0 - Kotlin Spark API: Parent + Kotlin Spark API: Parent for Spark 3.0+ Parent project for Kotlin for Apache Spark org.jetbrains.kotlinx.spark - kotlin-spark-api-parent - 1.0.3 + kotlin-spark-api-parent-3.0 + 1.1.0 pom - 1.5.30 - 1.4.32 - 0.16.0 - 4.6.0 - 1.0.1 - 3.2.0 + 0.17.0 + 1.6.10 + 3.1.0 + 3.3.1 + 1.3.1 + 5.2.3 + 0.11.0-95 + 1.6.21 + 0.7.5 + 3.0.3 - 2.10.0 0.8.7 5.5 + official 2.0.0 3.3.0 - 3.0.1 + 3.10.1 3.0.0-M1 3.0.0-M3 + 3.0.1 3.2.0 3.9.1 3.2.1 - 3.0.0-M5 + 3.0.0-M6 1.6.8 4.5.6 - kotlin-spark-api/common dummy @@ -51,11 +55,6 @@ kotlin-reflect ${kotlin.version} - - org.jetbrains.kotlinx.spark - kotlin-spark-api-common - ${project.version} - @@ -72,11 +71,6 @@ nexus-staging-maven-plugin ${nexus-staging-plugin.version} - - io.qameta.allure - allure-maven - ${allure-maven.version} - org.jacoco jacoco-maven-plugin @@ -180,6 +174,12 @@ update-file-header + + + **/*.json + **/*.css + + process-sources @@ -229,19 +229,7 @@ true false forked-path - scala-2.12,release-sign - - - - org.sonatype.plugins - nexus-staging-maven-plugin - ${nexus-staging-plugin.version} - true - - ossrh - https://oss.sonatype.org/ - false - 20 + scala-2.12,central-deploy @@ -279,7 +267,7 @@ scm:git:https://github.com/JetBrains/kotlin-spark-api.git https://github.com/JetBrains/kotlin-spark-api - kotlin-spark-3.2-1.0.3 + 3.0-1.1.0 @@ -287,8 +275,11 @@ https://oss.sonatype.org/service/local/staging/deploy/maven2/ - ossrh - https://oss.sonatype.org/content/repositories/snapshots + github + GitHub JetBrains Apache Maven Packages + https://maven.pkg.github.com/JetBrains/kotlin-spark-api + + @@ -303,7 +294,7 @@ - release-sign + central-deploy performRelease @@ -312,6 +303,18 @@ + + org.sonatype.plugins + nexus-staging-maven-plugin + ${nexus-staging-plugin.version} + true + + ossrh + https://oss.sonatype.org/ + false + 20 + + org.apache.maven.plugins maven-gpg-plugin diff --git a/pom_2.12.xml b/pom_2.12.xml index 1fc53d70..55044bf0 100644 --- a/pom_2.12.xml +++ b/pom_2.12.xml @@ -4,31 +4,39 @@ Kotlin Spark API: Parent (Scala 2.12) Parent project for Kotlin for Apache Spark - kotlin-spark-api-parent_2.12 + kotlin-spark-api-parent-3.0_2.12 org.jetbrains.kotlinx.spark - kotlin-spark-api-parent - 1.0.3 + kotlin-spark-api-parent-3.0 + 1.1.0 pom.xml pom + true 2.12.15 2.12 - core/3.2/pom_2.12.xml - kotlin-spark-api/3.2/pom_2.12.xml - examples/pom-3.2_2.12.xml + core/3.0/pom_2.12.xml + scala-tuples-in-kotlin/pom_2.12.xml + kotlin-spark-api/3.0/pom_2.12.xml + examples/pom-3.0_2.12.xml + jupyter org.jetbrains.kotlinx.spark - core-3.2_${scala.compat.version} + core-3.0_${scala.compat.version} + ${project.version} + + + org.jetbrains.kotlinx.spark + scala-tuples-in-kotlin ${project.version} diff --git a/qodana.yaml b/qodana.yaml new file mode 100644 index 00000000..3c9f0225 --- /dev/null +++ b/qodana.yaml @@ -0,0 +1,9 @@ +version: "1.0" +linter: jetbrains/qodana-jvm-community:2021.3 +profile: + name: qodana.recommended +exclude: + - name: All + paths: + - scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples + - kotlin-spark-api/3.0/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Arities.kt diff --git a/scala-tuples-in-kotlin/pom_2.12.xml b/scala-tuples-in-kotlin/pom_2.12.xml new file mode 100644 index 00000000..687de2ac --- /dev/null +++ b/scala-tuples-in-kotlin/pom_2.12.xml @@ -0,0 +1,153 @@ + + + 4.0.0 + + Kotlin Spark API: Scala Tuples in Kotlin + Scala Tuple helper functions for kotlin + scala-tuples-in-kotlin + + org.jetbrains.kotlinx.spark + kotlin-spark-api-parent-3.0_2.12 + 1.1.0 + ../pom_2.12.xml + + + + + org.scala-lang + scala-library + ${scala.version} + + + org.jetbrains.kotlin + kotlin-stdlib-jdk8 + ${kotlin.version} + + + + + io.kotest + kotest-runner-junit5-jvm + ${kotest.version} + test + + + com.beust + klaxon + ${klaxon.version} + test + + + ch.tutteli.atrium + atrium-fluent-en_GB + ${atrium.version} + test + + + + org.jetbrains.kotlin + kotlin-test + ${kotlin.version} + test + + + + + src/main/kotlin + src/test/kotlin + target/${scala.compat.version} + + + + org.jetbrains.dokka + dokka-maven-plugin + ${dokka.version} + + 8 + + + + dokka + + dokka + + pre-site + + + javadocjar + + javadocJar + + pre-integration-test + + + + + + org.jetbrains.kotlin + kotlin-maven-plugin + ${kotlin.version} + + + compile + compile + + compile + + + + test-compile + test-compile + + test-compile + + + + + 1.8 + + + + + org.apache.maven.plugins + maven-assembly-plugin + ${maven-assembly-plugin.version} + + + jar-with-dependencies + + + + org.jetbrains.spark.api.examples.WordCountKt + + + + + + + org.apache.maven.plugins + maven-site-plugin + + true + + + + + org.apache.maven.plugins + maven-deploy-plugin + + false + + + + + org.sonatype.plugins + nexus-staging-maven-plugin + + false + + + + + + diff --git a/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt b/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt new file mode 100644 index 00000000..ce4f7e83 --- /dev/null +++ b/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/Conversions.kt @@ -0,0 +1,51 @@ +/*- + * =LICENSE= + * Kotlin Spark API: API for Spark 3.0+ (Scala 2.12) + * ---------- + * Copyright (C) 2019 - 2021 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= + */ + +/** + * This files contains conversions of Tuples between the Scala- + * and Kotlin/Java variants. + */ + +@file:Suppress("NOTHING_TO_INLINE", "RemoveExplicitTypeArguments", "unused") + +package org.jetbrains.kotlinx.spark.api + +import scala.* + + +/** + * Returns a new [Tuple2] based on the arguments in the current [Pair]. + */ +fun Pair.toTuple(): Tuple2 = Tuple2(first, second) + +/** + * Returns a new [Pair] based on the arguments in the current [Tuple2]. + */ +fun Tuple2.toPair(): Pair = Pair(_1(), _2()) + +/** + * Returns a new [Tuple3] based on the arguments in the current [Triple]. + */ +fun Triple.toTuple(): Tuple3 = Tuple3(first, second, third) + +/** + * Returns a new [Triple] based on the arguments in the current [Tuple3]. + */ +fun Tuple3.toTriple(): Triple = Triple(_1(), _2(), _3()) diff --git a/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/DestructuredTupleBuilders.kt b/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/DestructuredTupleBuilders.kt new file mode 100644 index 00000000..a212e3aa --- /dev/null +++ b/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/DestructuredTupleBuilders.kt @@ -0,0 +1,83 @@ +/*- + * =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("RemoveExplicitTypeArguments", "FunctionName") + +package org.jetbrains.kotlinx.spark.api.tuples + +import scala.Tuple2 +import scala.Tuple3 +import scala.Tuple4 +import scala.Tuple5 +import scala.Tuple6 +import scala.Tuple7 +import scala.Tuple8 +import scala.Tuple9 +import scala.Tuple10 +import scala.Tuple11 +import scala.Tuple12 +import scala.Tuple13 +import scala.Tuple14 +import scala.Tuple15 +import scala.Tuple16 +import scala.Tuple17 +import scala.Tuple18 +import scala.Tuple19 +import scala.Tuple20 +import scala.Tuple21 +import scala.Tuple22 + +/** + * This file provides a descriptive way to create Tuples using [X]. + * Only use [X] to create new Tuples. + * To create Tuples of Tuples, it's recommended to use [t] or [tupleOf] instead as using [X] can lead + * to unexpected results. + * + * For instance: + * ```val yourTuple = 1 X "test" X a``` + * + */ + +/** + * Returns a new Tuple2 of the given arguments. + * @see tupleOf + * @see t + **/ +infix fun T1.X(other: T2): Tuple2 = Tuple2(this, other) + +infix fun Tuple2.X(next: T3): Tuple3 = Tuple3(this._1(), this._2(), next) +infix fun Tuple3.X(next: T4): Tuple4 = Tuple4(this._1(), this._2(), this._3(), next) +infix fun Tuple4.X(next: T5): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), next) +infix fun Tuple5.X(next: T6): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), next) +infix fun Tuple6.X(next: T7): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), next) +infix fun Tuple7.X(next: T8): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), next) +infix fun Tuple8.X(next: T9): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), next) +infix fun Tuple9.X(next: T10): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), next) +infix fun Tuple10.X(next: T11): Tuple11 = Tuple11(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), next) +infix fun Tuple11.X(next: T12): Tuple12 = Tuple12(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), next) +infix fun Tuple12.X(next: T13): 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(), next) +infix fun Tuple13.X(next: T14): 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(), next) +infix fun Tuple14.X(next: T15): 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(), next) +infix fun Tuple15.X(next: T16): 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(), next) +infix fun Tuple16.X(next: T17): 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(), next) +infix fun Tuple17.X(next: T18): 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(), next) +infix fun Tuple18.X(next: T19): 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(), next) +infix fun Tuple19.X(next: T20): 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(), next) +infix fun Tuple20.X(next: T21): 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(), next) +infix fun Tuple21.X(next: T22): 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(), next) diff --git a/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/DropFunctions.kt b/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/DropFunctions.kt new file mode 100644 index 00000000..f264245f --- /dev/null +++ b/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/DropFunctions.kt @@ -0,0 +1,99 @@ +/*- + * =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.tuples + +import scala.Tuple1 +import scala.Tuple2 +import scala.Tuple3 +import scala.Tuple4 +import scala.Tuple5 +import scala.Tuple6 +import scala.Tuple7 +import scala.Tuple8 +import scala.Tuple9 +import scala.Tuple10 +import scala.Tuple11 +import scala.Tuple12 +import scala.Tuple13 +import scala.Tuple14 +import scala.Tuple15 +import scala.Tuple16 +import scala.Tuple17 +import scala.Tuple18 +import scala.Tuple19 +import scala.Tuple20 +import scala.Tuple21 +import scala.Tuple22 + +/** + * This file contains functions to lower the amount of dimensions of tuples. + * This can be done using [dropFirst] and [dropLast]. + * + * For example: + * ```kotlin + * val yourTuple: Tuple2 = tupleOf(1, "test", a).dropLast() + * ``` + * + */ + +fun Tuple1<*>.dropFirst(): EmptyTuple = EmptyTuple +fun Tuple1<*>.dropLast(): EmptyTuple = EmptyTuple +fun Tuple2<*, T1>.dropFirst(): Tuple1 = Tuple1(this._2()) +fun Tuple2.dropLast(): Tuple1 = Tuple1(this._1()) +fun Tuple3<*, T1, T2>.dropFirst(): Tuple2 = Tuple2(this._2(), this._3()) +fun Tuple3.dropLast(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple4<*, T1, T2, T3>.dropFirst(): Tuple3 = Tuple3(this._2(), this._3(), this._4()) +fun Tuple4.dropLast(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple5<*, T1, T2, T3, T4>.dropFirst(): Tuple4 = Tuple4(this._2(), this._3(), this._4(), this._5()) +fun Tuple5.dropLast(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple6<*, T1, T2, T3, T4, T5>.dropFirst(): Tuple5 = Tuple5(this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple6.dropLast(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple7<*, T1, T2, T3, T4, T5, T6>.dropFirst(): Tuple6 = Tuple6(this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple7.dropLast(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple8<*, T1, T2, T3, T4, T5, T6, T7>.dropFirst(): Tuple7 = Tuple7(this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple8.dropLast(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple9<*, T1, T2, T3, T4, T5, T6, T7, T8>.dropFirst(): Tuple8 = Tuple8(this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple9.dropLast(): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple10<*, T1, T2, T3, T4, T5, T6, T7, T8, T9>.dropFirst(): Tuple9 = Tuple9(this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple10.dropLast(): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple11<*, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>.dropFirst(): Tuple10 = Tuple10(this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11()) +fun Tuple11.dropLast(): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple12<*, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11>.dropFirst(): Tuple11 = Tuple11(this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12()) +fun Tuple12.dropLast(): 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 Tuple13<*, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12>.dropFirst(): Tuple12 = Tuple12(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.dropLast(): 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 Tuple14<*, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>.dropFirst(): Tuple13 = Tuple13(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.dropLast(): 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 Tuple15<*, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>.dropFirst(): Tuple14 = Tuple14(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.dropLast(): 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 Tuple16<*, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>.dropFirst(): Tuple15 = Tuple15(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.dropLast(): 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 Tuple17<*, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>.dropFirst(): Tuple16 = Tuple16(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.dropLast(): 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 Tuple18<*, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>.dropFirst(): Tuple17 = Tuple17(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.dropLast(): 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 Tuple19<*, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>.dropFirst(): Tuple18 = Tuple18(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.dropLast(): 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 Tuple20<*, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>.dropFirst(): Tuple19 = Tuple19(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.dropLast(): 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 Tuple21<*, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>.dropFirst(): Tuple20 = Tuple20(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.dropLast(): 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 Tuple22<*, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>.dropFirst(): Tuple21 = Tuple21(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.dropLast(): 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()) diff --git a/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/EmptyTuple.kt b/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/EmptyTuple.kt new file mode 100644 index 00000000..0420c8b7 --- /dev/null +++ b/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/EmptyTuple.kt @@ -0,0 +1,37 @@ +/*- + * =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.tuples + +import scala.* +import java.io.Serializable + +/** + * Just as in Scala3, we provide the [EmptyTuple]. It is the result of dropping the last item from a [Tuple1] + * or when calling `tupleOf()` for instance. + */ + +object EmptyTuple : Product, Serializable { + override fun canEqual(that: Any?): Boolean = that == EmptyTuple + override fun productElement(n: Int): Nothing = throw IndexOutOfBoundsException("EmptyTuple has no members") + override fun productArity(): Int = 0 + override fun toString(): String = "()" +} + +public fun emptyTuple(): EmptyTuple = EmptyTuple diff --git a/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/MapTuples.kt b/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/MapTuples.kt new file mode 100644 index 00000000..164ac92a --- /dev/null +++ b/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/MapTuples.kt @@ -0,0 +1,87 @@ +/*- + * =LICENSE= + * Kotlin Spark API: Scala Tuples in Kotlin + * ---------- + * 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.tuples + +import scala.* + +/** + * This file provides map-functions to all Tuple variants. + * Given a tuple `t(a1, ..., an)`, returns a new tuple `t(func(a1), ..., func(an))`. + * Compared to Scala 3, no type mapping can occur in Kotlin, so to create a `TupleX` + * the user will need to explicitly [cast] the result. + * + * For example: + * ```kotlin + * val myTuple: Tuple4 = t(1, "3", 2, "4") + * val myStringTuple: Tuple4 = myTuple.map { + * when (it) { + * is Int -> it.toString() + * is String -> it.toInt() + * else -> error("") + * } + * }.cast() + * ``` + */ + +fun Tuple1.map(func: (T) -> R): Tuple1 = Tuple1(func(this._1())) +fun Tuple2.map(func: (T) -> R): Tuple2 = Tuple2(func(this._1()), func(this._2())) +fun Tuple3.map(func: (T) -> R): Tuple3 = Tuple3(func(this._1()), func(this._2()), func(this._3())) +fun Tuple4.map(func: (T) -> R): Tuple4 = Tuple4(func(this._1()), func(this._2()), func(this._3()), func(this._4())) +fun Tuple5.map(func: (T) -> R): Tuple5 = Tuple5(func(this._1()), func(this._2()), func(this._3()), func(this._4()), func(this._5())) +fun Tuple6.map(func: (T) -> R): Tuple6 = Tuple6(func(this._1()), func(this._2()), func(this._3()), func(this._4()), func(this._5()), func(this._6())) +fun Tuple7.map(func: (T) -> R): Tuple7 = Tuple7(func(this._1()), func(this._2()), func(this._3()), func(this._4()), func(this._5()), func(this._6()), func(this._7())) +fun Tuple8.map(func: (T) -> R): Tuple8 = Tuple8(func(this._1()), func(this._2()), func(this._3()), func(this._4()), func(this._5()), func(this._6()), func(this._7()), func(this._8())) +fun Tuple9.map(func: (T) -> R): Tuple9 = Tuple9(func(this._1()), func(this._2()), func(this._3()), func(this._4()), func(this._5()), func(this._6()), func(this._7()), func(this._8()), func(this._9())) +fun Tuple10.map(func: (T) -> R): Tuple10 = Tuple10(func(this._1()), func(this._2()), func(this._3()), func(this._4()), func(this._5()), func(this._6()), func(this._7()), func(this._8()), func(this._9()), func(this._10())) +fun Tuple11.map(func: (T) -> R): Tuple11 = Tuple11(func(this._1()), func(this._2()), func(this._3()), func(this._4()), func(this._5()), func(this._6()), func(this._7()), func(this._8()), func(this._9()), func(this._10()), func(this._11())) +fun Tuple12.map(func: (T) -> R): Tuple12 = Tuple12(func(this._1()), func(this._2()), func(this._3()), func(this._4()), func(this._5()), func(this._6()), func(this._7()), func(this._8()), func(this._9()), func(this._10()), func(this._11()), func(this._12())) +fun Tuple13.map(func: (T) -> R): Tuple13 = Tuple13(func(this._1()), func(this._2()), func(this._3()), func(this._4()), func(this._5()), func(this._6()), func(this._7()), func(this._8()), func(this._9()), func(this._10()), func(this._11()), func(this._12()), func(this._13())) +fun Tuple14.map(func: (T) -> R): Tuple14 = Tuple14(func(this._1()), func(this._2()), func(this._3()), func(this._4()), func(this._5()), func(this._6()), func(this._7()), func(this._8()), func(this._9()), func(this._10()), func(this._11()), func(this._12()), func(this._13()), func(this._14())) +fun Tuple15.map(func: (T) -> R): Tuple15 = Tuple15(func(this._1()), func(this._2()), func(this._3()), func(this._4()), func(this._5()), func(this._6()), func(this._7()), func(this._8()), func(this._9()), func(this._10()), func(this._11()), func(this._12()), func(this._13()), func(this._14()), func(this._15())) +fun Tuple16.map(func: (T) -> R): Tuple16 = Tuple16(func(this._1()), func(this._2()), func(this._3()), func(this._4()), func(this._5()), func(this._6()), func(this._7()), func(this._8()), func(this._9()), func(this._10()), func(this._11()), func(this._12()), func(this._13()), func(this._14()), func(this._15()), func(this._16())) +fun Tuple17.map(func: (T) -> R): Tuple17 = Tuple17(func(this._1()), func(this._2()), func(this._3()), func(this._4()), func(this._5()), func(this._6()), func(this._7()), func(this._8()), func(this._9()), func(this._10()), func(this._11()), func(this._12()), func(this._13()), func(this._14()), func(this._15()), func(this._16()), func(this._17())) +fun Tuple18.map(func: (T) -> R): Tuple18 = Tuple18(func(this._1()), func(this._2()), func(this._3()), func(this._4()), func(this._5()), func(this._6()), func(this._7()), func(this._8()), func(this._9()), func(this._10()), func(this._11()), func(this._12()), func(this._13()), func(this._14()), func(this._15()), func(this._16()), func(this._17()), func(this._18())) +fun Tuple19.map(func: (T) -> R): Tuple19 = Tuple19(func(this._1()), func(this._2()), func(this._3()), func(this._4()), func(this._5()), func(this._6()), func(this._7()), func(this._8()), func(this._9()), func(this._10()), func(this._11()), func(this._12()), func(this._13()), func(this._14()), func(this._15()), func(this._16()), func(this._17()), func(this._18()), func(this._19())) +fun Tuple20.map(func: (T) -> R): Tuple20 = Tuple20(func(this._1()), func(this._2()), func(this._3()), func(this._4()), func(this._5()), func(this._6()), func(this._7()), func(this._8()), func(this._9()), func(this._10()), func(this._11()), func(this._12()), func(this._13()), func(this._14()), func(this._15()), func(this._16()), func(this._17()), func(this._18()), func(this._19()), func(this._20())) +fun Tuple21.map(func: (T) -> R): Tuple21 = Tuple21(func(this._1()), func(this._2()), func(this._3()), func(this._4()), func(this._5()), func(this._6()), func(this._7()), func(this._8()), func(this._9()), func(this._10()), func(this._11()), func(this._12()), func(this._13()), func(this._14()), func(this._15()), func(this._16()), func(this._17()), func(this._18()), func(this._19()), func(this._20()), func(this._21())) +fun Tuple22.map(func: (T) -> R): Tuple22 = Tuple22(func(this._1()), func(this._2()), func(this._3()), func(this._4()), func(this._5()), func(this._6()), func(this._7()), func(this._8()), func(this._9()), func(this._10()), func(this._11()), func(this._12()), func(this._13()), func(this._14()), func(this._15()), func(this._16()), func(this._17()), func(this._18()), func(this._19()), func(this._20()), func(this._21()), func(this._22())) + +inline fun Tuple1<*>.cast(): Tuple1 = Tuple1(this._1() as T1) +inline fun Tuple2<*, *>.cast(): Tuple2 = Tuple2(this._1() as T1, this._2() as T2) +inline fun Tuple3<*, *, *>.cast(): Tuple3 = Tuple3(this._1() as T1, this._2() as T2, this._3() as T3) +inline fun Tuple4<*, *, *, *>.cast(): Tuple4 = Tuple4(this._1() as T1, this._2() as T2, this._3() as T3, this._4() as T4) +inline fun Tuple5<*, *, *, *, *>.cast(): Tuple5 = Tuple5(this._1() as T1, this._2() as T2, this._3() as T3, this._4() as T4, this._5() as T5) +inline fun Tuple6<*, *, *, *, *, *>.cast(): Tuple6 = Tuple6(this._1() as T1, this._2() as T2, this._3() as T3, this._4() as T4, this._5() as T5, this._6() as T6) +inline fun Tuple7<*, *, *, *, *, *, *>.cast(): Tuple7 = Tuple7(this._1() as T1, this._2() as T2, this._3() as T3, this._4() as T4, this._5() as T5, this._6() as T6, this._7() as T7) +inline fun Tuple8<*, *, *, *, *, *, *, *>.cast(): Tuple8 = Tuple8(this._1() as T1, this._2() as T2, this._3() as T3, this._4() as T4, this._5() as T5, this._6() as T6, this._7() as T7, this._8() as T8) +inline fun Tuple9<*, *, *, *, *, *, *, *, *>.cast(): Tuple9 = Tuple9(this._1() as T1, this._2() as T2, this._3() as T3, this._4() as T4, this._5() as T5, this._6() as T6, this._7() as T7, this._8() as T8, this._9() as T9) +inline fun Tuple10<*, *, *, *, *, *, *, *, *, *>.cast(): Tuple10 = Tuple10(this._1() as T1, this._2() as T2, this._3() as T3, this._4() as T4, this._5() as T5, this._6() as T6, this._7() as T7, this._8() as T8, this._9() as T9, this._10() as T10) +inline fun Tuple11<*, *, *, *, *, *, *, *, *, *, *>.cast(): Tuple11 = Tuple11(this._1() as T1, this._2() as T2, this._3() as T3, this._4() as T4, this._5() as T5, this._6() as T6, this._7() as T7, this._8() as T8, this._9() as T9, this._10() as T10, this._11() as T11) +inline fun Tuple12<*, *, *, *, *, *, *, *, *, *, *, *>.cast(): Tuple12 = Tuple12(this._1() as T1, this._2() as T2, this._3() as T3, this._4() as T4, this._5() as T5, this._6() as T6, this._7() as T7, this._8() as T8, this._9() as T9, this._10() as T10, this._11() as T11, this._12() as T12) +inline fun Tuple13<*, *, *, *, *, *, *, *, *, *, *, *, *>.cast(): Tuple13 = Tuple13(this._1() as T1, this._2() as T2, this._3() as T3, this._4() as T4, this._5() as T5, this._6() as T6, this._7() as T7, this._8() as T8, this._9() as T9, this._10() as T10, this._11() as T11, this._12() as T12, this._13() as T13) +inline fun Tuple14<*, *, *, *, *, *, *, *, *, *, *, *, *, *>.cast(): Tuple14 = Tuple14(this._1() as T1, this._2() as T2, this._3() as T3, this._4() as T4, this._5() as T5, this._6() as T6, this._7() as T7, this._8() as T8, this._9() as T9, this._10() as T10, this._11() as T11, this._12() as T12, this._13() as T13, this._14() as T14) +inline fun Tuple15<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.cast(): Tuple15 = Tuple15(this._1() as T1, this._2() as T2, this._3() as T3, this._4() as T4, this._5() as T5, this._6() as T6, this._7() as T7, this._8() as T8, this._9() as T9, this._10() as T10, this._11() as T11, this._12() as T12, this._13() as T13, this._14() as T14, this._15() as T15) +inline fun Tuple16<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.cast(): Tuple16 = Tuple16(this._1() as T1, this._2() as T2, this._3() as T3, this._4() as T4, this._5() as T5, this._6() as T6, this._7() as T7, this._8() as T8, this._9() as T9, this._10() as T10, this._11() as T11, this._12() as T12, this._13() as T13, this._14() as T14, this._15() as T15, this._16() as T16) +inline fun Tuple17<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.cast(): Tuple17 = Tuple17(this._1() as T1, this._2() as T2, this._3() as T3, this._4() as T4, this._5() as T5, this._6() as T6, this._7() as T7, this._8() as T8, this._9() as T9, this._10() as T10, this._11() as T11, this._12() as T12, this._13() as T13, this._14() as T14, this._15() as T15, this._16() as T16, this._17() as T17) +inline fun Tuple18<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.cast(): Tuple18 = Tuple18(this._1() as T1, this._2() as T2, this._3() as T3, this._4() as T4, this._5() as T5, this._6() as T6, this._7() as T7, this._8() as T8, this._9() as T9, this._10() as T10, this._11() as T11, this._12() as T12, this._13() as T13, this._14() as T14, this._15() as T15, this._16() as T16, this._17() as T17, this._18() as T18) +inline fun Tuple19<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.cast(): Tuple19 = Tuple19(this._1() as T1, this._2() as T2, this._3() as T3, this._4() as T4, this._5() as T5, this._6() as T6, this._7() as T7, this._8() as T8, this._9() as T9, this._10() as T10, this._11() as T11, this._12() as T12, this._13() as T13, this._14() as T14, this._15() as T15, this._16() as T16, this._17() as T17, this._18() as T18, this._19() as T19) +inline fun Tuple20<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.cast(): Tuple20 = Tuple20(this._1() as T1, this._2() as T2, this._3() as T3, this._4() as T4, this._5() as T5, this._6() as T6, this._7() as T7, this._8() as T8, this._9() as T9, this._10() as T10, this._11() as T11, this._12() as T12, this._13() as T13, this._14() as T14, this._15() as T15, this._16() as T16, this._17() as T17, this._18() as T18, this._19() as T19, this._20() as T20) +inline fun Tuple21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.cast(): Tuple21 = Tuple21(this._1() as T1, this._2() as T2, this._3() as T3, this._4() as T4, this._5() as T5, this._6() as T6, this._7() as T7, this._8() as T8, this._9() as T9, this._10() as T10, this._11() as T11, this._12() as T12, this._13() as T13, this._14() as T14, this._15() as T15, this._16() as T16, this._17() as T17, this._18() as T18, this._19() as T19, this._20() as T20, this._21() as T21) +inline fun Tuple22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.cast(): Tuple22 = Tuple22(this._1() as T1, this._2() as T2, this._3() as T3, this._4() as T4, this._5() as T5, this._6() as T6, this._7() as T7, this._8() as T8, this._9() as T9, this._10() as T10, this._11() as T11, this._12() as T12, this._13() as T13, this._14() as T14, this._15() as T15, this._16() as T16, this._17() as T17, this._18() as T18, this._19() as T19, this._20() as T20, this._21() as T21, this._22() as T22) diff --git a/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/ProductDestructuring.kt b/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/ProductDestructuring.kt new file mode 100644 index 00000000..e5abefa8 --- /dev/null +++ b/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/ProductDestructuring.kt @@ -0,0 +1,306 @@ +/*- + * =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.tuples + +import scala.Product1 +import scala.Product2 +import scala.Product3 +import scala.Product4 +import scala.Product5 +import scala.Product6 +import scala.Product7 +import scala.Product8 +import scala.Product9 +import scala.Product10 +import scala.Product11 +import scala.Product12 +import scala.Product13 +import scala.Product14 +import scala.Product15 +import scala.Product16 +import scala.Product17 +import scala.Product18 +import scala.Product19 +import scala.Product20 +import scala.Product21 +import scala.Product22 + +/** + * + * This file provides the operator functions to destructuring for Scala classes implementing ProductX, like Tuples. + * + * This means you can type `val (a, b, c, d) = yourTuple` to unpack its values, + * similar to how [Pair], [Triple] and other data classes work in Kotlin. + * + */ + +operator fun Product1.component1(): T = this._1() +operator fun Product2.component1(): T = this._1() +operator fun Product2<*, T>.component2(): T = this._2() +operator fun Product3.component1(): T = this._1() +operator fun Product3<*, T, *>.component2(): T = this._2() +operator fun Product3<*, *, T>.component3(): T = this._3() +operator fun Product4.component1(): T = this._1() +operator fun Product4<*, T, *, *>.component2(): T = this._2() +operator fun Product4<*, *, T, *>.component3(): T = this._3() +operator fun Product4<*, *, *, T>.component4(): T = this._4() +operator fun Product5.component1(): T = this._1() +operator fun Product5<*, T, *, *, *>.component2(): T = this._2() +operator fun Product5<*, *, T, *, *>.component3(): T = this._3() +operator fun Product5<*, *, *, T, *>.component4(): T = this._4() +operator fun Product5<*, *, *, *, T>.component5(): T = this._5() +operator fun Product6.component1(): T = this._1() +operator fun Product6<*, T, *, *, *, *>.component2(): T = this._2() +operator fun Product6<*, *, T, *, *, *>.component3(): T = this._3() +operator fun Product6<*, *, *, T, *, *>.component4(): T = this._4() +operator fun Product6<*, *, *, *, T, *>.component5(): T = this._5() +operator fun Product6<*, *, *, *, *, T>.component6(): T = this._6() +operator fun Product7.component1(): T = this._1() +operator fun Product7<*, T, *, *, *, *, *>.component2(): T = this._2() +operator fun Product7<*, *, T, *, *, *, *>.component3(): T = this._3() +operator fun Product7<*, *, *, T, *, *, *>.component4(): T = this._4() +operator fun Product7<*, *, *, *, T, *, *>.component5(): T = this._5() +operator fun Product7<*, *, *, *, *, T, *>.component6(): T = this._6() +operator fun Product7<*, *, *, *, *, *, T>.component7(): T = this._7() +operator fun Product8.component1(): T = this._1() +operator fun Product8<*, T, *, *, *, *, *, *>.component2(): T = this._2() +operator fun Product8<*, *, T, *, *, *, *, *>.component3(): T = this._3() +operator fun Product8<*, *, *, T, *, *, *, *>.component4(): T = this._4() +operator fun Product8<*, *, *, *, T, *, *, *>.component5(): T = this._5() +operator fun Product8<*, *, *, *, *, T, *, *>.component6(): T = this._6() +operator fun Product8<*, *, *, *, *, *, T, *>.component7(): T = this._7() +operator fun Product8<*, *, *, *, *, *, *, T>.component8(): T = this._8() +operator fun Product9.component1(): T = this._1() +operator fun Product9<*, T, *, *, *, *, *, *, *>.component2(): T = this._2() +operator fun Product9<*, *, T, *, *, *, *, *, *>.component3(): T = this._3() +operator fun Product9<*, *, *, T, *, *, *, *, *>.component4(): T = this._4() +operator fun Product9<*, *, *, *, T, *, *, *, *>.component5(): T = this._5() +operator fun Product9<*, *, *, *, *, T, *, *, *>.component6(): T = this._6() +operator fun Product9<*, *, *, *, *, *, T, *, *>.component7(): T = this._7() +operator fun Product9<*, *, *, *, *, *, *, T, *>.component8(): T = this._8() +operator fun Product9<*, *, *, *, *, *, *, *, T>.component9(): T = this._9() +operator fun Product10.component1(): T = this._1() +operator fun Product10<*, T, *, *, *, *, *, *, *, *>.component2(): T = this._2() +operator fun Product10<*, *, T, *, *, *, *, *, *, *>.component3(): T = this._3() +operator fun Product10<*, *, *, T, *, *, *, *, *, *>.component4(): T = this._4() +operator fun Product10<*, *, *, *, T, *, *, *, *, *>.component5(): T = this._5() +operator fun Product10<*, *, *, *, *, T, *, *, *, *>.component6(): T = this._6() +operator fun Product10<*, *, *, *, *, *, T, *, *, *>.component7(): T = this._7() +operator fun Product10<*, *, *, *, *, *, *, T, *, *>.component8(): T = this._8() +operator fun Product10<*, *, *, *, *, *, *, *, T, *>.component9(): T = this._9() +operator fun Product10<*, *, *, *, *, *, *, *, *, T>.component10(): T = this._10() +operator fun Product11.component1(): T = this._1() +operator fun Product11<*, T, *, *, *, *, *, *, *, *, *>.component2(): T = this._2() +operator fun Product11<*, *, T, *, *, *, *, *, *, *, *>.component3(): T = this._3() +operator fun Product11<*, *, *, T, *, *, *, *, *, *, *>.component4(): T = this._4() +operator fun Product11<*, *, *, *, T, *, *, *, *, *, *>.component5(): T = this._5() +operator fun Product11<*, *, *, *, *, T, *, *, *, *, *>.component6(): T = this._6() +operator fun Product11<*, *, *, *, *, *, T, *, *, *, *>.component7(): T = this._7() +operator fun Product11<*, *, *, *, *, *, *, T, *, *, *>.component8(): T = this._8() +operator fun Product11<*, *, *, *, *, *, *, *, T, *, *>.component9(): T = this._9() +operator fun Product11<*, *, *, *, *, *, *, *, *, T, *>.component10(): T = this._10() +operator fun Product11<*, *, *, *, *, *, *, *, *, *, T>.component11(): T = this._11() +operator fun Product12.component1(): T = this._1() +operator fun Product12<*, T, *, *, *, *, *, *, *, *, *, *>.component2(): T = this._2() +operator fun Product12<*, *, T, *, *, *, *, *, *, *, *, *>.component3(): T = this._3() +operator fun Product12<*, *, *, T, *, *, *, *, *, *, *, *>.component4(): T = this._4() +operator fun Product12<*, *, *, *, T, *, *, *, *, *, *, *>.component5(): T = this._5() +operator fun Product12<*, *, *, *, *, T, *, *, *, *, *, *>.component6(): T = this._6() +operator fun Product12<*, *, *, *, *, *, T, *, *, *, *, *>.component7(): T = this._7() +operator fun Product12<*, *, *, *, *, *, *, T, *, *, *, *>.component8(): T = this._8() +operator fun Product12<*, *, *, *, *, *, *, *, T, *, *, *>.component9(): T = this._9() +operator fun Product12<*, *, *, *, *, *, *, *, *, T, *, *>.component10(): T = this._10() +operator fun Product12<*, *, *, *, *, *, *, *, *, *, T, *>.component11(): T = this._11() +operator fun Product12<*, *, *, *, *, *, *, *, *, *, *, T>.component12(): T = this._12() +operator fun Product13.component1(): T = this._1() +operator fun Product13<*, T, *, *, *, *, *, *, *, *, *, *, *>.component2(): T = this._2() +operator fun Product13<*, *, T, *, *, *, *, *, *, *, *, *, *>.component3(): T = this._3() +operator fun Product13<*, *, *, T, *, *, *, *, *, *, *, *, *>.component4(): T = this._4() +operator fun Product13<*, *, *, *, T, *, *, *, *, *, *, *, *>.component5(): T = this._5() +operator fun Product13<*, *, *, *, *, T, *, *, *, *, *, *, *>.component6(): T = this._6() +operator fun Product13<*, *, *, *, *, *, T, *, *, *, *, *, *>.component7(): T = this._7() +operator fun Product13<*, *, *, *, *, *, *, T, *, *, *, *, *>.component8(): T = this._8() +operator fun Product13<*, *, *, *, *, *, *, *, T, *, *, *, *>.component9(): T = this._9() +operator fun Product13<*, *, *, *, *, *, *, *, *, T, *, *, *>.component10(): T = this._10() +operator fun Product13<*, *, *, *, *, *, *, *, *, *, T, *, *>.component11(): T = this._11() +operator fun Product13<*, *, *, *, *, *, *, *, *, *, *, T, *>.component12(): T = this._12() +operator fun Product13<*, *, *, *, *, *, *, *, *, *, *, *, T>.component13(): T = this._13() +operator fun Product14.component1(): T = this._1() +operator fun Product14<*, T, *, *, *, *, *, *, *, *, *, *, *, *>.component2(): T = this._2() +operator fun Product14<*, *, T, *, *, *, *, *, *, *, *, *, *, *>.component3(): T = this._3() +operator fun Product14<*, *, *, T, *, *, *, *, *, *, *, *, *, *>.component4(): T = this._4() +operator fun Product14<*, *, *, *, T, *, *, *, *, *, *, *, *, *>.component5(): T = this._5() +operator fun Product14<*, *, *, *, *, T, *, *, *, *, *, *, *, *>.component6(): T = this._6() +operator fun Product14<*, *, *, *, *, *, T, *, *, *, *, *, *, *>.component7(): T = this._7() +operator fun Product14<*, *, *, *, *, *, *, T, *, *, *, *, *, *>.component8(): T = this._8() +operator fun Product14<*, *, *, *, *, *, *, *, T, *, *, *, *, *>.component9(): T = this._9() +operator fun Product14<*, *, *, *, *, *, *, *, *, T, *, *, *, *>.component10(): T = this._10() +operator fun Product14<*, *, *, *, *, *, *, *, *, *, T, *, *, *>.component11(): T = this._11() +operator fun Product14<*, *, *, *, *, *, *, *, *, *, *, T, *, *>.component12(): T = this._12() +operator fun Product14<*, *, *, *, *, *, *, *, *, *, *, *, T, *>.component13(): T = this._13() +operator fun Product14<*, *, *, *, *, *, *, *, *, *, *, *, *, T>.component14(): T = this._14() +operator fun Product15.component1(): T = this._1() +operator fun Product15<*, T, *, *, *, *, *, *, *, *, *, *, *, *, *>.component2(): T = this._2() +operator fun Product15<*, *, T, *, *, *, *, *, *, *, *, *, *, *, *>.component3(): T = this._3() +operator fun Product15<*, *, *, T, *, *, *, *, *, *, *, *, *, *, *>.component4(): T = this._4() +operator fun Product15<*, *, *, *, T, *, *, *, *, *, *, *, *, *, *>.component5(): T = this._5() +operator fun Product15<*, *, *, *, *, T, *, *, *, *, *, *, *, *, *>.component6(): T = this._6() +operator fun Product15<*, *, *, *, *, *, T, *, *, *, *, *, *, *, *>.component7(): T = this._7() +operator fun Product15<*, *, *, *, *, *, *, T, *, *, *, *, *, *, *>.component8(): T = this._8() +operator fun Product15<*, *, *, *, *, *, *, *, T, *, *, *, *, *, *>.component9(): T = this._9() +operator fun Product15<*, *, *, *, *, *, *, *, *, T, *, *, *, *, *>.component10(): T = this._10() +operator fun Product15<*, *, *, *, *, *, *, *, *, *, T, *, *, *, *>.component11(): T = this._11() +operator fun Product15<*, *, *, *, *, *, *, *, *, *, *, T, *, *, *>.component12(): T = this._12() +operator fun Product15<*, *, *, *, *, *, *, *, *, *, *, *, T, *, *>.component13(): T = this._13() +operator fun Product15<*, *, *, *, *, *, *, *, *, *, *, *, *, T, *>.component14(): T = this._14() +operator fun Product15<*, *, *, *, *, *, *, *, *, *, *, *, *, *, T>.component15(): T = this._15() +operator fun Product16.component1(): T = this._1() +operator fun Product16<*, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.component2(): T = this._2() +operator fun Product16<*, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *>.component3(): T = this._3() +operator fun Product16<*, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *>.component4(): T = this._4() +operator fun Product16<*, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *>.component5(): T = this._5() +operator fun Product16<*, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *>.component6(): T = this._6() +operator fun Product16<*, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *>.component7(): T = this._7() +operator fun Product16<*, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *>.component8(): T = this._8() +operator fun Product16<*, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *>.component9(): T = this._9() +operator fun Product16<*, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *>.component10(): T = this._10() +operator fun Product16<*, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *>.component11(): T = this._11() +operator fun Product16<*, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *>.component12(): T = this._12() +operator fun Product16<*, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *>.component13(): T = this._13() +operator fun Product16<*, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *>.component14(): T = this._14() +operator fun Product16<*, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *>.component15(): T = this._15() +operator fun Product16<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T>.component16(): T = this._16() +operator fun Product17.component1(): T = this._1() +operator fun Product17<*, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.component2(): T = this._2() +operator fun Product17<*, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.component3(): T = this._3() +operator fun Product17<*, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *>.component4(): T = this._4() +operator fun Product17<*, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *>.component5(): T = this._5() +operator fun Product17<*, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *>.component6(): T = this._6() +operator fun Product17<*, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *>.component7(): T = this._7() +operator fun Product17<*, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *>.component8(): T = this._8() +operator fun Product17<*, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *>.component9(): T = this._9() +operator fun Product17<*, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *>.component10(): T = this._10() +operator fun Product17<*, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *>.component11(): T = this._11() +operator fun Product17<*, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *>.component12(): T = this._12() +operator fun Product17<*, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *>.component13(): T = this._13() +operator fun Product17<*, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *>.component14(): T = this._14() +operator fun Product17<*, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *>.component15(): T = this._15() +operator fun Product17<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *>.component16(): T = this._16() +operator fun Product17<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T>.component17(): T = this._17() +operator fun Product18.component1(): T = this._1() +operator fun Product18<*, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.component2(): T = this._2() +operator fun Product18<*, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.component3(): T = this._3() +operator fun Product18<*, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.component4(): T = this._4() +operator fun Product18<*, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *>.component5(): T = this._5() +operator fun Product18<*, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *>.component6(): T = this._6() +operator fun Product18<*, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *>.component7(): T = this._7() +operator fun Product18<*, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *>.component8(): T = this._8() +operator fun Product18<*, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *>.component9(): T = this._9() +operator fun Product18<*, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *>.component10(): T = this._10() +operator fun Product18<*, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *>.component11(): T = this._11() +operator fun Product18<*, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *>.component12(): T = this._12() +operator fun Product18<*, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *>.component13(): T = this._13() +operator fun Product18<*, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *>.component14(): T = this._14() +operator fun Product18<*, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *>.component15(): T = this._15() +operator fun Product18<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *>.component16(): T = this._16() +operator fun Product18<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *>.component17(): T = this._17() +operator fun Product18<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T>.component18(): T = this._18() +operator fun Product19.component1(): T = this._1() +operator fun Product19<*, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.component2(): T = this._2() +operator fun Product19<*, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.component3(): T = this._3() +operator fun Product19<*, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.component4(): T = this._4() +operator fun Product19<*, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.component5(): T = this._5() +operator fun Product19<*, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *>.component6(): T = this._6() +operator fun Product19<*, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *>.component7(): T = this._7() +operator fun Product19<*, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *>.component8(): T = this._8() +operator fun Product19<*, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *>.component9(): T = this._9() +operator fun Product19<*, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *>.component10(): T = this._10() +operator fun Product19<*, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *>.component11(): T = this._11() +operator fun Product19<*, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *>.component12(): T = this._12() +operator fun Product19<*, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *>.component13(): T = this._13() +operator fun Product19<*, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *>.component14(): T = this._14() +operator fun Product19<*, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *>.component15(): T = this._15() +operator fun Product19<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *>.component16(): T = this._16() +operator fun Product19<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *>.component17(): T = this._17() +operator fun Product19<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *>.component18(): T = this._18() +operator fun Product19<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T>.component19(): T = this._19() +operator fun Product20.component1(): T = this._1() +operator fun Product20<*, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.component2(): T = this._2() +operator fun Product20<*, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.component3(): T = this._3() +operator fun Product20<*, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.component4(): T = this._4() +operator fun Product20<*, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.component5(): T = this._5() +operator fun Product20<*, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.component6(): T = this._6() +operator fun Product20<*, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *>.component7(): T = this._7() +operator fun Product20<*, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *>.component8(): T = this._8() +operator fun Product20<*, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *>.component9(): T = this._9() +operator fun Product20<*, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *>.component10(): T = this._10() +operator fun Product20<*, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *>.component11(): T = this._11() +operator fun Product20<*, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *>.component12(): T = this._12() +operator fun Product20<*, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *>.component13(): T = this._13() +operator fun Product20<*, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *>.component14(): T = this._14() +operator fun Product20<*, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *>.component15(): T = this._15() +operator fun Product20<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *>.component16(): T = this._16() +operator fun Product20<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *>.component17(): T = this._17() +operator fun Product20<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *>.component18(): T = this._18() +operator fun Product20<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *>.component19(): T = this._19() +operator fun Product20<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T>.component20(): T = this._20() +operator fun Product21.component1(): T = this._1() +operator fun Product21<*, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.component2(): T = this._2() +operator fun Product21<*, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.component3(): T = this._3() +operator fun Product21<*, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.component4(): T = this._4() +operator fun Product21<*, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.component5(): T = this._5() +operator fun Product21<*, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.component6(): T = this._6() +operator fun Product21<*, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.component7(): T = this._7() +operator fun Product21<*, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *>.component8(): T = this._8() +operator fun Product21<*, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *>.component9(): T = this._9() +operator fun Product21<*, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *>.component10(): T = this._10() +operator fun Product21<*, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *>.component11(): T = this._11() +operator fun Product21<*, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *>.component12(): T = this._12() +operator fun Product21<*, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *>.component13(): T = this._13() +operator fun Product21<*, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *>.component14(): T = this._14() +operator fun Product21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *>.component15(): T = this._15() +operator fun Product21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *>.component16(): T = this._16() +operator fun Product21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *>.component17(): T = this._17() +operator fun Product21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *>.component18(): T = this._18() +operator fun Product21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *>.component19(): T = this._19() +operator fun Product21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *>.component20(): T = this._20() +operator fun Product21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T>.component21(): T = this._21() +operator fun Product22.component1(): T = this._1() +operator fun Product22<*, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.component2(): T = this._2() +operator fun Product22<*, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.component3(): T = this._3() +operator fun Product22<*, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.component4(): T = this._4() +operator fun Product22<*, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.component5(): T = this._5() +operator fun Product22<*, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.component6(): T = this._6() +operator fun Product22<*, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.component7(): T = this._7() +operator fun Product22<*, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.component8(): T = this._8() +operator fun Product22<*, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *>.component9(): T = this._9() +operator fun Product22<*, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *>.component10(): T = this._10() +operator fun Product22<*, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *>.component11(): T = this._11() +operator fun Product22<*, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *>.component12(): T = this._12() +operator fun Product22<*, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *>.component13(): T = this._13() +operator fun Product22<*, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *>.component14(): T = this._14() +operator fun Product22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *>.component15(): T = this._15() +operator fun Product22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *>.component16(): T = this._16() +operator fun Product22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *>.component17(): T = this._17() +operator fun Product22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *>.component18(): T = this._18() +operator fun Product22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *>.component19(): T = this._19() +operator fun Product22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *>.component20(): T = this._20() +operator fun Product22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *>.component21(): T = this._21() +operator fun Product22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T>.component22(): T = this._22() diff --git a/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/ProductExtensions.kt b/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/ProductExtensions.kt new file mode 100644 index 00000000..cca9d553 --- /dev/null +++ b/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/ProductExtensions.kt @@ -0,0 +1,157 @@ +/*- + * =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.tuples + +import scala.Product +import scala.collection.JavaConverters +import kotlin.jvm.Throws + +/** + * Extra extensions for Scala [Product]s such as Tuples. + * In most cases, the functions of `SameTypeProductExtensions.kt` will be used + * instead of these. But these help for the overview and generic case. + * + * For example: + * + * ```kotlin + * 1 in tupleOf(1, 2, 3) == true + * + * for (x in tupleOf("a", "b", "c")) { ... } + * + * val a: List = tupleOf(1, "a", 3L).asIterable().toList() + * + * tupleOf(1, 2, 3).size == 3 + * + * tupleOf(1, 2, 3)[0] == 1 + * + * tupleOf(1, 1, 2)[1..2] == tupleOf(1, 2, 2)[0..1] + * ``` + * + */ + +/** Tests whether this iterator contains a given value as an element. + * Note: may not terminate for infinite iterators. + * + * @param item the element to test. + * @return `true` if this iterator produces some value that + * is equal (as determined by `==`) to `elem`, `false` otherwise. + * @note Reuse: After calling this method, one should discard the iterator it was called on. + * Using it is undefined and subject to change. + */ +operator fun Product.contains(item: Any?): Boolean = productIterator().contains(item) + +/** + * An iterator over all the elements of this product. + * @return in the default implementation, an `Iterator` + */ +operator fun Product.iterator(): Iterator = JavaConverters.asJavaIterator(productIterator()) + +/** + * Converts this product to an `Any?` iterable. + */ +fun Product.asIterable(): Iterable = object : Iterable { + override fun iterator(): Iterator = JavaConverters.asJavaIterator(productIterator()) +} + +/** The size of this product. + * @return for a product `A(x,,1,,, ..., x,,k,,)`, returns `k` + */ +val Product.size: Int + get() = productArity() + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @throws IndexOutOfBoundsException + * @return the element `n` elements after the first element + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product.get(n: Int): Any? = productElement(n) + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @return the element `n` elements after the first element, `null` if out of bounds + */ +fun Product.getOrNull(n: Int): Any? = if (n in 0 until size) productElement(n) else null + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * The result is cast to the given type [T]. + * + * @param n the index of the element to return + * @throws IndexOutOfBoundsException + * @throws ClassCastException + * @return the element `n` elements after the first element + */ +@Suppress("UNCHECKED_CAST") +@Throws(IndexOutOfBoundsException::class, ClassCastException::class) +inline fun Product.getAs(n: Int): T = productElement(n) as T + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * The result is cast to the given type [T]. + * + * @param n the index of the element to return + * @return the element `n` elements after the first element, `null` if out of bounds or unable to be cast + */ +@Suppress("UNCHECKED_CAST") +inline fun Product.getAsOrNull(n: Int): T? = getOrNull(n) as? T + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @throws IndexOutOfBoundsException + * @return the elements in [indexRange] + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product.get(indexRange: IntRange): List = indexRange.map(::get) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @return the elements in [indexRange], `null` if out of bounds + */ +fun Product.getOrNull(indexRange: IntRange): List = indexRange.map(::getOrNull) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * The results are cast to the given type [T]. + * + * @param indexRange the indices of the elements to return + * @throws IndexOutOfBoundsException + * @throws ClassCastException + * @return the elements in [indexRange] + */ +@Throws(IndexOutOfBoundsException::class, ClassCastException::class) +inline fun Product.getAs(indexRange: IntRange): List = indexRange.map(::getAs) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * The results are cast to the given type [T]. + * + * @param indexRange the indices of the elements to return + * @return the elements in [indexRange], `null` is out of bounds or unable to be cast + */ +inline fun Product.getAsOrNull(indexRange: IntRange): List = indexRange.map(::getAsOrNull) diff --git a/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/ProductTextualAccessors.kt b/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/ProductTextualAccessors.kt new file mode 100644 index 00000000..4e9c676b --- /dev/null +++ b/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/ProductTextualAccessors.kt @@ -0,0 +1,690 @@ +/*- + * =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("ObjectPropertyName") + +package org.jetbrains.kotlinx.spark.api.tuples + +import scala.Product1 +import scala.Product2 +import scala.Product3 +import scala.Product4 +import scala.Product5 +import scala.Product6 +import scala.Product7 +import scala.Product8 +import scala.Product9 +import scala.Product10 +import scala.Product11 +import scala.Product12 +import scala.Product13 +import scala.Product14 +import scala.Product15 +import scala.Product16 +import scala.Product17 +import scala.Product18 +import scala.Product19 +import scala.Product20 +import scala.Product21 +import scala.Product22 + +/** + * + * This file provides the functions `yourTuple.first()` and `yourTuple.last()` to access + * the value you require. + * + */ + +val Product1._1: T get() = this._1() + +/** Returns the first value of this Tuple or Product. */ +fun Product1.first(): T = this._1() + +/** Returns the last value of this Tuple or Product. */ +fun Product1.last(): T = this._1() + +val Product2._1: T get() = this._1() + +val Product2<*, T>._2: T get() = this._2() + +/** Returns the first value of this Tuple or Product. */ +fun Product2.first(): T = this._1() + +/** Returns the last value of this Tuple or Product. */ +fun Product2<*, T>.last(): T = this._2() + +val Product3._1: T get() = this._1() + +val Product3<*, T, *>._2: T get() = this._2() + +val Product3<*, *, T>._3: T get() = this._3() + +/** Returns the first value of this Tuple or Product. */ +fun Product3.first(): T = this._1() + +/** Returns the last value of this Tuple or Product. */ +fun Product3<*, *, T>.last(): T = this._3() + +val Product4._1: T get() = this._1() + +val Product4<*, T, *, *>._2: T get() = this._2() + +val Product4<*, *, T, *>._3: T get() = this._3() + +val Product4<*, *, *, T>._4: T get() = this._4() + +/** Returns the first value of this Tuple or Product. */ +fun Product4.first(): T = this._1() + +/** Returns the last value of this Tuple or Product. */ +fun Product4<*, *, *, T>.last(): T = this._4() + +val Product5._1: T get() = this._1() + +val Product5<*, T, *, *, *>._2: T get() = this._2() + +val Product5<*, *, T, *, *>._3: T get() = this._3() + +val Product5<*, *, *, T, *>._4: T get() = this._4() + +val Product5<*, *, *, *, T>._5: T get() = this._5() + +/** Returns the first value of this Tuple or Product. */ +fun Product5.first(): T = this._1() + +/** Returns the last value of this Tuple or Product. */ +fun Product5<*, *, *, *, T>.last(): T = this._5() + +val Product6._1: T get() = this._1() + +val Product6<*, T, *, *, *, *>._2: T get() = this._2() + +val Product6<*, *, T, *, *, *>._3: T get() = this._3() + +val Product6<*, *, *, T, *, *>._4: T get() = this._4() + +val Product6<*, *, *, *, T, *>._5: T get() = this._5() + +val Product6<*, *, *, *, *, T>._6: T get() = this._6() + +/** Returns the first value of this Tuple or Product. */ +fun Product6.first(): T = this._1() + +/** Returns the last value of this Tuple or Product. */ +fun Product6<*, *, *, *, *, T>.last(): T = this._6() + +val Product7._1: T get() = this._1() + +val Product7<*, T, *, *, *, *, *>._2: T get() = this._2() + +val Product7<*, *, T, *, *, *, *>._3: T get() = this._3() + +val Product7<*, *, *, T, *, *, *>._4: T get() = this._4() + +val Product7<*, *, *, *, T, *, *>._5: T get() = this._5() + +val Product7<*, *, *, *, *, T, *>._6: T get() = this._6() + +val Product7<*, *, *, *, *, *, T>._7: T get() = this._7() + +/** Returns the first value of this Tuple or Product. */ +fun Product7.first(): T = this._1() + +/** Returns the last value of this Tuple or Product. */ +fun Product7<*, *, *, *, *, *, T>.last(): T = this._7() + +val Product8._1: T get() = this._1() + +val Product8<*, T, *, *, *, *, *, *>._2: T get() = this._2() + +val Product8<*, *, T, *, *, *, *, *>._3: T get() = this._3() + +val Product8<*, *, *, T, *, *, *, *>._4: T get() = this._4() + +val Product8<*, *, *, *, T, *, *, *>._5: T get() = this._5() + +val Product8<*, *, *, *, *, T, *, *>._6: T get() = this._6() + +val Product8<*, *, *, *, *, *, T, *>._7: T get() = this._7() + +val Product8<*, *, *, *, *, *, *, T>._8: T get() = this._8() + +/** Returns the first value of this Tuple or Product. */ +fun Product8.first(): T = this._1() + +/** Returns the last value of this Tuple or Product. */ +fun Product8<*, *, *, *, *, *, *, T>.last(): T = this._8() + +val Product9._1: T get() = this._1() + +val Product9<*, T, *, *, *, *, *, *, *>._2: T get() = this._2() + +val Product9<*, *, T, *, *, *, *, *, *>._3: T get() = this._3() + +val Product9<*, *, *, T, *, *, *, *, *>._4: T get() = this._4() + +val Product9<*, *, *, *, T, *, *, *, *>._5: T get() = this._5() + +val Product9<*, *, *, *, *, T, *, *, *>._6: T get() = this._6() + +val Product9<*, *, *, *, *, *, T, *, *>._7: T get() = this._7() + +val Product9<*, *, *, *, *, *, *, T, *>._8: T get() = this._8() + +val Product9<*, *, *, *, *, *, *, *, T>._9: T get() = this._9() + +/** Returns the first value of this Tuple or Product. */ +fun Product9.first(): T = this._1() + +/** Returns the last value of this Tuple or Product. */ +fun Product9<*, *, *, *, *, *, *, *, T>.last(): T = this._9() + +val Product10._1: T get() = this._1() + +val Product10<*, T, *, *, *, *, *, *, *, *>._2: T get() = this._2() + +val Product10<*, *, T, *, *, *, *, *, *, *>._3: T get() = this._3() + +val Product10<*, *, *, T, *, *, *, *, *, *>._4: T get() = this._4() + +val Product10<*, *, *, *, T, *, *, *, *, *>._5: T get() = this._5() + +val Product10<*, *, *, *, *, T, *, *, *, *>._6: T get() = this._6() + +val Product10<*, *, *, *, *, *, T, *, *, *>._7: T get() = this._7() + +val Product10<*, *, *, *, *, *, *, T, *, *>._8: T get() = this._8() + +val Product10<*, *, *, *, *, *, *, *, T, *>._9: T get() = this._9() + +val Product10<*, *, *, *, *, *, *, *, *, T>._10: T get() = this._10() + +/** Returns the first value of this Tuple or Product. */ +fun Product10.first(): T = this._1() + +/** Returns the last value of this Tuple or Product. */ +fun Product10<*, *, *, *, *, *, *, *, *, T>.last(): T = this._10() + +val Product11._1: T get() = this._1() + +val Product11<*, T, *, *, *, *, *, *, *, *, *>._2: T get() = this._2() + +val Product11<*, *, T, *, *, *, *, *, *, *, *>._3: T get() = this._3() + +val Product11<*, *, *, T, *, *, *, *, *, *, *>._4: T get() = this._4() + +val Product11<*, *, *, *, T, *, *, *, *, *, *>._5: T get() = this._5() + +val Product11<*, *, *, *, *, T, *, *, *, *, *>._6: T get() = this._6() + +val Product11<*, *, *, *, *, *, T, *, *, *, *>._7: T get() = this._7() + +val Product11<*, *, *, *, *, *, *, T, *, *, *>._8: T get() = this._8() + +val Product11<*, *, *, *, *, *, *, *, T, *, *>._9: T get() = this._9() + +val Product11<*, *, *, *, *, *, *, *, *, T, *>._10: T get() = this._10() + +val Product11<*, *, *, *, *, *, *, *, *, *, T>._11: T get() = this._11() + +/** Returns the first value of this Tuple or Product. */ +fun Product11.first(): T = this._1() + +/** Returns the last value of this Tuple or Product. */ +fun Product11<*, *, *, *, *, *, *, *, *, *, T>.last(): T = this._11() + +val Product12._1: T get() = this._1() + +val Product12<*, T, *, *, *, *, *, *, *, *, *, *>._2: T get() = this._2() + +val Product12<*, *, T, *, *, *, *, *, *, *, *, *>._3: T get() = this._3() + +val Product12<*, *, *, T, *, *, *, *, *, *, *, *>._4: T get() = this._4() + +val Product12<*, *, *, *, T, *, *, *, *, *, *, *>._5: T get() = this._5() + +val Product12<*, *, *, *, *, T, *, *, *, *, *, *>._6: T get() = this._6() + +val Product12<*, *, *, *, *, *, T, *, *, *, *, *>._7: T get() = this._7() + +val Product12<*, *, *, *, *, *, *, T, *, *, *, *>._8: T get() = this._8() + +val Product12<*, *, *, *, *, *, *, *, T, *, *, *>._9: T get() = this._9() + +val Product12<*, *, *, *, *, *, *, *, *, T, *, *>._10: T get() = this._10() + +val Product12<*, *, *, *, *, *, *, *, *, *, T, *>._11: T get() = this._11() + +val Product12<*, *, *, *, *, *, *, *, *, *, *, T>._12: T get() = this._12() + +/** Returns the first value of this Tuple or Product. */ +fun Product12.first(): T = this._1() + +/** Returns the last value of this Tuple or Product. */ +fun Product12<*, *, *, *, *, *, *, *, *, *, *, T>.last(): T = this._12() + +val Product13._1: T get() = this._1() + +val Product13<*, T, *, *, *, *, *, *, *, *, *, *, *>._2: T get() = this._2() + +val Product13<*, *, T, *, *, *, *, *, *, *, *, *, *>._3: T get() = this._3() + +val Product13<*, *, *, T, *, *, *, *, *, *, *, *, *>._4: T get() = this._4() + +val Product13<*, *, *, *, T, *, *, *, *, *, *, *, *>._5: T get() = this._5() + +val Product13<*, *, *, *, *, T, *, *, *, *, *, *, *>._6: T get() = this._6() + +val Product13<*, *, *, *, *, *, T, *, *, *, *, *, *>._7: T get() = this._7() + +val Product13<*, *, *, *, *, *, *, T, *, *, *, *, *>._8: T get() = this._8() + +val Product13<*, *, *, *, *, *, *, *, T, *, *, *, *>._9: T get() = this._9() + +val Product13<*, *, *, *, *, *, *, *, *, T, *, *, *>._10: T get() = this._10() + +val Product13<*, *, *, *, *, *, *, *, *, *, T, *, *>._11: T get() = this._11() + +val Product13<*, *, *, *, *, *, *, *, *, *, *, T, *>._12: T get() = this._12() + +val Product13<*, *, *, *, *, *, *, *, *, *, *, *, T>._13: T get() = this._13() + +/** Returns the first value of this Tuple or Product. */ +fun Product13.first(): T = this._1() + +/** Returns the last value of this Tuple or Product. */ +fun Product13<*, *, *, *, *, *, *, *, *, *, *, *, T>.last(): T = this._13() + +val Product14._1: T get() = this._1() + +val Product14<*, T, *, *, *, *, *, *, *, *, *, *, *, *>._2: T get() = this._2() + +val Product14<*, *, T, *, *, *, *, *, *, *, *, *, *, *>._3: T get() = this._3() + +val Product14<*, *, *, T, *, *, *, *, *, *, *, *, *, *>._4: T get() = this._4() + +val Product14<*, *, *, *, T, *, *, *, *, *, *, *, *, *>._5: T get() = this._5() + +val Product14<*, *, *, *, *, T, *, *, *, *, *, *, *, *>._6: T get() = this._6() + +val Product14<*, *, *, *, *, *, T, *, *, *, *, *, *, *>._7: T get() = this._7() + +val Product14<*, *, *, *, *, *, *, T, *, *, *, *, *, *>._8: T get() = this._8() + +val Product14<*, *, *, *, *, *, *, *, T, *, *, *, *, *>._9: T get() = this._9() + +val Product14<*, *, *, *, *, *, *, *, *, T, *, *, *, *>._10: T get() = this._10() + +val Product14<*, *, *, *, *, *, *, *, *, *, T, *, *, *>._11: T get() = this._11() + +val Product14<*, *, *, *, *, *, *, *, *, *, *, T, *, *>._12: T get() = this._12() + +val Product14<*, *, *, *, *, *, *, *, *, *, *, *, T, *>._13: T get() = this._13() + +val Product14<*, *, *, *, *, *, *, *, *, *, *, *, *, T>._14: T get() = this._14() + +/** Returns the first value of this Tuple or Product. */ +fun Product14.first(): T = this._1() + +/** Returns the last value of this Tuple or Product. */ +fun Product14<*, *, *, *, *, *, *, *, *, *, *, *, *, T>.last(): T = this._14() + +val Product15._1: T get() = this._1() + +val Product15<*, T, *, *, *, *, *, *, *, *, *, *, *, *, *>._2: T get() = this._2() + +val Product15<*, *, T, *, *, *, *, *, *, *, *, *, *, *, *>._3: T get() = this._3() + +val Product15<*, *, *, T, *, *, *, *, *, *, *, *, *, *, *>._4: T get() = this._4() + +val Product15<*, *, *, *, T, *, *, *, *, *, *, *, *, *, *>._5: T get() = this._5() + +val Product15<*, *, *, *, *, T, *, *, *, *, *, *, *, *, *>._6: T get() = this._6() + +val Product15<*, *, *, *, *, *, T, *, *, *, *, *, *, *, *>._7: T get() = this._7() + +val Product15<*, *, *, *, *, *, *, T, *, *, *, *, *, *, *>._8: T get() = this._8() + +val Product15<*, *, *, *, *, *, *, *, T, *, *, *, *, *, *>._9: T get() = this._9() + +val Product15<*, *, *, *, *, *, *, *, *, T, *, *, *, *, *>._10: T get() = this._10() + +val Product15<*, *, *, *, *, *, *, *, *, *, T, *, *, *, *>._11: T get() = this._11() + +val Product15<*, *, *, *, *, *, *, *, *, *, *, T, *, *, *>._12: T get() = this._12() + +val Product15<*, *, *, *, *, *, *, *, *, *, *, *, T, *, *>._13: T get() = this._13() + +val Product15<*, *, *, *, *, *, *, *, *, *, *, *, *, T, *>._14: T get() = this._14() + +val Product15<*, *, *, *, *, *, *, *, *, *, *, *, *, *, T>._15: T get() = this._15() + +/** Returns the first value of this Tuple or Product. */ +fun Product15.first(): T = this._1() + +/** Returns the last value of this Tuple or Product. */ +fun Product15<*, *, *, *, *, *, *, *, *, *, *, *, *, *, T>.last(): T = this._15() + +val Product16._1: T get() = this._1() + +val Product16<*, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *>._2: T get() = this._2() + +val Product16<*, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *>._3: T get() = this._3() + +val Product16<*, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *>._4: T get() = this._4() + +val Product16<*, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *>._5: T get() = this._5() + +val Product16<*, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *>._6: T get() = this._6() + +val Product16<*, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *>._7: T get() = this._7() + +val Product16<*, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *>._8: T get() = this._8() + +val Product16<*, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *>._9: T get() = this._9() + +val Product16<*, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *>._10: T get() = this._10() + +val Product16<*, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *>._11: T get() = this._11() + +val Product16<*, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *>._12: T get() = this._12() + +val Product16<*, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *>._13: T get() = this._13() + +val Product16<*, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *>._14: T get() = this._14() + +val Product16<*, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *>._15: T get() = this._15() + +val Product16<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T>._16: T get() = this._16() + +/** Returns the first value of this Tuple or Product. */ +fun Product16.first(): T = this._1() + +/** Returns the last value of this Tuple or Product. */ +fun Product16<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T>.last(): T = this._16() + +val Product17._1: T get() = this._1() + +val Product17<*, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>._2: T get() = this._2() + +val Product17<*, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *>._3: T get() = this._3() + +val Product17<*, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *>._4: T get() = this._4() + +val Product17<*, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *>._5: T get() = this._5() + +val Product17<*, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *>._6: T get() = this._6() + +val Product17<*, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *>._7: T get() = this._7() + +val Product17<*, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *>._8: T get() = this._8() + +val Product17<*, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *>._9: T get() = this._9() + +val Product17<*, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *>._10: T get() = this._10() + +val Product17<*, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *>._11: T get() = this._11() + +val Product17<*, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *>._12: T get() = this._12() + +val Product17<*, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *>._13: T get() = this._13() + +val Product17<*, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *>._14: T get() = this._14() + +val Product17<*, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *>._15: T get() = this._15() + +val Product17<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *>._16: T get() = this._16() + +val Product17<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T>._17: T get() = this._17() + +/** Returns the first value of this Tuple or Product. */ +fun Product17.first(): T = this._1() + +/** Returns the last value of this Tuple or Product. */ +fun Product17<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T>.last(): T = this._17() + +val Product18._1: T get() = this._1() + +val Product18<*, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>._2: T get() = this._2() + +val Product18<*, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>._3: T get() = this._3() + +val Product18<*, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *>._4: T get() = this._4() + +val Product18<*, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *>._5: T get() = this._5() + +val Product18<*, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *>._6: T get() = this._6() + +val Product18<*, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *>._7: T get() = this._7() + +val Product18<*, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *>._8: T get() = this._8() + +val Product18<*, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *>._9: T get() = this._9() + +val Product18<*, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *>._10: T get() = this._10() + +val Product18<*, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *>._11: T get() = this._11() + +val Product18<*, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *>._12: T get() = this._12() + +val Product18<*, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *>._13: T get() = this._13() + +val Product18<*, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *>._14: T get() = this._14() + +val Product18<*, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *>._15: T get() = this._15() + +val Product18<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *>._16: T get() = this._16() + +val Product18<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *>._17: T get() = this._17() + +val Product18<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T>._18: T get() = this._18() + +/** Returns the first value of this Tuple or Product. */ +fun Product18.first(): T = this._1() + +/** Returns the last value of this Tuple or Product. */ +fun Product18<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T>.last(): T = this._18() + +val Product19._1: T get() = this._1() + +val Product19<*, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>._2: T get() = this._2() + +val Product19<*, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>._3: T get() = this._3() + +val Product19<*, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>._4: T get() = this._4() + +val Product19<*, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *>._5: T get() = this._5() + +val Product19<*, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *>._6: T get() = this._6() + +val Product19<*, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *>._7: T get() = this._7() + +val Product19<*, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *>._8: T get() = this._8() + +val Product19<*, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *>._9: T get() = this._9() + +val Product19<*, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *>._10: T get() = this._10() + +val Product19<*, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *>._11: T get() = this._11() + +val Product19<*, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *>._12: T get() = this._12() + +val Product19<*, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *>._13: T get() = this._13() + +val Product19<*, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *>._14: T get() = this._14() + +val Product19<*, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *>._15: T get() = this._15() + +val Product19<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *>._16: T get() = this._16() + +val Product19<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *>._17: T get() = this._17() + +val Product19<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *>._18: T get() = this._18() + +val Product19<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T>._19: T get() = this._19() + +/** Returns the first value of this Tuple or Product. */ +fun Product19.first(): T = this._1() + +/** Returns the last value of this Tuple or Product. */ +fun Product19<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T>.last(): T = this._19() + +val Product20._1: T get() = this._1() + +val Product20<*, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>._2: T get() = this._2() + +val Product20<*, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>._3: T get() = this._3() + +val Product20<*, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>._4: T get() = this._4() + +val Product20<*, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>._5: T get() = this._5() + +val Product20<*, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *>._6: T get() = this._6() + +val Product20<*, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *>._7: T get() = this._7() + +val Product20<*, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *>._8: T get() = this._8() + +val Product20<*, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *>._9: T get() = this._9() + +val Product20<*, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *>._10: T get() = this._10() + +val Product20<*, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *>._11: T get() = this._11() + +val Product20<*, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *>._12: T get() = this._12() + +val Product20<*, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *>._13: T get() = this._13() + +val Product20<*, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *>._14: T get() = this._14() + +val Product20<*, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *>._15: T get() = this._15() + +val Product20<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *>._16: T get() = this._16() + +val Product20<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *>._17: T get() = this._17() + +val Product20<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *>._18: T get() = this._18() + +val Product20<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *>._19: T get() = this._19() + +val Product20<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T>._20: T get() = this._20() + +/** Returns the first value of this Tuple or Product. */ +fun Product20.first(): T = this._1() + +/** Returns the last value of this Tuple or Product. */ +fun Product20<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T>.last(): T = this._20() + +val Product21._1: T get() = this._1() + +val Product21<*, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>._2: T get() = this._2() + +val Product21<*, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>._3: T get() = this._3() + +val Product21<*, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>._4: T get() = this._4() + +val Product21<*, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>._5: T get() = this._5() + +val Product21<*, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>._6: T get() = this._6() + +val Product21<*, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *>._7: T get() = this._7() + +val Product21<*, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *>._8: T get() = this._8() + +val Product21<*, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *>._9: T get() = this._9() + +val Product21<*, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *>._10: T get() = this._10() + +val Product21<*, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *>._11: T get() = this._11() + +val Product21<*, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *>._12: T get() = this._12() + +val Product21<*, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *>._13: T get() = this._13() + +val Product21<*, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *>._14: T get() = this._14() + +val Product21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *>._15: T get() = this._15() + +val Product21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *>._16: T get() = this._16() + +val Product21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *>._17: T get() = this._17() + +val Product21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *>._18: T get() = this._18() + +val Product21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *>._19: T get() = this._19() + +val Product21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *>._20: T get() = this._20() + +val Product21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T>._21: T get() = this._21() + +/** Returns the first value of this Tuple or Product. */ +fun Product21.first(): T = this._1() + +/** Returns the last value of this Tuple or Product. */ +fun Product21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T>.last(): T = this._21() + +val Product22._1: T get() = this._1() + +val Product22<*, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>._2: T get() = this._2() + +val Product22<*, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>._3: T get() = this._3() + +val Product22<*, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>._4: T get() = this._4() + +val Product22<*, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>._5: T get() = this._5() + +val Product22<*, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>._6: T get() = this._6() + +val Product22<*, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>._7: T get() = this._7() + +val Product22<*, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *, *>._8: T get() = this._8() + +val Product22<*, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *, *>._9: T get() = this._9() + +val Product22<*, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *, *>._10: T get() = this._10() + +val Product22<*, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *, *>._11: T get() = this._11() + +val Product22<*, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *, *>._12: T get() = this._12() + +val Product22<*, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *, *>._13: T get() = this._13() + +val Product22<*, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *, *>._14: T get() = this._14() + +val Product22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *, *>._15: T get() = this._15() + +val Product22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *, *>._16: T get() = this._16() + +val Product22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *, *>._17: T get() = this._17() + +val Product22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *, *>._18: T get() = this._18() + +val Product22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *, *>._19: T get() = this._19() + +val Product22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *, *>._20: T get() = this._20() + +val Product22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T, *>._21: T get() = this._21() + +val Product22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T>._22: T get() = this._22() + +/** Returns the first value of this Tuple or Product. */ +fun Product22.first(): T = this._1() + +/** Returns the last value of this Tuple or Product. */ +fun Product22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T>.last(): T = this._22() diff --git a/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/TupleBuilders.kt b/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/TupleBuilders.kt new file mode 100644 index 00000000..365be37a --- /dev/null +++ b/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/TupleBuilders.kt @@ -0,0 +1,378 @@ +/*- + * =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("FunctionName", "RemoveExplicitTypeArguments", "DuplicatedCode") +package org.jetbrains.kotlinx.spark.api.tuples + +import scala.Tuple1 +import scala.Tuple2 +import scala.Tuple3 +import scala.Tuple4 +import scala.Tuple5 +import scala.Tuple6 +import scala.Tuple7 +import scala.Tuple8 +import scala.Tuple9 +import scala.Tuple10 +import scala.Tuple11 +import scala.Tuple12 +import scala.Tuple13 +import scala.Tuple14 +import scala.Tuple15 +import scala.Tuple16 +import scala.Tuple17 +import scala.Tuple18 +import scala.Tuple19 +import scala.Tuple20 +import scala.Tuple21 +import scala.Tuple22 + +/** + * This file contains simple functional Tuple builders in the form of `tupleOf()`. + * + * This allows you to easily create the correct type of tuple with correct types like + * ```val yourTuple = tupleOf(1, "test", a)``` + * or + * ```val yourTuple = t(1, "test", a)``` + * + * As replacement of `to` there is + * ```val tuple: Tuple2 = 5 X "test"``` + */ + +/** + * Returns the instance of Tuple0. + * @see t + */ +fun tupleOf(): EmptyTuple = EmptyTuple + + +/** + * Returns a new Tuple1 of the given arguments. + * @see t + * @see X + */ +fun tupleOf(_1: T1): Tuple1 = Tuple1(_1) + +/** + * Returns a new Tuple2 of the given arguments. + * @see t + * @see X + */ +fun tupleOf(_1: T1, _2: T2): Tuple2 = Tuple2(_1, _2) + +/** + * Returns a new Tuple3 of the given arguments. + * @see t + * @see X + */ +fun tupleOf(_1: T1, _2: T2, _3: T3): Tuple3 = Tuple3(_1, _2, _3) + +/** + * Returns a new Tuple4 of the given arguments. + * @see t + * @see X + */ +fun tupleOf(_1: T1, _2: T2, _3: T3, _4: T4): Tuple4 = Tuple4(_1, _2, _3, _4) + +/** + * Returns a new Tuple5 of the given arguments. + * @see t + * @see X + */ +fun tupleOf(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5): Tuple5 = Tuple5(_1, _2, _3, _4, _5) + +/** + * Returns a new Tuple6 of the given arguments. + * @see t + * @see X + */ +fun tupleOf(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6): Tuple6 = Tuple6(_1, _2, _3, _4, _5, _6) + +/** + * Returns a new Tuple7 of the given arguments. + * @see t + * @see X + */ +fun tupleOf(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7): Tuple7 = Tuple7(_1, _2, _3, _4, _5, _6, _7) + +/** + * Returns a new Tuple8 of the given arguments. + * @see t + * @see X + */ +fun tupleOf(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8): Tuple8 = Tuple8(_1, _2, _3, _4, _5, _6, _7, _8) + +/** + * Returns a new Tuple9 of the given arguments. + * @see t + * @see X + */ +fun tupleOf(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9): Tuple9 = Tuple9(_1, _2, _3, _4, _5, _6, _7, _8, _9) + +/** + * Returns a new Tuple10 of the given arguments. + * @see t + * @see X + */ +fun tupleOf(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10): Tuple10 = Tuple10(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10) + +/** + * Returns a new Tuple11 of the given arguments. + * @see t + * @see X + */ +fun tupleOf(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11): Tuple11 = Tuple11(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11) + +/** + * Returns a new Tuple12 of the given arguments. + * @see t + * @see X + */ +fun tupleOf(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12): Tuple12 = Tuple12(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12) + +/** + * Returns a new Tuple13 of the given arguments. + * @see t + * @see X + */ +fun tupleOf(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13): Tuple13 = Tuple13(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13) + +/** + * Returns a new Tuple14 of the given arguments. + * @see t + * @see X + */ +fun tupleOf(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14): Tuple14 = Tuple14(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14) + +/** + * Returns a new Tuple15 of the given arguments. + * @see t + * @see X + */ +fun tupleOf(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15): Tuple15 = Tuple15(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15) + +/** + * Returns a new Tuple16 of the given arguments. + * @see t + * @see X + */ +fun tupleOf(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16): Tuple16 = Tuple16(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16) + +/** + * Returns a new Tuple17 of the given arguments. + * @see t + * @see X + */ +fun tupleOf(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16, _17: T17): Tuple17 = Tuple17(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17) + +/** + * Returns a new Tuple18 of the given arguments. + * @see t + * @see X + */ +fun tupleOf(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16, _17: T17, _18: T18): Tuple18 = Tuple18(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18) + +/** + * Returns a new Tuple19 of the given arguments. + * @see t + * @see X + */ +fun tupleOf(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16, _17: T17, _18: T18, _19: T19): Tuple19 = Tuple19(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19) + +/** + * Returns a new Tuple20 of the given arguments. + * @see t + * @see X + */ +fun tupleOf(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16, _17: T17, _18: T18, _19: T19, _20: T20): Tuple20 = Tuple20(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19, _20) + +/** + * Returns a new Tuple21 of the given arguments. + * @see t + * @see X + */ +fun tupleOf(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16, _17: T17, _18: T18, _19: T19, _20: T20, _21: T21): Tuple21 = Tuple21(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19, _20, _21) + +/** + * Returns a new Tuple22 of the given arguments. + * @see t + * @see X + */ +fun tupleOf(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16, _17: T17, _18: T18, _19: T19, _20: T20, _21: T21, _22: T22): Tuple22 = Tuple22(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19, _20, _21, _22) + +/** + * Returns the instance of Tuple0. + * @see tupleOf + */ +fun t(): EmptyTuple = EmptyTuple + + +/** + * Returns a new Tuple1 of the given arguments. + * @see tupleOf + * @see X + */ +fun t(_1: T1): Tuple1 = Tuple1(_1) + +/** + * Returns a new Tuple2 of the given arguments. + * @see tupleOf + * @see X + */ +fun t(_1: T1, _2: T2): Tuple2 = Tuple2(_1, _2) + +/** + * Returns a new Tuple3 of the given arguments. + * @see tupleOf + * @see X + */ +fun t(_1: T1, _2: T2, _3: T3): Tuple3 = Tuple3(_1, _2, _3) + +/** + * Returns a new Tuple4 of the given arguments. + * @see tupleOf + * @see X + */ +fun t(_1: T1, _2: T2, _3: T3, _4: T4): Tuple4 = Tuple4(_1, _2, _3, _4) + +/** + * Returns a new Tuple5 of the given arguments. + * @see tupleOf + * @see X + */ +fun t(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5): Tuple5 = Tuple5(_1, _2, _3, _4, _5) + +/** + * Returns a new Tuple6 of the given arguments. + * @see tupleOf + * @see X + */ +fun t(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6): Tuple6 = Tuple6(_1, _2, _3, _4, _5, _6) + +/** + * Returns a new Tuple7 of the given arguments. + * @see tupleOf + * @see X + */ +fun t(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7): Tuple7 = Tuple7(_1, _2, _3, _4, _5, _6, _7) + +/** + * Returns a new Tuple8 of the given arguments. + * @see tupleOf + * @see X + */ +fun t(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8): Tuple8 = Tuple8(_1, _2, _3, _4, _5, _6, _7, _8) + +/** + * Returns a new Tuple9 of the given arguments. + * @see tupleOf + * @see X + */ +fun t(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9): Tuple9 = Tuple9(_1, _2, _3, _4, _5, _6, _7, _8, _9) + +/** + * Returns a new Tuple10 of the given arguments. + * @see tupleOf + * @see X + */ +fun t(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10): Tuple10 = Tuple10(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10) + +/** + * Returns a new Tuple11 of the given arguments. + * @see tupleOf + * @see X + */ +fun t(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11): Tuple11 = Tuple11(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11) + +/** + * Returns a new Tuple12 of the given arguments. + * @see tupleOf + * @see X + */ +fun t(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12): Tuple12 = Tuple12(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12) + +/** + * Returns a new Tuple13 of the given arguments. + * @see tupleOf + * @see X + */ +fun t(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13): Tuple13 = Tuple13(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13) + +/** + * Returns a new Tuple14 of the given arguments. + * @see tupleOf + * @see X + */ +fun t(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14): Tuple14 = Tuple14(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14) + +/** + * Returns a new Tuple15 of the given arguments. + * @see tupleOf + * @see X + */ +fun t(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15): Tuple15 = Tuple15(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15) + +/** + * Returns a new Tuple16 of the given arguments. + * @see tupleOf + * @see X + */ +fun t(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16): Tuple16 = Tuple16(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16) + +/** + * Returns a new Tuple17 of the given arguments. + * @see tupleOf + * @see X + */ +fun t(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16, _17: T17): Tuple17 = Tuple17(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17) + +/** + * Returns a new Tuple18 of the given arguments. + * @see tupleOf + * @see X + */ +fun t(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16, _17: T17, _18: T18): Tuple18 = Tuple18(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18) + +/** + * Returns a new Tuple19 of the given arguments. + * @see tupleOf + * @see X + */ +fun t(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16, _17: T17, _18: T18, _19: T19): Tuple19 = Tuple19(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19) + +/** + * Returns a new Tuple20 of the given arguments. + * @see tupleOf + * @see X + */ +fun t(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16, _17: T17, _18: T18, _19: T19, _20: T20): Tuple20 = Tuple20(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19, _20) + +/** + * Returns a new Tuple21 of the given arguments. + * @see tupleOf + * @see X + */ +fun t(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16, _17: T17, _18: T18, _19: T19, _20: T20, _21: T21): Tuple21 = Tuple21(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19, _20, _21) + +/** + * Returns a new Tuple22 of the given arguments. + * @see tupleOf + * @see X + */ +fun t(_1: T1, _2: T2, _3: T3, _4: T4, _5: T5, _6: T6, _7: T7, _8: T8, _9: T9, _10: T10, _11: T11, _12: T12, _13: T13, _14: T14, _15: T15, _16: T16, _17: T17, _18: T18, _19: T19, _20: T20, _21: T21, _22: T22): Tuple22 = Tuple22(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19, _20, _21, _22) diff --git a/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/TupleConcatenation.kt b/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/TupleConcatenation.kt new file mode 100644 index 00000000..fe07a4ff --- /dev/null +++ b/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/TupleConcatenation.kt @@ -0,0 +1,613 @@ +/*- + * =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("FunctionName", "RemoveExplicitTypeArguments") +package org.jetbrains.kotlinx.spark.api.tuples + +import scala.Tuple1 +import scala.Tuple2 +import scala.Tuple3 +import scala.Tuple4 +import scala.Tuple5 +import scala.Tuple6 +import scala.Tuple7 +import scala.Tuple8 +import scala.Tuple9 +import scala.Tuple10 +import scala.Tuple11 +import scala.Tuple12 +import scala.Tuple13 +import scala.Tuple14 +import scala.Tuple15 +import scala.Tuple16 +import scala.Tuple17 +import scala.Tuple18 +import scala.Tuple19 +import scala.Tuple20 +import scala.Tuple21 +import scala.Tuple22 + +/** + * This file provides functions to easily merge two separate tuples into one. + * + * For example (using tupleOf() to create a new tuple): + * ```tupleOf(a, b) concat tupleOf(c, d) == tupleOf(a, b, c, d)``` + * or using the shorthand: + * ```tupleOf(a, b) + tupleOf(c, d) == tupleOf(a, b, c, d)``` + * + * If you mean to create ```tupleOf(a, b, tupleOf(c, d))``` or ```tupleOf(tupleOf(a, b), c, d)```, + * use [appendedBy] and [prependedBy] explicitly: + * ```t(a, b).appendedBy(t(c, d)) == t(a, b, t(c, d))``` + * or wrap it in another [Tuple1]: + * ```t(a, b) + t(t(c, d)) == t(a, b, t(c, d))``` + * + */ + +infix fun EmptyTuple.concat(other: Tuple1): Tuple1 = other.copy() +infix fun Tuple1.concat(other: EmptyTuple): Tuple1 = this.copy() +infix fun Tuple1.concat(other: Tuple1): Tuple2 = Tuple2(this._1(), other._1()) +infix fun Tuple1.concat(other: Tuple2): Tuple3 = Tuple3(this._1(), other._1(), other._2()) +infix fun Tuple1.concat(other: Tuple3): Tuple4 = Tuple4(this._1(), other._1(), other._2(), other._3()) +infix fun Tuple1.concat(other: Tuple4): Tuple5 = Tuple5(this._1(), other._1(), other._2(), other._3(), other._4()) +infix fun Tuple1.concat(other: Tuple5): Tuple6 = Tuple6(this._1(), other._1(), other._2(), other._3(), other._4(), other._5()) +infix fun Tuple1.concat(other: Tuple6): Tuple7 = Tuple7(this._1(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6()) +infix fun Tuple1.concat(other: Tuple7): Tuple8 = Tuple8(this._1(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7()) +infix fun Tuple1.concat(other: Tuple8): Tuple9 = Tuple9(this._1(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8()) +infix fun Tuple1.concat(other: Tuple9): Tuple10 = Tuple10(this._1(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9()) +infix fun Tuple1.concat(other: Tuple10): Tuple11 = Tuple11(this._1(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10()) +infix fun Tuple1.concat(other: Tuple11): Tuple12 = Tuple12(this._1(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11()) +infix fun Tuple1.concat(other: Tuple12): Tuple13 = Tuple13(this._1(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12()) +infix fun Tuple1.concat(other: Tuple13): Tuple14 = Tuple14(this._1(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13()) +infix fun Tuple1.concat(other: Tuple14): Tuple15 = Tuple15(this._1(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14()) +infix fun Tuple1.concat(other: Tuple15): Tuple16 = Tuple16(this._1(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15()) +infix fun Tuple1.concat(other: Tuple16): Tuple17 = Tuple17(this._1(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16()) +infix fun Tuple1.concat(other: Tuple17): Tuple18 = Tuple18(this._1(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16(), other._17()) +infix fun Tuple1.concat(other: Tuple18): Tuple19 = Tuple19(this._1(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16(), other._17(), other._18()) +infix fun Tuple1.concat(other: Tuple19): Tuple20 = Tuple20(this._1(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16(), other._17(), other._18(), other._19()) +infix fun Tuple1.concat(other: Tuple20): Tuple21 = Tuple21(this._1(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16(), other._17(), other._18(), other._19(), other._20()) +infix fun Tuple1.concat(other: Tuple21): Tuple22 = Tuple22(this._1(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16(), other._17(), other._18(), other._19(), other._20(), other._21()) +infix fun EmptyTuple.concat(other: Tuple2): Tuple2 = other.copy() +infix fun Tuple2.concat(other: EmptyTuple): Tuple2 = this.copy() +infix fun Tuple2.concat(other: Tuple1): Tuple3 = Tuple3(this._1(), this._2(), other._1()) +infix fun Tuple2.concat(other: Tuple2): Tuple4 = Tuple4(this._1(), this._2(), other._1(), other._2()) +infix fun Tuple2.concat(other: Tuple3): Tuple5 = Tuple5(this._1(), this._2(), other._1(), other._2(), other._3()) +infix fun Tuple2.concat(other: Tuple4): Tuple6 = Tuple6(this._1(), this._2(), other._1(), other._2(), other._3(), other._4()) +infix fun Tuple2.concat(other: Tuple5): Tuple7 = Tuple7(this._1(), this._2(), other._1(), other._2(), other._3(), other._4(), other._5()) +infix fun Tuple2.concat(other: Tuple6): Tuple8 = Tuple8(this._1(), this._2(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6()) +infix fun Tuple2.concat(other: Tuple7): Tuple9 = Tuple9(this._1(), this._2(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7()) +infix fun Tuple2.concat(other: Tuple8): Tuple10 = Tuple10(this._1(), this._2(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8()) +infix fun Tuple2.concat(other: Tuple9): Tuple11 = Tuple11(this._1(), this._2(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9()) +infix fun Tuple2.concat(other: Tuple10): Tuple12 = Tuple12(this._1(), this._2(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10()) +infix fun Tuple2.concat(other: Tuple11): Tuple13 = Tuple13(this._1(), this._2(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11()) +infix fun Tuple2.concat(other: Tuple12): Tuple14 = Tuple14(this._1(), this._2(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12()) +infix fun Tuple2.concat(other: Tuple13): Tuple15 = Tuple15(this._1(), this._2(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13()) +infix fun Tuple2.concat(other: Tuple14): Tuple16 = Tuple16(this._1(), this._2(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14()) +infix fun Tuple2.concat(other: Tuple15): Tuple17 = Tuple17(this._1(), this._2(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15()) +infix fun Tuple2.concat(other: Tuple16): Tuple18 = Tuple18(this._1(), this._2(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16()) +infix fun Tuple2.concat(other: Tuple17): Tuple19 = Tuple19(this._1(), this._2(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16(), other._17()) +infix fun Tuple2.concat(other: Tuple18): Tuple20 = Tuple20(this._1(), this._2(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16(), other._17(), other._18()) +infix fun Tuple2.concat(other: Tuple19): Tuple21 = Tuple21(this._1(), this._2(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16(), other._17(), other._18(), other._19()) +infix fun Tuple2.concat(other: Tuple20): Tuple22 = Tuple22(this._1(), this._2(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16(), other._17(), other._18(), other._19(), other._20()) +infix fun EmptyTuple.concat(other: Tuple3): Tuple3 = other.copy() +infix fun Tuple3.concat(other: EmptyTuple): Tuple3 = this.copy() +infix fun Tuple3.concat(other: Tuple1): Tuple4 = Tuple4(this._1(), this._2(), this._3(), other._1()) +infix fun Tuple3.concat(other: Tuple2): Tuple5 = Tuple5(this._1(), this._2(), this._3(), other._1(), other._2()) +infix fun Tuple3.concat(other: Tuple3): Tuple6 = Tuple6(this._1(), this._2(), this._3(), other._1(), other._2(), other._3()) +infix fun Tuple3.concat(other: Tuple4): Tuple7 = Tuple7(this._1(), this._2(), this._3(), other._1(), other._2(), other._3(), other._4()) +infix fun Tuple3.concat(other: Tuple5): Tuple8 = Tuple8(this._1(), this._2(), this._3(), other._1(), other._2(), other._3(), other._4(), other._5()) +infix fun Tuple3.concat(other: Tuple6): Tuple9 = Tuple9(this._1(), this._2(), this._3(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6()) +infix fun Tuple3.concat(other: Tuple7): Tuple10 = Tuple10(this._1(), this._2(), this._3(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7()) +infix fun Tuple3.concat(other: Tuple8): Tuple11 = Tuple11(this._1(), this._2(), this._3(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8()) +infix fun Tuple3.concat(other: Tuple9): Tuple12 = Tuple12(this._1(), this._2(), this._3(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9()) +infix fun Tuple3.concat(other: Tuple10): Tuple13 = Tuple13(this._1(), this._2(), this._3(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10()) +infix fun Tuple3.concat(other: Tuple11): Tuple14 = Tuple14(this._1(), this._2(), this._3(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11()) +infix fun Tuple3.concat(other: Tuple12): Tuple15 = Tuple15(this._1(), this._2(), this._3(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12()) +infix fun Tuple3.concat(other: Tuple13): Tuple16 = Tuple16(this._1(), this._2(), this._3(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13()) +infix fun Tuple3.concat(other: Tuple14): Tuple17 = Tuple17(this._1(), this._2(), this._3(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14()) +infix fun Tuple3.concat(other: Tuple15): Tuple18 = Tuple18(this._1(), this._2(), this._3(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15()) +infix fun Tuple3.concat(other: Tuple16): Tuple19 = Tuple19(this._1(), this._2(), this._3(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16()) +infix fun Tuple3.concat(other: Tuple17): Tuple20 = Tuple20(this._1(), this._2(), this._3(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16(), other._17()) +infix fun Tuple3.concat(other: Tuple18): Tuple21 = Tuple21(this._1(), this._2(), this._3(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16(), other._17(), other._18()) +infix fun Tuple3.concat(other: Tuple19): Tuple22 = Tuple22(this._1(), this._2(), this._3(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16(), other._17(), other._18(), other._19()) +infix fun EmptyTuple.concat(other: Tuple4): Tuple4 = other.copy() +infix fun Tuple4.concat(other: EmptyTuple): Tuple4 = this.copy() +infix fun Tuple4.concat(other: Tuple1): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), other._1()) +infix fun Tuple4.concat(other: Tuple2): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), other._1(), other._2()) +infix fun Tuple4.concat(other: Tuple3): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), other._1(), other._2(), other._3()) +infix fun Tuple4.concat(other: Tuple4): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), other._1(), other._2(), other._3(), other._4()) +infix fun Tuple4.concat(other: Tuple5): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), other._1(), other._2(), other._3(), other._4(), other._5()) +infix fun Tuple4.concat(other: Tuple6): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6()) +infix fun Tuple4.concat(other: Tuple7): Tuple11 = Tuple11(this._1(), this._2(), this._3(), this._4(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7()) +infix fun Tuple4.concat(other: Tuple8): Tuple12 = Tuple12(this._1(), this._2(), this._3(), this._4(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8()) +infix fun Tuple4.concat(other: Tuple9): Tuple13 = Tuple13(this._1(), this._2(), this._3(), this._4(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9()) +infix fun Tuple4.concat(other: Tuple10): Tuple14 = Tuple14(this._1(), this._2(), this._3(), this._4(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10()) +infix fun Tuple4.concat(other: Tuple11): Tuple15 = Tuple15(this._1(), this._2(), this._3(), this._4(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11()) +infix fun Tuple4.concat(other: Tuple12): Tuple16 = Tuple16(this._1(), this._2(), this._3(), this._4(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12()) +infix fun Tuple4.concat(other: Tuple13): Tuple17 = Tuple17(this._1(), this._2(), this._3(), this._4(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13()) +infix fun Tuple4.concat(other: Tuple14): Tuple18 = Tuple18(this._1(), this._2(), this._3(), this._4(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14()) +infix fun Tuple4.concat(other: Tuple15): Tuple19 = Tuple19(this._1(), this._2(), this._3(), this._4(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15()) +infix fun Tuple4.concat(other: Tuple16): Tuple20 = Tuple20(this._1(), this._2(), this._3(), this._4(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16()) +infix fun Tuple4.concat(other: Tuple17): Tuple21 = Tuple21(this._1(), this._2(), this._3(), this._4(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16(), other._17()) +infix fun Tuple4.concat(other: Tuple18): Tuple22 = Tuple22(this._1(), this._2(), this._3(), this._4(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16(), other._17(), other._18()) +infix fun EmptyTuple.concat(other: Tuple5): Tuple5 = other.copy() +infix fun Tuple5.concat(other: EmptyTuple): Tuple5 = this.copy() +infix fun Tuple5.concat(other: Tuple1): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), other._1()) +infix fun Tuple5.concat(other: Tuple2): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), other._1(), other._2()) +infix fun Tuple5.concat(other: Tuple3): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), other._1(), other._2(), other._3()) +infix fun Tuple5.concat(other: Tuple4): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), other._1(), other._2(), other._3(), other._4()) +infix fun Tuple5.concat(other: Tuple5): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), other._1(), other._2(), other._3(), other._4(), other._5()) +infix fun Tuple5.concat(other: Tuple6): Tuple11 = Tuple11(this._1(), this._2(), this._3(), this._4(), this._5(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6()) +infix fun Tuple5.concat(other: Tuple7): Tuple12 = Tuple12(this._1(), this._2(), this._3(), this._4(), this._5(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7()) +infix fun Tuple5.concat(other: Tuple8): Tuple13 = Tuple13(this._1(), this._2(), this._3(), this._4(), this._5(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8()) +infix fun Tuple5.concat(other: Tuple9): Tuple14 = Tuple14(this._1(), this._2(), this._3(), this._4(), this._5(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9()) +infix fun Tuple5.concat(other: Tuple10): Tuple15 = Tuple15(this._1(), this._2(), this._3(), this._4(), this._5(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10()) +infix fun Tuple5.concat(other: Tuple11): Tuple16 = Tuple16(this._1(), this._2(), this._3(), this._4(), this._5(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11()) +infix fun Tuple5.concat(other: Tuple12): Tuple17 = Tuple17(this._1(), this._2(), this._3(), this._4(), this._5(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12()) +infix fun Tuple5.concat(other: Tuple13): Tuple18 = Tuple18(this._1(), this._2(), this._3(), this._4(), this._5(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13()) +infix fun Tuple5.concat(other: Tuple14): Tuple19 = Tuple19(this._1(), this._2(), this._3(), this._4(), this._5(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14()) +infix fun Tuple5.concat(other: Tuple15): Tuple20 = Tuple20(this._1(), this._2(), this._3(), this._4(), this._5(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15()) +infix fun Tuple5.concat(other: Tuple16): Tuple21 = Tuple21(this._1(), this._2(), this._3(), this._4(), this._5(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16()) +infix fun Tuple5.concat(other: Tuple17): Tuple22 = Tuple22(this._1(), this._2(), this._3(), this._4(), this._5(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16(), other._17()) +infix fun EmptyTuple.concat(other: Tuple6): Tuple6 = other.copy() +infix fun Tuple6.concat(other: EmptyTuple): Tuple6 = this.copy() +infix fun Tuple6.concat(other: Tuple1): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), other._1()) +infix fun Tuple6.concat(other: Tuple2): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), other._1(), other._2()) +infix fun Tuple6.concat(other: Tuple3): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), other._1(), other._2(), other._3()) +infix fun Tuple6.concat(other: Tuple4): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), other._1(), other._2(), other._3(), other._4()) +infix fun Tuple6.concat(other: Tuple5): Tuple11 = Tuple11(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), other._1(), other._2(), other._3(), other._4(), other._5()) +infix fun Tuple6.concat(other: Tuple6): Tuple12 = Tuple12(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6()) +infix fun Tuple6.concat(other: Tuple7): Tuple13 = Tuple13(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7()) +infix fun Tuple6.concat(other: Tuple8): Tuple14 = Tuple14(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8()) +infix fun Tuple6.concat(other: Tuple9): Tuple15 = Tuple15(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9()) +infix fun Tuple6.concat(other: Tuple10): Tuple16 = Tuple16(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10()) +infix fun Tuple6.concat(other: Tuple11): Tuple17 = Tuple17(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11()) +infix fun Tuple6.concat(other: Tuple12): Tuple18 = Tuple18(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12()) +infix fun Tuple6.concat(other: Tuple13): Tuple19 = Tuple19(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13()) +infix fun Tuple6.concat(other: Tuple14): Tuple20 = Tuple20(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14()) +infix fun Tuple6.concat(other: Tuple15): Tuple21 = Tuple21(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15()) +infix fun Tuple6.concat(other: Tuple16): Tuple22 = Tuple22(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16()) +infix fun EmptyTuple.concat(other: Tuple7): Tuple7 = other.copy() +infix fun Tuple7.concat(other: EmptyTuple): Tuple7 = this.copy() +infix fun Tuple7.concat(other: Tuple1): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), other._1()) +infix fun Tuple7.concat(other: Tuple2): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), other._1(), other._2()) +infix fun Tuple7.concat(other: Tuple3): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), other._1(), other._2(), other._3()) +infix fun Tuple7.concat(other: Tuple4): Tuple11 = Tuple11(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), other._1(), other._2(), other._3(), other._4()) +infix fun Tuple7.concat(other: Tuple5): Tuple12 = Tuple12(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), other._1(), other._2(), other._3(), other._4(), other._5()) +infix fun Tuple7.concat(other: Tuple6): Tuple13 = Tuple13(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6()) +infix fun Tuple7.concat(other: Tuple7): Tuple14 = Tuple14(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7()) +infix fun Tuple7.concat(other: Tuple8): Tuple15 = Tuple15(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8()) +infix fun Tuple7.concat(other: Tuple9): Tuple16 = Tuple16(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9()) +infix fun Tuple7.concat(other: Tuple10): Tuple17 = Tuple17(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10()) +infix fun Tuple7.concat(other: Tuple11): Tuple18 = Tuple18(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11()) +infix fun Tuple7.concat(other: Tuple12): Tuple19 = Tuple19(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12()) +infix fun Tuple7.concat(other: Tuple13): Tuple20 = Tuple20(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13()) +infix fun Tuple7.concat(other: Tuple14): Tuple21 = Tuple21(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14()) +infix fun Tuple7.concat(other: Tuple15): Tuple22 = Tuple22(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15()) +infix fun EmptyTuple.concat(other: Tuple8): Tuple8 = other.copy() +infix fun Tuple8.concat(other: EmptyTuple): Tuple8 = this.copy() +infix fun Tuple8.concat(other: Tuple1): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), other._1()) +infix fun Tuple8.concat(other: Tuple2): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), other._1(), other._2()) +infix fun Tuple8.concat(other: Tuple3): Tuple11 = Tuple11(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), other._1(), other._2(), other._3()) +infix fun Tuple8.concat(other: Tuple4): Tuple12 = Tuple12(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), other._1(), other._2(), other._3(), other._4()) +infix fun Tuple8.concat(other: Tuple5): Tuple13 = Tuple13(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), other._1(), other._2(), other._3(), other._4(), other._5()) +infix fun Tuple8.concat(other: Tuple6): Tuple14 = Tuple14(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6()) +infix fun Tuple8.concat(other: Tuple7): Tuple15 = Tuple15(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7()) +infix fun Tuple8.concat(other: Tuple8): Tuple16 = Tuple16(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8()) +infix fun Tuple8.concat(other: Tuple9): Tuple17 = Tuple17(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9()) +infix fun Tuple8.concat(other: Tuple10): Tuple18 = Tuple18(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10()) +infix fun Tuple8.concat(other: Tuple11): Tuple19 = Tuple19(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11()) +infix fun Tuple8.concat(other: Tuple12): Tuple20 = Tuple20(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12()) +infix fun Tuple8.concat(other: Tuple13): Tuple21 = Tuple21(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13()) +infix fun Tuple8.concat(other: Tuple14): Tuple22 = Tuple22(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14()) +infix fun EmptyTuple.concat(other: Tuple9): Tuple9 = other.copy() +infix fun Tuple9.concat(other: EmptyTuple): Tuple9 = this.copy() +infix fun Tuple9.concat(other: Tuple1): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), other._1()) +infix fun Tuple9.concat(other: Tuple2): Tuple11 = Tuple11(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), other._1(), other._2()) +infix fun Tuple9.concat(other: Tuple3): Tuple12 = Tuple12(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), other._1(), other._2(), other._3()) +infix fun Tuple9.concat(other: Tuple4): Tuple13 = Tuple13(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), other._1(), other._2(), other._3(), other._4()) +infix fun Tuple9.concat(other: Tuple5): Tuple14 = Tuple14(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), other._1(), other._2(), other._3(), other._4(), other._5()) +infix fun Tuple9.concat(other: Tuple6): Tuple15 = Tuple15(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6()) +infix fun Tuple9.concat(other: Tuple7): Tuple16 = Tuple16(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7()) +infix fun Tuple9.concat(other: Tuple8): Tuple17 = Tuple17(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8()) +infix fun Tuple9.concat(other: Tuple9): Tuple18 = Tuple18(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9()) +infix fun Tuple9.concat(other: Tuple10): Tuple19 = Tuple19(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10()) +infix fun Tuple9.concat(other: Tuple11): Tuple20 = Tuple20(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11()) +infix fun Tuple9.concat(other: Tuple12): Tuple21 = Tuple21(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12()) +infix fun Tuple9.concat(other: Tuple13): Tuple22 = Tuple22(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13()) +infix fun EmptyTuple.concat(other: Tuple10): Tuple10 = other.copy() +infix fun Tuple10.concat(other: EmptyTuple): Tuple10 = this.copy() +infix fun Tuple10.concat(other: Tuple1): Tuple11 = Tuple11(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), other._1()) +infix fun Tuple10.concat(other: Tuple2): Tuple12 = Tuple12(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), other._1(), other._2()) +infix fun Tuple10.concat(other: Tuple3): Tuple13 = Tuple13(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), other._1(), other._2(), other._3()) +infix fun Tuple10.concat(other: Tuple4): Tuple14 = Tuple14(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), other._1(), other._2(), other._3(), other._4()) +infix fun Tuple10.concat(other: Tuple5): Tuple15 = Tuple15(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), other._1(), other._2(), other._3(), other._4(), other._5()) +infix fun Tuple10.concat(other: Tuple6): Tuple16 = Tuple16(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6()) +infix fun Tuple10.concat(other: Tuple7): Tuple17 = Tuple17(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7()) +infix fun Tuple10.concat(other: Tuple8): Tuple18 = Tuple18(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8()) +infix fun Tuple10.concat(other: Tuple9): Tuple19 = Tuple19(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9()) +infix fun Tuple10.concat(other: Tuple10): Tuple20 = Tuple20(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10()) +infix fun Tuple10.concat(other: Tuple11): Tuple21 = Tuple21(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11()) +infix fun Tuple10.concat(other: Tuple12): Tuple22 = Tuple22(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12()) +infix fun EmptyTuple.concat(other: Tuple11): Tuple11 = other.copy() +infix fun Tuple11.concat(other: EmptyTuple): Tuple11 = this.copy() +infix fun Tuple11.concat(other: Tuple1): Tuple12 = Tuple12(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), other._1()) +infix fun Tuple11.concat(other: Tuple2): Tuple13 = Tuple13(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), other._1(), other._2()) +infix fun Tuple11.concat(other: Tuple3): Tuple14 = Tuple14(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), other._1(), other._2(), other._3()) +infix fun Tuple11.concat(other: Tuple4): Tuple15 = Tuple15(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), other._1(), other._2(), other._3(), other._4()) +infix fun Tuple11.concat(other: Tuple5): Tuple16 = Tuple16(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), other._1(), other._2(), other._3(), other._4(), other._5()) +infix fun Tuple11.concat(other: Tuple6): Tuple17 = Tuple17(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6()) +infix fun Tuple11.concat(other: Tuple7): Tuple18 = Tuple18(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7()) +infix fun Tuple11.concat(other: Tuple8): Tuple19 = Tuple19(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8()) +infix fun Tuple11.concat(other: Tuple9): Tuple20 = Tuple20(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9()) +infix fun Tuple11.concat(other: Tuple10): Tuple21 = Tuple21(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10()) +infix fun Tuple11.concat(other: Tuple11): Tuple22 = Tuple22(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11()) +infix fun EmptyTuple.concat(other: Tuple12): Tuple12 = other.copy() +infix fun Tuple12.concat(other: EmptyTuple): Tuple12 = this.copy() +infix fun Tuple12.concat(other: Tuple1): 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(), other._1()) +infix fun Tuple12.concat(other: Tuple2): 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(), other._1(), other._2()) +infix fun Tuple12.concat(other: Tuple3): 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(), other._1(), other._2(), other._3()) +infix fun Tuple12.concat(other: Tuple4): 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(), other._1(), other._2(), other._3(), other._4()) +infix fun Tuple12.concat(other: Tuple5): 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(), other._1(), other._2(), other._3(), other._4(), other._5()) +infix fun Tuple12.concat(other: Tuple6): 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(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6()) +infix fun Tuple12.concat(other: Tuple7): 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(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7()) +infix fun Tuple12.concat(other: Tuple8): 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(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8()) +infix fun Tuple12.concat(other: Tuple9): 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(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9()) +infix fun Tuple12.concat(other: Tuple10): 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(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10()) +infix fun EmptyTuple.concat(other: Tuple13): Tuple13 = other.copy() +infix fun Tuple13.concat(other: EmptyTuple): Tuple13 = this.copy() +infix fun Tuple13.concat(other: Tuple1): 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(), other._1()) +infix fun Tuple13.concat(other: Tuple2): 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(), other._1(), other._2()) +infix fun Tuple13.concat(other: Tuple3): 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(), other._1(), other._2(), other._3()) +infix fun Tuple13.concat(other: Tuple4): 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(), other._1(), other._2(), other._3(), other._4()) +infix fun Tuple13.concat(other: Tuple5): 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(), other._1(), other._2(), other._3(), other._4(), other._5()) +infix fun Tuple13.concat(other: Tuple6): 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(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6()) +infix fun Tuple13.concat(other: Tuple7): 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(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7()) +infix fun Tuple13.concat(other: Tuple8): 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(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8()) +infix fun Tuple13.concat(other: Tuple9): 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(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9()) +infix fun EmptyTuple.concat(other: Tuple14): Tuple14 = other.copy() +infix fun Tuple14.concat(other: EmptyTuple): Tuple14 = this.copy() +infix fun Tuple14.concat(other: Tuple1): 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(), other._1()) +infix fun Tuple14.concat(other: Tuple2): 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(), other._1(), other._2()) +infix fun Tuple14.concat(other: Tuple3): 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(), other._1(), other._2(), other._3()) +infix fun Tuple14.concat(other: Tuple4): 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(), other._1(), other._2(), other._3(), other._4()) +infix fun Tuple14.concat(other: Tuple5): 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(), other._1(), other._2(), other._3(), other._4(), other._5()) +infix fun Tuple14.concat(other: Tuple6): 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(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6()) +infix fun Tuple14.concat(other: Tuple7): 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(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7()) +infix fun Tuple14.concat(other: Tuple8): 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(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8()) +infix fun EmptyTuple.concat(other: Tuple15): Tuple15 = other.copy() +infix fun Tuple15.concat(other: EmptyTuple): Tuple15 = this.copy() +infix fun Tuple15.concat(other: Tuple1): 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(), other._1()) +infix fun Tuple15.concat(other: Tuple2): 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(), other._1(), other._2()) +infix fun Tuple15.concat(other: Tuple3): 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(), other._1(), other._2(), other._3()) +infix fun Tuple15.concat(other: Tuple4): 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(), other._1(), other._2(), other._3(), other._4()) +infix fun Tuple15.concat(other: Tuple5): 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(), other._1(), other._2(), other._3(), other._4(), other._5()) +infix fun Tuple15.concat(other: Tuple6): 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(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6()) +infix fun Tuple15.concat(other: Tuple7): 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(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7()) +infix fun EmptyTuple.concat(other: Tuple16): Tuple16 = other.copy() +infix fun Tuple16.concat(other: EmptyTuple): Tuple16 = this.copy() +infix fun Tuple16.concat(other: Tuple1): 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(), other._1()) +infix fun Tuple16.concat(other: Tuple2): 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(), other._1(), other._2()) +infix fun Tuple16.concat(other: Tuple3): 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(), other._1(), other._2(), other._3()) +infix fun Tuple16.concat(other: Tuple4): 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(), other._1(), other._2(), other._3(), other._4()) +infix fun Tuple16.concat(other: Tuple5): 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(), other._1(), other._2(), other._3(), other._4(), other._5()) +infix fun Tuple16.concat(other: Tuple6): 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(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6()) +infix fun EmptyTuple.concat(other: Tuple17): Tuple17 = other.copy() +infix fun Tuple17.concat(other: EmptyTuple): Tuple17 = this.copy() +infix fun Tuple17.concat(other: Tuple1): 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(), other._1()) +infix fun Tuple17.concat(other: Tuple2): 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(), other._1(), other._2()) +infix fun Tuple17.concat(other: Tuple3): 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(), other._1(), other._2(), other._3()) +infix fun Tuple17.concat(other: Tuple4): 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(), other._1(), other._2(), other._3(), other._4()) +infix fun Tuple17.concat(other: Tuple5): 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(), other._1(), other._2(), other._3(), other._4(), other._5()) +infix fun EmptyTuple.concat(other: Tuple18): Tuple18 = other.copy() +infix fun Tuple18.concat(other: EmptyTuple): Tuple18 = this.copy() +infix fun Tuple18.concat(other: Tuple1): 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(), other._1()) +infix fun Tuple18.concat(other: Tuple2): 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(), other._1(), other._2()) +infix fun Tuple18.concat(other: Tuple3): 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(), other._1(), other._2(), other._3()) +infix fun Tuple18.concat(other: Tuple4): 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(), other._1(), other._2(), other._3(), other._4()) +infix fun EmptyTuple.concat(other: Tuple19): Tuple19 = other.copy() +infix fun Tuple19.concat(other: EmptyTuple): Tuple19 = this.copy() +infix fun Tuple19.concat(other: Tuple1): 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(), other._1()) +infix fun Tuple19.concat(other: Tuple2): 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(), other._1(), other._2()) +infix fun Tuple19.concat(other: Tuple3): 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(), other._1(), other._2(), other._3()) +infix fun EmptyTuple.concat(other: Tuple20): Tuple20 = other.copy() +infix fun Tuple20.concat(other: EmptyTuple): Tuple20 = this.copy() +infix fun Tuple20.concat(other: Tuple1): 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(), other._1()) +infix fun Tuple20.concat(other: Tuple2): 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(), other._1(), other._2()) +infix fun EmptyTuple.concat(other: Tuple21): Tuple21 = other.copy() +infix fun Tuple21.concat(other: EmptyTuple): Tuple21 = this.copy() +infix fun Tuple21.concat(other: Tuple1): 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(), other._1()) +infix fun EmptyTuple.concat(other: Tuple22): Tuple22 = other.copy() +infix fun Tuple22.concat(other: EmptyTuple): Tuple22 = this.copy() + +operator fun EmptyTuple.plus(other: Tuple1): Tuple1 = other.copy() +operator fun Tuple1.plus(other: EmptyTuple): Tuple1 = this.copy() +operator fun Tuple1.plus(other: Tuple1): Tuple2 = Tuple2(this._1(), other._1()) +operator fun Tuple1.plus(other: Tuple2): Tuple3 = Tuple3(this._1(), other._1(), other._2()) +operator fun Tuple1.plus(other: Tuple3): Tuple4 = Tuple4(this._1(), other._1(), other._2(), other._3()) +operator fun Tuple1.plus(other: Tuple4): Tuple5 = Tuple5(this._1(), other._1(), other._2(), other._3(), other._4()) +operator fun Tuple1.plus(other: Tuple5): Tuple6 = Tuple6(this._1(), other._1(), other._2(), other._3(), other._4(), other._5()) +operator fun Tuple1.plus(other: Tuple6): Tuple7 = Tuple7(this._1(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6()) +operator fun Tuple1.plus(other: Tuple7): Tuple8 = Tuple8(this._1(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7()) +operator fun Tuple1.plus(other: Tuple8): Tuple9 = Tuple9(this._1(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8()) +operator fun Tuple1.plus(other: Tuple9): Tuple10 = Tuple10(this._1(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9()) +operator fun Tuple1.plus(other: Tuple10): Tuple11 = Tuple11(this._1(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10()) +operator fun Tuple1.plus(other: Tuple11): Tuple12 = Tuple12(this._1(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11()) +operator fun Tuple1.plus(other: Tuple12): Tuple13 = Tuple13(this._1(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12()) +operator fun Tuple1.plus(other: Tuple13): Tuple14 = Tuple14(this._1(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13()) +operator fun Tuple1.plus(other: Tuple14): Tuple15 = Tuple15(this._1(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14()) +operator fun Tuple1.plus(other: Tuple15): Tuple16 = Tuple16(this._1(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15()) +operator fun Tuple1.plus(other: Tuple16): Tuple17 = Tuple17(this._1(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16()) +operator fun Tuple1.plus(other: Tuple17): Tuple18 = Tuple18(this._1(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16(), other._17()) +operator fun Tuple1.plus(other: Tuple18): Tuple19 = Tuple19(this._1(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16(), other._17(), other._18()) +operator fun Tuple1.plus(other: Tuple19): Tuple20 = Tuple20(this._1(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16(), other._17(), other._18(), other._19()) +operator fun Tuple1.plus(other: Tuple20): Tuple21 = Tuple21(this._1(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16(), other._17(), other._18(), other._19(), other._20()) +operator fun Tuple1.plus(other: Tuple21): Tuple22 = Tuple22(this._1(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16(), other._17(), other._18(), other._19(), other._20(), other._21()) +operator fun EmptyTuple.plus(other: Tuple2): Tuple2 = other.copy() +operator fun Tuple2.plus(other: EmptyTuple): Tuple2 = this.copy() +operator fun Tuple2.plus(other: Tuple1): Tuple3 = Tuple3(this._1(), this._2(), other._1()) +operator fun Tuple2.plus(other: Tuple2): Tuple4 = Tuple4(this._1(), this._2(), other._1(), other._2()) +operator fun Tuple2.plus(other: Tuple3): Tuple5 = Tuple5(this._1(), this._2(), other._1(), other._2(), other._3()) +operator fun Tuple2.plus(other: Tuple4): Tuple6 = Tuple6(this._1(), this._2(), other._1(), other._2(), other._3(), other._4()) +operator fun Tuple2.plus(other: Tuple5): Tuple7 = Tuple7(this._1(), this._2(), other._1(), other._2(), other._3(), other._4(), other._5()) +operator fun Tuple2.plus(other: Tuple6): Tuple8 = Tuple8(this._1(), this._2(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6()) +operator fun Tuple2.plus(other: Tuple7): Tuple9 = Tuple9(this._1(), this._2(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7()) +operator fun Tuple2.plus(other: Tuple8): Tuple10 = Tuple10(this._1(), this._2(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8()) +operator fun Tuple2.plus(other: Tuple9): Tuple11 = Tuple11(this._1(), this._2(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9()) +operator fun Tuple2.plus(other: Tuple10): Tuple12 = Tuple12(this._1(), this._2(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10()) +operator fun Tuple2.plus(other: Tuple11): Tuple13 = Tuple13(this._1(), this._2(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11()) +operator fun Tuple2.plus(other: Tuple12): Tuple14 = Tuple14(this._1(), this._2(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12()) +operator fun Tuple2.plus(other: Tuple13): Tuple15 = Tuple15(this._1(), this._2(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13()) +operator fun Tuple2.plus(other: Tuple14): Tuple16 = Tuple16(this._1(), this._2(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14()) +operator fun Tuple2.plus(other: Tuple15): Tuple17 = Tuple17(this._1(), this._2(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15()) +operator fun Tuple2.plus(other: Tuple16): Tuple18 = Tuple18(this._1(), this._2(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16()) +operator fun Tuple2.plus(other: Tuple17): Tuple19 = Tuple19(this._1(), this._2(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16(), other._17()) +operator fun Tuple2.plus(other: Tuple18): Tuple20 = Tuple20(this._1(), this._2(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16(), other._17(), other._18()) +operator fun Tuple2.plus(other: Tuple19): Tuple21 = Tuple21(this._1(), this._2(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16(), other._17(), other._18(), other._19()) +operator fun Tuple2.plus(other: Tuple20): Tuple22 = Tuple22(this._1(), this._2(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16(), other._17(), other._18(), other._19(), other._20()) +operator fun EmptyTuple.plus(other: Tuple3): Tuple3 = other.copy() +operator fun Tuple3.plus(other: EmptyTuple): Tuple3 = this.copy() +operator fun Tuple3.plus(other: Tuple1): Tuple4 = Tuple4(this._1(), this._2(), this._3(), other._1()) +operator fun Tuple3.plus(other: Tuple2): Tuple5 = Tuple5(this._1(), this._2(), this._3(), other._1(), other._2()) +operator fun Tuple3.plus(other: Tuple3): Tuple6 = Tuple6(this._1(), this._2(), this._3(), other._1(), other._2(), other._3()) +operator fun Tuple3.plus(other: Tuple4): Tuple7 = Tuple7(this._1(), this._2(), this._3(), other._1(), other._2(), other._3(), other._4()) +operator fun Tuple3.plus(other: Tuple5): Tuple8 = Tuple8(this._1(), this._2(), this._3(), other._1(), other._2(), other._3(), other._4(), other._5()) +operator fun Tuple3.plus(other: Tuple6): Tuple9 = Tuple9(this._1(), this._2(), this._3(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6()) +operator fun Tuple3.plus(other: Tuple7): Tuple10 = Tuple10(this._1(), this._2(), this._3(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7()) +operator fun Tuple3.plus(other: Tuple8): Tuple11 = Tuple11(this._1(), this._2(), this._3(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8()) +operator fun Tuple3.plus(other: Tuple9): Tuple12 = Tuple12(this._1(), this._2(), this._3(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9()) +operator fun Tuple3.plus(other: Tuple10): Tuple13 = Tuple13(this._1(), this._2(), this._3(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10()) +operator fun Tuple3.plus(other: Tuple11): Tuple14 = Tuple14(this._1(), this._2(), this._3(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11()) +operator fun Tuple3.plus(other: Tuple12): Tuple15 = Tuple15(this._1(), this._2(), this._3(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12()) +operator fun Tuple3.plus(other: Tuple13): Tuple16 = Tuple16(this._1(), this._2(), this._3(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13()) +operator fun Tuple3.plus(other: Tuple14): Tuple17 = Tuple17(this._1(), this._2(), this._3(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14()) +operator fun Tuple3.plus(other: Tuple15): Tuple18 = Tuple18(this._1(), this._2(), this._3(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15()) +operator fun Tuple3.plus(other: Tuple16): Tuple19 = Tuple19(this._1(), this._2(), this._3(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16()) +operator fun Tuple3.plus(other: Tuple17): Tuple20 = Tuple20(this._1(), this._2(), this._3(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16(), other._17()) +operator fun Tuple3.plus(other: Tuple18): Tuple21 = Tuple21(this._1(), this._2(), this._3(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16(), other._17(), other._18()) +operator fun Tuple3.plus(other: Tuple19): Tuple22 = Tuple22(this._1(), this._2(), this._3(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16(), other._17(), other._18(), other._19()) +operator fun EmptyTuple.plus(other: Tuple4): Tuple4 = other.copy() +operator fun Tuple4.plus(other: EmptyTuple): Tuple4 = this.copy() +operator fun Tuple4.plus(other: Tuple1): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), other._1()) +operator fun Tuple4.plus(other: Tuple2): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), other._1(), other._2()) +operator fun Tuple4.plus(other: Tuple3): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), other._1(), other._2(), other._3()) +operator fun Tuple4.plus(other: Tuple4): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), other._1(), other._2(), other._3(), other._4()) +operator fun Tuple4.plus(other: Tuple5): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), other._1(), other._2(), other._3(), other._4(), other._5()) +operator fun Tuple4.plus(other: Tuple6): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6()) +operator fun Tuple4.plus(other: Tuple7): Tuple11 = Tuple11(this._1(), this._2(), this._3(), this._4(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7()) +operator fun Tuple4.plus(other: Tuple8): Tuple12 = Tuple12(this._1(), this._2(), this._3(), this._4(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8()) +operator fun Tuple4.plus(other: Tuple9): Tuple13 = Tuple13(this._1(), this._2(), this._3(), this._4(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9()) +operator fun Tuple4.plus(other: Tuple10): Tuple14 = Tuple14(this._1(), this._2(), this._3(), this._4(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10()) +operator fun Tuple4.plus(other: Tuple11): Tuple15 = Tuple15(this._1(), this._2(), this._3(), this._4(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11()) +operator fun Tuple4.plus(other: Tuple12): Tuple16 = Tuple16(this._1(), this._2(), this._3(), this._4(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12()) +operator fun Tuple4.plus(other: Tuple13): Tuple17 = Tuple17(this._1(), this._2(), this._3(), this._4(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13()) +operator fun Tuple4.plus(other: Tuple14): Tuple18 = Tuple18(this._1(), this._2(), this._3(), this._4(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14()) +operator fun Tuple4.plus(other: Tuple15): Tuple19 = Tuple19(this._1(), this._2(), this._3(), this._4(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15()) +operator fun Tuple4.plus(other: Tuple16): Tuple20 = Tuple20(this._1(), this._2(), this._3(), this._4(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16()) +operator fun Tuple4.plus(other: Tuple17): Tuple21 = Tuple21(this._1(), this._2(), this._3(), this._4(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16(), other._17()) +operator fun Tuple4.plus(other: Tuple18): Tuple22 = Tuple22(this._1(), this._2(), this._3(), this._4(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16(), other._17(), other._18()) +operator fun EmptyTuple.plus(other: Tuple5): Tuple5 = other.copy() +operator fun Tuple5.plus(other: EmptyTuple): Tuple5 = this.copy() +operator fun Tuple5.plus(other: Tuple1): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), other._1()) +operator fun Tuple5.plus(other: Tuple2): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), other._1(), other._2()) +operator fun Tuple5.plus(other: Tuple3): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), other._1(), other._2(), other._3()) +operator fun Tuple5.plus(other: Tuple4): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), other._1(), other._2(), other._3(), other._4()) +operator fun Tuple5.plus(other: Tuple5): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), other._1(), other._2(), other._3(), other._4(), other._5()) +operator fun Tuple5.plus(other: Tuple6): Tuple11 = Tuple11(this._1(), this._2(), this._3(), this._4(), this._5(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6()) +operator fun Tuple5.plus(other: Tuple7): Tuple12 = Tuple12(this._1(), this._2(), this._3(), this._4(), this._5(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7()) +operator fun Tuple5.plus(other: Tuple8): Tuple13 = Tuple13(this._1(), this._2(), this._3(), this._4(), this._5(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8()) +operator fun Tuple5.plus(other: Tuple9): Tuple14 = Tuple14(this._1(), this._2(), this._3(), this._4(), this._5(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9()) +operator fun Tuple5.plus(other: Tuple10): Tuple15 = Tuple15(this._1(), this._2(), this._3(), this._4(), this._5(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10()) +operator fun Tuple5.plus(other: Tuple11): Tuple16 = Tuple16(this._1(), this._2(), this._3(), this._4(), this._5(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11()) +operator fun Tuple5.plus(other: Tuple12): Tuple17 = Tuple17(this._1(), this._2(), this._3(), this._4(), this._5(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12()) +operator fun Tuple5.plus(other: Tuple13): Tuple18 = Tuple18(this._1(), this._2(), this._3(), this._4(), this._5(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13()) +operator fun Tuple5.plus(other: Tuple14): Tuple19 = Tuple19(this._1(), this._2(), this._3(), this._4(), this._5(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14()) +operator fun Tuple5.plus(other: Tuple15): Tuple20 = Tuple20(this._1(), this._2(), this._3(), this._4(), this._5(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15()) +operator fun Tuple5.plus(other: Tuple16): Tuple21 = Tuple21(this._1(), this._2(), this._3(), this._4(), this._5(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16()) +operator fun Tuple5.plus(other: Tuple17): Tuple22 = Tuple22(this._1(), this._2(), this._3(), this._4(), this._5(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16(), other._17()) +operator fun EmptyTuple.plus(other: Tuple6): Tuple6 = other.copy() +operator fun Tuple6.plus(other: EmptyTuple): Tuple6 = this.copy() +operator fun Tuple6.plus(other: Tuple1): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), other._1()) +operator fun Tuple6.plus(other: Tuple2): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), other._1(), other._2()) +operator fun Tuple6.plus(other: Tuple3): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), other._1(), other._2(), other._3()) +operator fun Tuple6.plus(other: Tuple4): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), other._1(), other._2(), other._3(), other._4()) +operator fun Tuple6.plus(other: Tuple5): Tuple11 = Tuple11(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), other._1(), other._2(), other._3(), other._4(), other._5()) +operator fun Tuple6.plus(other: Tuple6): Tuple12 = Tuple12(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6()) +operator fun Tuple6.plus(other: Tuple7): Tuple13 = Tuple13(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7()) +operator fun Tuple6.plus(other: Tuple8): Tuple14 = Tuple14(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8()) +operator fun Tuple6.plus(other: Tuple9): Tuple15 = Tuple15(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9()) +operator fun Tuple6.plus(other: Tuple10): Tuple16 = Tuple16(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10()) +operator fun Tuple6.plus(other: Tuple11): Tuple17 = Tuple17(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11()) +operator fun Tuple6.plus(other: Tuple12): Tuple18 = Tuple18(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12()) +operator fun Tuple6.plus(other: Tuple13): Tuple19 = Tuple19(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13()) +operator fun Tuple6.plus(other: Tuple14): Tuple20 = Tuple20(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14()) +operator fun Tuple6.plus(other: Tuple15): Tuple21 = Tuple21(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15()) +operator fun Tuple6.plus(other: Tuple16): Tuple22 = Tuple22(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15(), other._16()) +operator fun EmptyTuple.plus(other: Tuple7): Tuple7 = other.copy() +operator fun Tuple7.plus(other: EmptyTuple): Tuple7 = this.copy() +operator fun Tuple7.plus(other: Tuple1): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), other._1()) +operator fun Tuple7.plus(other: Tuple2): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), other._1(), other._2()) +operator fun Tuple7.plus(other: Tuple3): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), other._1(), other._2(), other._3()) +operator fun Tuple7.plus(other: Tuple4): Tuple11 = Tuple11(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), other._1(), other._2(), other._3(), other._4()) +operator fun Tuple7.plus(other: Tuple5): Tuple12 = Tuple12(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), other._1(), other._2(), other._3(), other._4(), other._5()) +operator fun Tuple7.plus(other: Tuple6): Tuple13 = Tuple13(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6()) +operator fun Tuple7.plus(other: Tuple7): Tuple14 = Tuple14(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7()) +operator fun Tuple7.plus(other: Tuple8): Tuple15 = Tuple15(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8()) +operator fun Tuple7.plus(other: Tuple9): Tuple16 = Tuple16(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9()) +operator fun Tuple7.plus(other: Tuple10): Tuple17 = Tuple17(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10()) +operator fun Tuple7.plus(other: Tuple11): Tuple18 = Tuple18(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11()) +operator fun Tuple7.plus(other: Tuple12): Tuple19 = Tuple19(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12()) +operator fun Tuple7.plus(other: Tuple13): Tuple20 = Tuple20(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13()) +operator fun Tuple7.plus(other: Tuple14): Tuple21 = Tuple21(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14()) +operator fun Tuple7.plus(other: Tuple15): Tuple22 = Tuple22(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14(), other._15()) +operator fun EmptyTuple.plus(other: Tuple8): Tuple8 = other.copy() +operator fun Tuple8.plus(other: EmptyTuple): Tuple8 = this.copy() +operator fun Tuple8.plus(other: Tuple1): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), other._1()) +operator fun Tuple8.plus(other: Tuple2): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), other._1(), other._2()) +operator fun Tuple8.plus(other: Tuple3): Tuple11 = Tuple11(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), other._1(), other._2(), other._3()) +operator fun Tuple8.plus(other: Tuple4): Tuple12 = Tuple12(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), other._1(), other._2(), other._3(), other._4()) +operator fun Tuple8.plus(other: Tuple5): Tuple13 = Tuple13(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), other._1(), other._2(), other._3(), other._4(), other._5()) +operator fun Tuple8.plus(other: Tuple6): Tuple14 = Tuple14(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6()) +operator fun Tuple8.plus(other: Tuple7): Tuple15 = Tuple15(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7()) +operator fun Tuple8.plus(other: Tuple8): Tuple16 = Tuple16(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8()) +operator fun Tuple8.plus(other: Tuple9): Tuple17 = Tuple17(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9()) +operator fun Tuple8.plus(other: Tuple10): Tuple18 = Tuple18(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10()) +operator fun Tuple8.plus(other: Tuple11): Tuple19 = Tuple19(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11()) +operator fun Tuple8.plus(other: Tuple12): Tuple20 = Tuple20(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12()) +operator fun Tuple8.plus(other: Tuple13): Tuple21 = Tuple21(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13()) +operator fun Tuple8.plus(other: Tuple14): Tuple22 = Tuple22(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13(), other._14()) +operator fun EmptyTuple.plus(other: Tuple9): Tuple9 = other.copy() +operator fun Tuple9.plus(other: EmptyTuple): Tuple9 = this.copy() +operator fun Tuple9.plus(other: Tuple1): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), other._1()) +operator fun Tuple9.plus(other: Tuple2): Tuple11 = Tuple11(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), other._1(), other._2()) +operator fun Tuple9.plus(other: Tuple3): Tuple12 = Tuple12(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), other._1(), other._2(), other._3()) +operator fun Tuple9.plus(other: Tuple4): Tuple13 = Tuple13(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), other._1(), other._2(), other._3(), other._4()) +operator fun Tuple9.plus(other: Tuple5): Tuple14 = Tuple14(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), other._1(), other._2(), other._3(), other._4(), other._5()) +operator fun Tuple9.plus(other: Tuple6): Tuple15 = Tuple15(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6()) +operator fun Tuple9.plus(other: Tuple7): Tuple16 = Tuple16(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7()) +operator fun Tuple9.plus(other: Tuple8): Tuple17 = Tuple17(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8()) +operator fun Tuple9.plus(other: Tuple9): Tuple18 = Tuple18(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9()) +operator fun Tuple9.plus(other: Tuple10): Tuple19 = Tuple19(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10()) +operator fun Tuple9.plus(other: Tuple11): Tuple20 = Tuple20(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11()) +operator fun Tuple9.plus(other: Tuple12): Tuple21 = Tuple21(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12()) +operator fun Tuple9.plus(other: Tuple13): Tuple22 = Tuple22(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12(), other._13()) +operator fun EmptyTuple.plus(other: Tuple10): Tuple10 = other.copy() +operator fun Tuple10.plus(other: EmptyTuple): Tuple10 = this.copy() +operator fun Tuple10.plus(other: Tuple1): Tuple11 = Tuple11(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), other._1()) +operator fun Tuple10.plus(other: Tuple2): Tuple12 = Tuple12(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), other._1(), other._2()) +operator fun Tuple10.plus(other: Tuple3): Tuple13 = Tuple13(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), other._1(), other._2(), other._3()) +operator fun Tuple10.plus(other: Tuple4): Tuple14 = Tuple14(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), other._1(), other._2(), other._3(), other._4()) +operator fun Tuple10.plus(other: Tuple5): Tuple15 = Tuple15(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), other._1(), other._2(), other._3(), other._4(), other._5()) +operator fun Tuple10.plus(other: Tuple6): Tuple16 = Tuple16(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6()) +operator fun Tuple10.plus(other: Tuple7): Tuple17 = Tuple17(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7()) +operator fun Tuple10.plus(other: Tuple8): Tuple18 = Tuple18(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8()) +operator fun Tuple10.plus(other: Tuple9): Tuple19 = Tuple19(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9()) +operator fun Tuple10.plus(other: Tuple10): Tuple20 = Tuple20(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10()) +operator fun Tuple10.plus(other: Tuple11): Tuple21 = Tuple21(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11()) +operator fun Tuple10.plus(other: Tuple12): Tuple22 = Tuple22(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11(), other._12()) +operator fun EmptyTuple.plus(other: Tuple11): Tuple11 = other.copy() +operator fun Tuple11.plus(other: EmptyTuple): Tuple11 = this.copy() +operator fun Tuple11.plus(other: Tuple1): Tuple12 = Tuple12(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), other._1()) +operator fun Tuple11.plus(other: Tuple2): Tuple13 = Tuple13(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), other._1(), other._2()) +operator fun Tuple11.plus(other: Tuple3): Tuple14 = Tuple14(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), other._1(), other._2(), other._3()) +operator fun Tuple11.plus(other: Tuple4): Tuple15 = Tuple15(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), other._1(), other._2(), other._3(), other._4()) +operator fun Tuple11.plus(other: Tuple5): Tuple16 = Tuple16(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), other._1(), other._2(), other._3(), other._4(), other._5()) +operator fun Tuple11.plus(other: Tuple6): Tuple17 = Tuple17(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6()) +operator fun Tuple11.plus(other: Tuple7): Tuple18 = Tuple18(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7()) +operator fun Tuple11.plus(other: Tuple8): Tuple19 = Tuple19(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8()) +operator fun Tuple11.plus(other: Tuple9): Tuple20 = Tuple20(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9()) +operator fun Tuple11.plus(other: Tuple10): Tuple21 = Tuple21(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10()) +operator fun Tuple11.plus(other: Tuple11): Tuple22 = Tuple22(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10(), other._11()) +operator fun EmptyTuple.plus(other: Tuple12): Tuple12 = other.copy() +operator fun Tuple12.plus(other: EmptyTuple): Tuple12 = this.copy() +operator fun Tuple12.plus(other: Tuple1): 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(), other._1()) +operator fun Tuple12.plus(other: Tuple2): 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(), other._1(), other._2()) +operator fun Tuple12.plus(other: Tuple3): 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(), other._1(), other._2(), other._3()) +operator fun Tuple12.plus(other: Tuple4): 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(), other._1(), other._2(), other._3(), other._4()) +operator fun Tuple12.plus(other: Tuple5): 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(), other._1(), other._2(), other._3(), other._4(), other._5()) +operator fun Tuple12.plus(other: Tuple6): 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(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6()) +operator fun Tuple12.plus(other: Tuple7): 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(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7()) +operator fun Tuple12.plus(other: Tuple8): 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(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8()) +operator fun Tuple12.plus(other: Tuple9): 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(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9()) +operator fun Tuple12.plus(other: Tuple10): 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(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9(), other._10()) +operator fun EmptyTuple.plus(other: Tuple13): Tuple13 = other.copy() +operator fun Tuple13.plus(other: EmptyTuple): Tuple13 = this.copy() +operator fun Tuple13.plus(other: Tuple1): 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(), other._1()) +operator fun Tuple13.plus(other: Tuple2): 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(), other._1(), other._2()) +operator fun Tuple13.plus(other: Tuple3): 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(), other._1(), other._2(), other._3()) +operator fun Tuple13.plus(other: Tuple4): 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(), other._1(), other._2(), other._3(), other._4()) +operator fun Tuple13.plus(other: Tuple5): 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(), other._1(), other._2(), other._3(), other._4(), other._5()) +operator fun Tuple13.plus(other: Tuple6): 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(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6()) +operator fun Tuple13.plus(other: Tuple7): 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(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7()) +operator fun Tuple13.plus(other: Tuple8): 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(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8()) +operator fun Tuple13.plus(other: Tuple9): 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(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8(), other._9()) +operator fun EmptyTuple.plus(other: Tuple14): Tuple14 = other.copy() +operator fun Tuple14.plus(other: EmptyTuple): Tuple14 = this.copy() +operator fun Tuple14.plus(other: Tuple1): 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(), other._1()) +operator fun Tuple14.plus(other: Tuple2): 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(), other._1(), other._2()) +operator fun Tuple14.plus(other: Tuple3): 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(), other._1(), other._2(), other._3()) +operator fun Tuple14.plus(other: Tuple4): 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(), other._1(), other._2(), other._3(), other._4()) +operator fun Tuple14.plus(other: Tuple5): 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(), other._1(), other._2(), other._3(), other._4(), other._5()) +operator fun Tuple14.plus(other: Tuple6): 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(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6()) +operator fun Tuple14.plus(other: Tuple7): 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(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7()) +operator fun Tuple14.plus(other: Tuple8): 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(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7(), other._8()) +operator fun EmptyTuple.plus(other: Tuple15): Tuple15 = other.copy() +operator fun Tuple15.plus(other: EmptyTuple): Tuple15 = this.copy() +operator fun Tuple15.plus(other: Tuple1): 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(), other._1()) +operator fun Tuple15.plus(other: Tuple2): 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(), other._1(), other._2()) +operator fun Tuple15.plus(other: Tuple3): 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(), other._1(), other._2(), other._3()) +operator fun Tuple15.plus(other: Tuple4): 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(), other._1(), other._2(), other._3(), other._4()) +operator fun Tuple15.plus(other: Tuple5): 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(), other._1(), other._2(), other._3(), other._4(), other._5()) +operator fun Tuple15.plus(other: Tuple6): 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(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6()) +operator fun Tuple15.plus(other: Tuple7): 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(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6(), other._7()) +operator fun EmptyTuple.plus(other: Tuple16): Tuple16 = other.copy() +operator fun Tuple16.plus(other: EmptyTuple): Tuple16 = this.copy() +operator fun Tuple16.plus(other: Tuple1): 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(), other._1()) +operator fun Tuple16.plus(other: Tuple2): 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(), other._1(), other._2()) +operator fun Tuple16.plus(other: Tuple3): 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(), other._1(), other._2(), other._3()) +operator fun Tuple16.plus(other: Tuple4): 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(), other._1(), other._2(), other._3(), other._4()) +operator fun Tuple16.plus(other: Tuple5): 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(), other._1(), other._2(), other._3(), other._4(), other._5()) +operator fun Tuple16.plus(other: Tuple6): 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(), other._1(), other._2(), other._3(), other._4(), other._5(), other._6()) +operator fun EmptyTuple.plus(other: Tuple17): Tuple17 = other.copy() +operator fun Tuple17.plus(other: EmptyTuple): Tuple17 = this.copy() +operator fun Tuple17.plus(other: Tuple1): 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(), other._1()) +operator fun Tuple17.plus(other: Tuple2): 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(), other._1(), other._2()) +operator fun Tuple17.plus(other: Tuple3): 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(), other._1(), other._2(), other._3()) +operator fun Tuple17.plus(other: Tuple4): 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(), other._1(), other._2(), other._3(), other._4()) +operator fun Tuple17.plus(other: Tuple5): 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(), other._1(), other._2(), other._3(), other._4(), other._5()) +operator fun EmptyTuple.plus(other: Tuple18): Tuple18 = other.copy() +operator fun Tuple18.plus(other: EmptyTuple): Tuple18 = this.copy() +operator fun Tuple18.plus(other: Tuple1): 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(), other._1()) +operator fun Tuple18.plus(other: Tuple2): 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(), other._1(), other._2()) +operator fun Tuple18.plus(other: Tuple3): 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(), other._1(), other._2(), other._3()) +operator fun Tuple18.plus(other: Tuple4): 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(), other._1(), other._2(), other._3(), other._4()) +operator fun EmptyTuple.plus(other: Tuple19): Tuple19 = other.copy() +operator fun Tuple19.plus(other: EmptyTuple): Tuple19 = this.copy() +operator fun Tuple19.plus(other: Tuple1): 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(), other._1()) +operator fun Tuple19.plus(other: Tuple2): 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(), other._1(), other._2()) +operator fun Tuple19.plus(other: Tuple3): 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(), other._1(), other._2(), other._3()) +operator fun EmptyTuple.plus(other: Tuple20): Tuple20 = other.copy() +operator fun Tuple20.plus(other: EmptyTuple): Tuple20 = this.copy() +operator fun Tuple20.plus(other: Tuple1): 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(), other._1()) +operator fun Tuple20.plus(other: Tuple2): 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(), other._1(), other._2()) +operator fun EmptyTuple.plus(other: Tuple21): Tuple21 = other.copy() +operator fun Tuple21.plus(other: EmptyTuple): Tuple21 = this.copy() +operator fun Tuple21.plus(other: Tuple1): 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(), other._1()) +operator fun EmptyTuple.plus(other: Tuple22): Tuple22 = other.copy() +operator fun Tuple22.plus(other: EmptyTuple): Tuple22 = this.copy() + diff --git a/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/TupleCopy.kt b/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/TupleCopy.kt new file mode 100644 index 00000000..8d5ea7ac --- /dev/null +++ b/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/TupleCopy.kt @@ -0,0 +1,48 @@ +/*- + * =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("EXTENSION_SHADOWED_BY_MEMBER") + +package org.jetbrains.kotlinx.spark.api.tuples + +import scala.* + +fun EmptyTuple.copy(): EmptyTuple = EmptyTuple +fun Tuple1.copy(_1: T1 = this._1()): Tuple1 = Tuple1(_1) +fun Tuple2.copy(_1: T1 = this._1(), _2: T2 = this._2()): Tuple2 = Tuple2(_1, _2) +fun Tuple3.copy(_1: T1 = this._1(), _2: T2 = this._2(), _3: T3 = this._3()): Tuple3 = Tuple3(_1, _2, _3) +fun Tuple4.copy(_1: T1 = this._1(), _2: T2 = this._2(), _3: T3 = this._3(), _4: T4 = this._4()): Tuple4 = Tuple4(_1, _2, _3, _4) +fun Tuple5.copy(_1: T1 = this._1(), _2: T2 = this._2(), _3: T3 = this._3(), _4: T4 = this._4(), _5: T5 = this._5()): Tuple5 = Tuple5(_1, _2, _3, _4, _5) +fun Tuple6.copy(_1: T1 = this._1(), _2: T2 = this._2(), _3: T3 = this._3(), _4: T4 = this._4(), _5: T5 = this._5(), _6: T6 = this._6()): Tuple6 = Tuple6(_1, _2, _3, _4, _5, _6) +fun Tuple7.copy(_1: T1 = this._1(), _2: T2 = this._2(), _3: T3 = this._3(), _4: T4 = this._4(), _5: T5 = this._5(), _6: T6 = this._6(), _7: T7 = this._7()): Tuple7 = Tuple7(_1, _2, _3, _4, _5, _6, _7) +fun Tuple8.copy(_1: T1 = this._1(), _2: T2 = this._2(), _3: T3 = this._3(), _4: T4 = this._4(), _5: T5 = this._5(), _6: T6 = this._6(), _7: T7 = this._7(), _8: T8 = this._8()): Tuple8 = Tuple8(_1, _2, _3, _4, _5, _6, _7, _8) +fun Tuple9.copy(_1: T1 = this._1(), _2: T2 = this._2(), _3: T3 = this._3(), _4: T4 = this._4(), _5: T5 = this._5(), _6: T6 = this._6(), _7: T7 = this._7(), _8: T8 = this._8(), _9: T9 = this._9()): Tuple9 = Tuple9(_1, _2, _3, _4, _5, _6, _7, _8, _9) +fun Tuple10.copy(_1: T1 = this._1(), _2: T2 = this._2(), _3: T3 = this._3(), _4: T4 = this._4(), _5: T5 = this._5(), _6: T6 = this._6(), _7: T7 = this._7(), _8: T8 = this._8(), _9: T9 = this._9(), _10: T10 = this._10()): Tuple10 = Tuple10(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10) +fun Tuple11.copy(_1: T1 = this._1(), _2: T2 = this._2(), _3: T3 = this._3(), _4: T4 = this._4(), _5: T5 = this._5(), _6: T6 = this._6(), _7: T7 = this._7(), _8: T8 = this._8(), _9: T9 = this._9(), _10: T10 = this._10(), _11: T11 = this._11()): Tuple11 = Tuple11(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11) +fun Tuple12.copy(_1: T1 = this._1(), _2: T2 = this._2(), _3: T3 = this._3(), _4: T4 = this._4(), _5: T5 = this._5(), _6: T6 = this._6(), _7: T7 = this._7(), _8: T8 = this._8(), _9: T9 = this._9(), _10: T10 = this._10(), _11: T11 = this._11(), _12: T12 = this._12()): Tuple12 = Tuple12(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12) +fun Tuple13.copy(_1: T1 = this._1(), _2: T2 = this._2(), _3: T3 = this._3(), _4: T4 = this._4(), _5: T5 = this._5(), _6: T6 = this._6(), _7: T7 = this._7(), _8: T8 = this._8(), _9: T9 = this._9(), _10: T10 = this._10(), _11: T11 = this._11(), _12: T12 = this._12(), _13: T13 = this._13()): Tuple13 = Tuple13(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13) +fun Tuple14.copy(_1: T1 = this._1(), _2: T2 = this._2(), _3: T3 = this._3(), _4: T4 = this._4(), _5: T5 = this._5(), _6: T6 = this._6(), _7: T7 = this._7(), _8: T8 = this._8(), _9: T9 = this._9(), _10: T10 = this._10(), _11: T11 = this._11(), _12: T12 = this._12(), _13: T13 = this._13(), _14: T14 = this._14()): Tuple14 = Tuple14(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14) +fun Tuple15.copy(_1: T1 = this._1(), _2: T2 = this._2(), _3: T3 = this._3(), _4: T4 = this._4(), _5: T5 = this._5(), _6: T6 = this._6(), _7: T7 = this._7(), _8: T8 = this._8(), _9: T9 = this._9(), _10: T10 = this._10(), _11: T11 = this._11(), _12: T12 = this._12(), _13: T13 = this._13(), _14: T14 = this._14(), _15: T15 = this._15()): Tuple15 = Tuple15(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15) +fun Tuple16.copy(_1: T1 = this._1(), _2: T2 = this._2(), _3: T3 = this._3(), _4: T4 = this._4(), _5: T5 = this._5(), _6: T6 = this._6(), _7: T7 = this._7(), _8: T8 = this._8(), _9: T9 = this._9(), _10: T10 = this._10(), _11: T11 = this._11(), _12: T12 = this._12(), _13: T13 = this._13(), _14: T14 = this._14(), _15: T15 = this._15(), _16: T16 = this._16()): Tuple16 = Tuple16(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16) +fun Tuple17.copy(_1: T1 = this._1(), _2: T2 = this._2(), _3: T3 = this._3(), _4: T4 = this._4(), _5: T5 = this._5(), _6: T6 = this._6(), _7: T7 = this._7(), _8: T8 = this._8(), _9: T9 = this._9(), _10: T10 = this._10(), _11: T11 = this._11(), _12: T12 = this._12(), _13: T13 = this._13(), _14: T14 = this._14(), _15: T15 = this._15(), _16: T16 = this._16(), _17: T17 = this._17()): Tuple17 = Tuple17(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17) +fun Tuple18.copy(_1: T1 = this._1(), _2: T2 = this._2(), _3: T3 = this._3(), _4: T4 = this._4(), _5: T5 = this._5(), _6: T6 = this._6(), _7: T7 = this._7(), _8: T8 = this._8(), _9: T9 = this._9(), _10: T10 = this._10(), _11: T11 = this._11(), _12: T12 = this._12(), _13: T13 = this._13(), _14: T14 = this._14(), _15: T15 = this._15(), _16: T16 = this._16(), _17: T17 = this._17(), _18: T18 = this._18()): Tuple18 = Tuple18(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18) +fun Tuple19.copy(_1: T1 = this._1(), _2: T2 = this._2(), _3: T3 = this._3(), _4: T4 = this._4(), _5: T5 = this._5(), _6: T6 = this._6(), _7: T7 = this._7(), _8: T8 = this._8(), _9: T9 = this._9(), _10: T10 = this._10(), _11: T11 = this._11(), _12: T12 = this._12(), _13: T13 = this._13(), _14: T14 = this._14(), _15: T15 = this._15(), _16: T16 = this._16(), _17: T17 = this._17(), _18: T18 = this._18(), _19: T19 = this._19()): Tuple19 = Tuple19(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19) +fun Tuple20.copy(_1: T1 = this._1(), _2: T2 = this._2(), _3: T3 = this._3(), _4: T4 = this._4(), _5: T5 = this._5(), _6: T6 = this._6(), _7: T7 = this._7(), _8: T8 = this._8(), _9: T9 = this._9(), _10: T10 = this._10(), _11: T11 = this._11(), _12: T12 = this._12(), _13: T13 = this._13(), _14: T14 = this._14(), _15: T15 = this._15(), _16: T16 = this._16(), _17: T17 = this._17(), _18: T18 = this._18(), _19: T19 = this._19(), _20: T20 = this._20()): Tuple20 = Tuple20(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19, _20) +fun Tuple21.copy(_1: T1 = this._1(), _2: T2 = this._2(), _3: T3 = this._3(), _4: T4 = this._4(), _5: T5 = this._5(), _6: T6 = this._6(), _7: T7 = this._7(), _8: T8 = this._8(), _9: T9 = this._9(), _10: T10 = this._10(), _11: T11 = this._11(), _12: T12 = this._12(), _13: T13 = this._13(), _14: T14 = this._14(), _15: T15 = this._15(), _16: T16 = this._16(), _17: T17 = this._17(), _18: T18 = this._18(), _19: T19 = this._19(), _20: T20 = this._20(), _21: T21 = this._21()): Tuple21 = Tuple21(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19, _20, _21) +fun Tuple22.copy(_1: T1 = this._1(), _2: T2 = this._2(), _3: T3 = this._3(), _4: T4 = this._4(), _5: T5 = this._5(), _6: T6 = this._6(), _7: T7 = this._7(), _8: T8 = this._8(), _9: T9 = this._9(), _10: T10 = this._10(), _11: T11 = this._11(), _12: T12 = this._12(), _13: T13 = this._13(), _14: T14 = this._14(), _15: T15 = this._15(), _16: T16 = this._16(), _17: T17 = this._17(), _18: T18 = this._18(), _19: T19 = this._19(), _20: T20 = this._20(), _21: T21 = this._21(), _22: T22 = this._22()): Tuple22 = Tuple22(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, _11, _12, _13, _14, _15, _16, _17, _18, _19, _20, _21, _22) diff --git a/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/TupleDrop.kt b/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/TupleDrop.kt new file mode 100644 index 00000000..e8ed8706 --- /dev/null +++ b/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/TupleDrop.kt @@ -0,0 +1,587 @@ +/*- + * =LICENSE= + * Kotlin Spark API: Scala Tuples in Kotlin + * ---------- + * 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.tuples + +import scala.* + +/** + * This file contains all functions to drop N items from the beginning or end of a Tuple. + * If all items are dropped, the result will be [EmptyTuple]. + * + * For example: + * ```kotlin + * tupleOf(1, 2, 3, 4).drop2() == tupleOf(3, 4) + * tupleOf(1, 2, 3, 4).dropLast2() == tupleOf(1, 2) + * ``` + */ + +fun Tuple1.drop0(): Tuple1 = Tuple1(this._1()) +fun Tuple1<*>.drop1(): EmptyTuple = EmptyTuple +fun Tuple2.drop0(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple2<*, T2>.drop1(): Tuple1 = Tuple1(this._2()) +fun Tuple2<*, *>.drop2(): EmptyTuple = EmptyTuple +fun Tuple3.drop0(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple3<*, T2, T3>.drop1(): Tuple2 = Tuple2(this._2(), this._3()) +fun Tuple3<*, *, T3>.drop2(): Tuple1 = Tuple1(this._3()) +fun Tuple3<*, *, *>.drop3(): EmptyTuple = EmptyTuple +fun Tuple4.drop0(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple4<*, T2, T3, T4>.drop1(): Tuple3 = Tuple3(this._2(), this._3(), this._4()) +fun Tuple4<*, *, T3, T4>.drop2(): Tuple2 = Tuple2(this._3(), this._4()) +fun Tuple4<*, *, *, T4>.drop3(): Tuple1 = Tuple1(this._4()) +fun Tuple4<*, *, *, *>.drop4(): EmptyTuple = EmptyTuple +fun Tuple5.drop0(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple5<*, T2, T3, T4, T5>.drop1(): Tuple4 = Tuple4(this._2(), this._3(), this._4(), this._5()) +fun Tuple5<*, *, T3, T4, T5>.drop2(): Tuple3 = Tuple3(this._3(), this._4(), this._5()) +fun Tuple5<*, *, *, T4, T5>.drop3(): Tuple2 = Tuple2(this._4(), this._5()) +fun Tuple5<*, *, *, *, T5>.drop4(): Tuple1 = Tuple1(this._5()) +fun Tuple5<*, *, *, *, *>.drop5(): EmptyTuple = EmptyTuple +fun Tuple6.drop0(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple6<*, T2, T3, T4, T5, T6>.drop1(): Tuple5 = Tuple5(this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple6<*, *, T3, T4, T5, T6>.drop2(): Tuple4 = Tuple4(this._3(), this._4(), this._5(), this._6()) +fun Tuple6<*, *, *, T4, T5, T6>.drop3(): Tuple3 = Tuple3(this._4(), this._5(), this._6()) +fun Tuple6<*, *, *, *, T5, T6>.drop4(): Tuple2 = Tuple2(this._5(), this._6()) +fun Tuple6<*, *, *, *, *, T6>.drop5(): Tuple1 = Tuple1(this._6()) +fun Tuple6<*, *, *, *, *, *>.drop6(): EmptyTuple = EmptyTuple +fun Tuple7.drop0(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple7<*, T2, T3, T4, T5, T6, T7>.drop1(): Tuple6 = Tuple6(this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple7<*, *, T3, T4, T5, T6, T7>.drop2(): Tuple5 = Tuple5(this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple7<*, *, *, T4, T5, T6, T7>.drop3(): Tuple4 = Tuple4(this._4(), this._5(), this._6(), this._7()) +fun Tuple7<*, *, *, *, T5, T6, T7>.drop4(): Tuple3 = Tuple3(this._5(), this._6(), this._7()) +fun Tuple7<*, *, *, *, *, T6, T7>.drop5(): Tuple2 = Tuple2(this._6(), this._7()) +fun Tuple7<*, *, *, *, *, *, T7>.drop6(): Tuple1 = Tuple1(this._7()) +fun Tuple7<*, *, *, *, *, *, *>.drop7(): EmptyTuple = EmptyTuple +fun Tuple8.drop0(): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple8<*, T2, T3, T4, T5, T6, T7, T8>.drop1(): Tuple7 = Tuple7(this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple8<*, *, T3, T4, T5, T6, T7, T8>.drop2(): Tuple6 = Tuple6(this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple8<*, *, *, T4, T5, T6, T7, T8>.drop3(): Tuple5 = Tuple5(this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple8<*, *, *, *, T5, T6, T7, T8>.drop4(): Tuple4 = Tuple4(this._5(), this._6(), this._7(), this._8()) +fun Tuple8<*, *, *, *, *, T6, T7, T8>.drop5(): Tuple3 = Tuple3(this._6(), this._7(), this._8()) +fun Tuple8<*, *, *, *, *, *, T7, T8>.drop6(): Tuple2 = Tuple2(this._7(), this._8()) +fun Tuple8<*, *, *, *, *, *, *, T8>.drop7(): Tuple1 = Tuple1(this._8()) +fun Tuple8<*, *, *, *, *, *, *, *>.drop8(): EmptyTuple = EmptyTuple +fun Tuple9.drop0(): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple9<*, T2, T3, T4, T5, T6, T7, T8, T9>.drop1(): Tuple8 = Tuple8(this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple9<*, *, T3, T4, T5, T6, T7, T8, T9>.drop2(): Tuple7 = Tuple7(this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple9<*, *, *, T4, T5, T6, T7, T8, T9>.drop3(): Tuple6 = Tuple6(this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple9<*, *, *, *, T5, T6, T7, T8, T9>.drop4(): Tuple5 = Tuple5(this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple9<*, *, *, *, *, T6, T7, T8, T9>.drop5(): Tuple4 = Tuple4(this._6(), this._7(), this._8(), this._9()) +fun Tuple9<*, *, *, *, *, *, T7, T8, T9>.drop6(): Tuple3 = Tuple3(this._7(), this._8(), this._9()) +fun Tuple9<*, *, *, *, *, *, *, T8, T9>.drop7(): Tuple2 = Tuple2(this._8(), this._9()) +fun Tuple9<*, *, *, *, *, *, *, *, T9>.drop8(): Tuple1 = Tuple1(this._9()) +fun Tuple9<*, *, *, *, *, *, *, *, *>.drop9(): EmptyTuple = EmptyTuple +fun Tuple10.drop0(): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple10<*, T2, T3, T4, T5, T6, T7, T8, T9, T10>.drop1(): Tuple9 = Tuple9(this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple10<*, *, T3, T4, T5, T6, T7, T8, T9, T10>.drop2(): Tuple8 = Tuple8(this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple10<*, *, *, T4, T5, T6, T7, T8, T9, T10>.drop3(): Tuple7 = Tuple7(this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple10<*, *, *, *, T5, T6, T7, T8, T9, T10>.drop4(): Tuple6 = Tuple6(this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple10<*, *, *, *, *, T6, T7, T8, T9, T10>.drop5(): Tuple5 = Tuple5(this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple10<*, *, *, *, *, *, T7, T8, T9, T10>.drop6(): Tuple4 = Tuple4(this._7(), this._8(), this._9(), this._10()) +fun Tuple10<*, *, *, *, *, *, *, T8, T9, T10>.drop7(): Tuple3 = Tuple3(this._8(), this._9(), this._10()) +fun Tuple10<*, *, *, *, *, *, *, *, T9, T10>.drop8(): Tuple2 = Tuple2(this._9(), this._10()) +fun Tuple10<*, *, *, *, *, *, *, *, *, T10>.drop9(): Tuple1 = Tuple1(this._10()) +fun Tuple10<*, *, *, *, *, *, *, *, *, *>.drop10(): EmptyTuple = EmptyTuple +fun Tuple11.drop0(): 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 Tuple11<*, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11>.drop1(): Tuple10 = Tuple10(this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11()) +fun Tuple11<*, *, T3, T4, T5, T6, T7, T8, T9, T10, T11>.drop2(): Tuple9 = Tuple9(this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11()) +fun Tuple11<*, *, *, T4, T5, T6, T7, T8, T9, T10, T11>.drop3(): Tuple8 = Tuple8(this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11()) +fun Tuple11<*, *, *, *, T5, T6, T7, T8, T9, T10, T11>.drop4(): Tuple7 = Tuple7(this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11()) +fun Tuple11<*, *, *, *, *, T6, T7, T8, T9, T10, T11>.drop5(): Tuple6 = Tuple6(this._6(), this._7(), this._8(), this._9(), this._10(), this._11()) +fun Tuple11<*, *, *, *, *, *, T7, T8, T9, T10, T11>.drop6(): Tuple5 = Tuple5(this._7(), this._8(), this._9(), this._10(), this._11()) +fun Tuple11<*, *, *, *, *, *, *, T8, T9, T10, T11>.drop7(): Tuple4 = Tuple4(this._8(), this._9(), this._10(), this._11()) +fun Tuple11<*, *, *, *, *, *, *, *, T9, T10, T11>.drop8(): Tuple3 = Tuple3(this._9(), this._10(), this._11()) +fun Tuple11<*, *, *, *, *, *, *, *, *, T10, T11>.drop9(): Tuple2 = Tuple2(this._10(), this._11()) +fun Tuple11<*, *, *, *, *, *, *, *, *, *, T11>.drop10(): Tuple1 = Tuple1(this._11()) +fun Tuple11<*, *, *, *, *, *, *, *, *, *, *>.drop11(): EmptyTuple = EmptyTuple +fun Tuple12.drop0(): 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 Tuple12<*, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12>.drop1(): Tuple11 = Tuple11(this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12()) +fun Tuple12<*, *, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12>.drop2(): Tuple10 = Tuple10(this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12()) +fun Tuple12<*, *, *, T4, T5, T6, T7, T8, T9, T10, T11, T12>.drop3(): Tuple9 = Tuple9(this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12()) +fun Tuple12<*, *, *, *, T5, T6, T7, T8, T9, T10, T11, T12>.drop4(): Tuple8 = Tuple8(this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12()) +fun Tuple12<*, *, *, *, *, T6, T7, T8, T9, T10, T11, T12>.drop5(): Tuple7 = Tuple7(this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12()) +fun Tuple12<*, *, *, *, *, *, T7, T8, T9, T10, T11, T12>.drop6(): Tuple6 = Tuple6(this._7(), this._8(), this._9(), this._10(), this._11(), this._12()) +fun Tuple12<*, *, *, *, *, *, *, T8, T9, T10, T11, T12>.drop7(): Tuple5 = Tuple5(this._8(), this._9(), this._10(), this._11(), this._12()) +fun Tuple12<*, *, *, *, *, *, *, *, T9, T10, T11, T12>.drop8(): Tuple4 = Tuple4(this._9(), this._10(), this._11(), this._12()) +fun Tuple12<*, *, *, *, *, *, *, *, *, T10, T11, T12>.drop9(): Tuple3 = Tuple3(this._10(), this._11(), this._12()) +fun Tuple12<*, *, *, *, *, *, *, *, *, *, T11, T12>.drop10(): Tuple2 = Tuple2(this._11(), this._12()) +fun Tuple12<*, *, *, *, *, *, *, *, *, *, *, T12>.drop11(): Tuple1 = Tuple1(this._12()) +fun Tuple12<*, *, *, *, *, *, *, *, *, *, *, *>.drop12(): EmptyTuple = EmptyTuple +fun Tuple13.drop0(): 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 Tuple13<*, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>.drop1(): Tuple12 = Tuple12(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<*, *, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>.drop2(): Tuple11 = Tuple11(this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13()) +fun Tuple13<*, *, *, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>.drop3(): Tuple10 = Tuple10(this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13()) +fun Tuple13<*, *, *, *, T5, T6, T7, T8, T9, T10, T11, T12, T13>.drop4(): Tuple9 = Tuple9(this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13()) +fun Tuple13<*, *, *, *, *, T6, T7, T8, T9, T10, T11, T12, T13>.drop5(): Tuple8 = Tuple8(this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13()) +fun Tuple13<*, *, *, *, *, *, T7, T8, T9, T10, T11, T12, T13>.drop6(): Tuple7 = Tuple7(this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13()) +fun Tuple13<*, *, *, *, *, *, *, T8, T9, T10, T11, T12, T13>.drop7(): Tuple6 = Tuple6(this._8(), this._9(), this._10(), this._11(), this._12(), this._13()) +fun Tuple13<*, *, *, *, *, *, *, *, T9, T10, T11, T12, T13>.drop8(): Tuple5 = Tuple5(this._9(), this._10(), this._11(), this._12(), this._13()) +fun Tuple13<*, *, *, *, *, *, *, *, *, T10, T11, T12, T13>.drop9(): Tuple4 = Tuple4(this._10(), this._11(), this._12(), this._13()) +fun Tuple13<*, *, *, *, *, *, *, *, *, *, T11, T12, T13>.drop10(): Tuple3 = Tuple3(this._11(), this._12(), this._13()) +fun Tuple13<*, *, *, *, *, *, *, *, *, *, *, T12, T13>.drop11(): Tuple2 = Tuple2(this._12(), this._13()) +fun Tuple13<*, *, *, *, *, *, *, *, *, *, *, *, T13>.drop12(): Tuple1 = Tuple1(this._13()) +fun Tuple13<*, *, *, *, *, *, *, *, *, *, *, *, *>.drop13(): EmptyTuple = EmptyTuple +fun Tuple14.drop0(): 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 Tuple14<*, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>.drop1(): Tuple13 = Tuple13(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<*, *, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>.drop2(): Tuple12 = Tuple12(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<*, *, *, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>.drop3(): Tuple11 = Tuple11(this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14()) +fun Tuple14<*, *, *, *, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>.drop4(): Tuple10 = Tuple10(this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14()) +fun Tuple14<*, *, *, *, *, T6, T7, T8, T9, T10, T11, T12, T13, T14>.drop5(): Tuple9 = Tuple9(this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14()) +fun Tuple14<*, *, *, *, *, *, T7, T8, T9, T10, T11, T12, T13, T14>.drop6(): Tuple8 = Tuple8(this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14()) +fun Tuple14<*, *, *, *, *, *, *, T8, T9, T10, T11, T12, T13, T14>.drop7(): Tuple7 = Tuple7(this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14()) +fun Tuple14<*, *, *, *, *, *, *, *, T9, T10, T11, T12, T13, T14>.drop8(): Tuple6 = Tuple6(this._9(), this._10(), this._11(), this._12(), this._13(), this._14()) +fun Tuple14<*, *, *, *, *, *, *, *, *, T10, T11, T12, T13, T14>.drop9(): Tuple5 = Tuple5(this._10(), this._11(), this._12(), this._13(), this._14()) +fun Tuple14<*, *, *, *, *, *, *, *, *, *, T11, T12, T13, T14>.drop10(): Tuple4 = Tuple4(this._11(), this._12(), this._13(), this._14()) +fun Tuple14<*, *, *, *, *, *, *, *, *, *, *, T12, T13, T14>.drop11(): Tuple3 = Tuple3(this._12(), this._13(), this._14()) +fun Tuple14<*, *, *, *, *, *, *, *, *, *, *, *, T13, T14>.drop12(): Tuple2 = Tuple2(this._13(), this._14()) +fun Tuple14<*, *, *, *, *, *, *, *, *, *, *, *, *, T14>.drop13(): Tuple1 = Tuple1(this._14()) +fun Tuple14<*, *, *, *, *, *, *, *, *, *, *, *, *, *>.drop14(): EmptyTuple = EmptyTuple +fun Tuple15.drop0(): 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 Tuple15<*, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>.drop1(): Tuple14 = Tuple14(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<*, *, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>.drop2(): Tuple13 = Tuple13(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<*, *, *, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>.drop3(): Tuple12 = Tuple12(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<*, *, *, *, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>.drop4(): Tuple11 = Tuple11(this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15()) +fun Tuple15<*, *, *, *, *, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>.drop5(): Tuple10 = Tuple10(this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15()) +fun Tuple15<*, *, *, *, *, *, T7, T8, T9, T10, T11, T12, T13, T14, T15>.drop6(): Tuple9 = Tuple9(this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15()) +fun Tuple15<*, *, *, *, *, *, *, T8, T9, T10, T11, T12, T13, T14, T15>.drop7(): Tuple8 = Tuple8(this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15()) +fun Tuple15<*, *, *, *, *, *, *, *, T9, T10, T11, T12, T13, T14, T15>.drop8(): Tuple7 = Tuple7(this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15()) +fun Tuple15<*, *, *, *, *, *, *, *, *, T10, T11, T12, T13, T14, T15>.drop9(): Tuple6 = Tuple6(this._10(), this._11(), this._12(), this._13(), this._14(), this._15()) +fun Tuple15<*, *, *, *, *, *, *, *, *, *, T11, T12, T13, T14, T15>.drop10(): Tuple5 = Tuple5(this._11(), this._12(), this._13(), this._14(), this._15()) +fun Tuple15<*, *, *, *, *, *, *, *, *, *, *, T12, T13, T14, T15>.drop11(): Tuple4 = Tuple4(this._12(), this._13(), this._14(), this._15()) +fun Tuple15<*, *, *, *, *, *, *, *, *, *, *, *, T13, T14, T15>.drop12(): Tuple3 = Tuple3(this._13(), this._14(), this._15()) +fun Tuple15<*, *, *, *, *, *, *, *, *, *, *, *, *, T14, T15>.drop13(): Tuple2 = Tuple2(this._14(), this._15()) +fun Tuple15<*, *, *, *, *, *, *, *, *, *, *, *, *, *, T15>.drop14(): Tuple1 = Tuple1(this._15()) +fun Tuple15<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.drop15(): EmptyTuple = EmptyTuple +fun Tuple16.drop0(): 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 Tuple16<*, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>.drop1(): Tuple15 = Tuple15(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<*, *, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>.drop2(): Tuple14 = Tuple14(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<*, *, *, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>.drop3(): Tuple13 = Tuple13(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<*, *, *, *, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>.drop4(): Tuple12 = Tuple12(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<*, *, *, *, *, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>.drop5(): Tuple11 = Tuple11(this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16()) +fun Tuple16<*, *, *, *, *, *, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>.drop6(): Tuple10 = Tuple10(this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16()) +fun Tuple16<*, *, *, *, *, *, *, T8, T9, T10, T11, T12, T13, T14, T15, T16>.drop7(): Tuple9 = Tuple9(this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16()) +fun Tuple16<*, *, *, *, *, *, *, *, T9, T10, T11, T12, T13, T14, T15, T16>.drop8(): Tuple8 = Tuple8(this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16()) +fun Tuple16<*, *, *, *, *, *, *, *, *, T10, T11, T12, T13, T14, T15, T16>.drop9(): Tuple7 = Tuple7(this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16()) +fun Tuple16<*, *, *, *, *, *, *, *, *, *, T11, T12, T13, T14, T15, T16>.drop10(): Tuple6 = Tuple6(this._11(), this._12(), this._13(), this._14(), this._15(), this._16()) +fun Tuple16<*, *, *, *, *, *, *, *, *, *, *, T12, T13, T14, T15, T16>.drop11(): Tuple5 = Tuple5(this._12(), this._13(), this._14(), this._15(), this._16()) +fun Tuple16<*, *, *, *, *, *, *, *, *, *, *, *, T13, T14, T15, T16>.drop12(): Tuple4 = Tuple4(this._13(), this._14(), this._15(), this._16()) +fun Tuple16<*, *, *, *, *, *, *, *, *, *, *, *, *, T14, T15, T16>.drop13(): Tuple3 = Tuple3(this._14(), this._15(), this._16()) +fun Tuple16<*, *, *, *, *, *, *, *, *, *, *, *, *, *, T15, T16>.drop14(): Tuple2 = Tuple2(this._15(), this._16()) +fun Tuple16<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T16>.drop15(): Tuple1 = Tuple1(this._16()) +fun Tuple16<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.drop16(): EmptyTuple = EmptyTuple +fun Tuple17.drop0(): 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 Tuple17<*, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>.drop1(): Tuple16 = Tuple16(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<*, *, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>.drop2(): Tuple15 = Tuple15(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<*, *, *, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>.drop3(): Tuple14 = Tuple14(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<*, *, *, *, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>.drop4(): Tuple13 = Tuple13(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<*, *, *, *, *, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>.drop5(): Tuple12 = Tuple12(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<*, *, *, *, *, *, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>.drop6(): Tuple11 = Tuple11(this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17()) +fun Tuple17<*, *, *, *, *, *, *, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>.drop7(): Tuple10 = Tuple10(this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17()) +fun Tuple17<*, *, *, *, *, *, *, *, T9, T10, T11, T12, T13, T14, T15, T16, T17>.drop8(): Tuple9 = Tuple9(this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17()) +fun Tuple17<*, *, *, *, *, *, *, *, *, T10, T11, T12, T13, T14, T15, T16, T17>.drop9(): Tuple8 = Tuple8(this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17()) +fun Tuple17<*, *, *, *, *, *, *, *, *, *, T11, T12, T13, T14, T15, T16, T17>.drop10(): Tuple7 = Tuple7(this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17()) +fun Tuple17<*, *, *, *, *, *, *, *, *, *, *, T12, T13, T14, T15, T16, T17>.drop11(): Tuple6 = Tuple6(this._12(), this._13(), this._14(), this._15(), this._16(), this._17()) +fun Tuple17<*, *, *, *, *, *, *, *, *, *, *, *, T13, T14, T15, T16, T17>.drop12(): Tuple5 = Tuple5(this._13(), this._14(), this._15(), this._16(), this._17()) +fun Tuple17<*, *, *, *, *, *, *, *, *, *, *, *, *, T14, T15, T16, T17>.drop13(): Tuple4 = Tuple4(this._14(), this._15(), this._16(), this._17()) +fun Tuple17<*, *, *, *, *, *, *, *, *, *, *, *, *, *, T15, T16, T17>.drop14(): Tuple3 = Tuple3(this._15(), this._16(), this._17()) +fun Tuple17<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T16, T17>.drop15(): Tuple2 = Tuple2(this._16(), this._17()) +fun Tuple17<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T17>.drop16(): Tuple1 = Tuple1(this._17()) +fun Tuple17<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.drop17(): EmptyTuple = EmptyTuple +fun Tuple18.drop0(): 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 Tuple18<*, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>.drop1(): Tuple17 = Tuple17(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<*, *, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>.drop2(): Tuple16 = Tuple16(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<*, *, *, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>.drop3(): Tuple15 = Tuple15(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<*, *, *, *, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>.drop4(): Tuple14 = Tuple14(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<*, *, *, *, *, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>.drop5(): Tuple13 = Tuple13(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<*, *, *, *, *, *, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>.drop6(): Tuple12 = Tuple12(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<*, *, *, *, *, *, *, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>.drop7(): Tuple11 = Tuple11(this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18()) +fun Tuple18<*, *, *, *, *, *, *, *, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>.drop8(): Tuple10 = Tuple10(this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18()) +fun Tuple18<*, *, *, *, *, *, *, *, *, T10, T11, T12, T13, T14, T15, T16, T17, T18>.drop9(): Tuple9 = Tuple9(this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18()) +fun Tuple18<*, *, *, *, *, *, *, *, *, *, T11, T12, T13, T14, T15, T16, T17, T18>.drop10(): Tuple8 = Tuple8(this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18()) +fun Tuple18<*, *, *, *, *, *, *, *, *, *, *, T12, T13, T14, T15, T16, T17, T18>.drop11(): Tuple7 = Tuple7(this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18()) +fun Tuple18<*, *, *, *, *, *, *, *, *, *, *, *, T13, T14, T15, T16, T17, T18>.drop12(): Tuple6 = Tuple6(this._13(), this._14(), this._15(), this._16(), this._17(), this._18()) +fun Tuple18<*, *, *, *, *, *, *, *, *, *, *, *, *, T14, T15, T16, T17, T18>.drop13(): Tuple5 = Tuple5(this._14(), this._15(), this._16(), this._17(), this._18()) +fun Tuple18<*, *, *, *, *, *, *, *, *, *, *, *, *, *, T15, T16, T17, T18>.drop14(): Tuple4 = Tuple4(this._15(), this._16(), this._17(), this._18()) +fun Tuple18<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T16, T17, T18>.drop15(): Tuple3 = Tuple3(this._16(), this._17(), this._18()) +fun Tuple18<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T17, T18>.drop16(): Tuple2 = Tuple2(this._17(), this._18()) +fun Tuple18<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T18>.drop17(): Tuple1 = Tuple1(this._18()) +fun Tuple18<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.drop18(): EmptyTuple = EmptyTuple +fun Tuple19.drop0(): 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 Tuple19<*, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>.drop1(): Tuple18 = Tuple18(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<*, *, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>.drop2(): Tuple17 = Tuple17(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<*, *, *, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>.drop3(): Tuple16 = Tuple16(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<*, *, *, *, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>.drop4(): Tuple15 = Tuple15(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<*, *, *, *, *, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>.drop5(): Tuple14 = Tuple14(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<*, *, *, *, *, *, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>.drop6(): Tuple13 = Tuple13(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<*, *, *, *, *, *, *, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>.drop7(): Tuple12 = Tuple12(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<*, *, *, *, *, *, *, *, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>.drop8(): Tuple11 = Tuple11(this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19()) +fun Tuple19<*, *, *, *, *, *, *, *, *, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>.drop9(): Tuple10 = Tuple10(this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19()) +fun Tuple19<*, *, *, *, *, *, *, *, *, *, T11, T12, T13, T14, T15, T16, T17, T18, T19>.drop10(): Tuple9 = Tuple9(this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19()) +fun Tuple19<*, *, *, *, *, *, *, *, *, *, *, T12, T13, T14, T15, T16, T17, T18, T19>.drop11(): Tuple8 = Tuple8(this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19()) +fun Tuple19<*, *, *, *, *, *, *, *, *, *, *, *, T13, T14, T15, T16, T17, T18, T19>.drop12(): Tuple7 = Tuple7(this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19()) +fun Tuple19<*, *, *, *, *, *, *, *, *, *, *, *, *, T14, T15, T16, T17, T18, T19>.drop13(): Tuple6 = Tuple6(this._14(), this._15(), this._16(), this._17(), this._18(), this._19()) +fun Tuple19<*, *, *, *, *, *, *, *, *, *, *, *, *, *, T15, T16, T17, T18, T19>.drop14(): Tuple5 = Tuple5(this._15(), this._16(), this._17(), this._18(), this._19()) +fun Tuple19<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T16, T17, T18, T19>.drop15(): Tuple4 = Tuple4(this._16(), this._17(), this._18(), this._19()) +fun Tuple19<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T17, T18, T19>.drop16(): Tuple3 = Tuple3(this._17(), this._18(), this._19()) +fun Tuple19<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T18, T19>.drop17(): Tuple2 = Tuple2(this._18(), this._19()) +fun Tuple19<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T19>.drop18(): Tuple1 = Tuple1(this._19()) +fun Tuple19<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.drop19(): EmptyTuple = EmptyTuple +fun Tuple20.drop0(): 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 Tuple20<*, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>.drop1(): Tuple19 = Tuple19(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<*, *, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>.drop2(): Tuple18 = Tuple18(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<*, *, *, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>.drop3(): Tuple17 = Tuple17(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<*, *, *, *, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>.drop4(): Tuple16 = Tuple16(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<*, *, *, *, *, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>.drop5(): Tuple15 = Tuple15(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<*, *, *, *, *, *, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>.drop6(): Tuple14 = Tuple14(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<*, *, *, *, *, *, *, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>.drop7(): Tuple13 = Tuple13(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<*, *, *, *, *, *, *, *, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>.drop8(): Tuple12 = Tuple12(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<*, *, *, *, *, *, *, *, *, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>.drop9(): Tuple11 = Tuple11(this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20()) +fun Tuple20<*, *, *, *, *, *, *, *, *, *, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>.drop10(): Tuple10 = Tuple10(this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20()) +fun Tuple20<*, *, *, *, *, *, *, *, *, *, *, T12, T13, T14, T15, T16, T17, T18, T19, T20>.drop11(): Tuple9 = Tuple9(this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20()) +fun Tuple20<*, *, *, *, *, *, *, *, *, *, *, *, T13, T14, T15, T16, T17, T18, T19, T20>.drop12(): Tuple8 = Tuple8(this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20()) +fun Tuple20<*, *, *, *, *, *, *, *, *, *, *, *, *, T14, T15, T16, T17, T18, T19, T20>.drop13(): Tuple7 = Tuple7(this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20()) +fun Tuple20<*, *, *, *, *, *, *, *, *, *, *, *, *, *, T15, T16, T17, T18, T19, T20>.drop14(): Tuple6 = Tuple6(this._15(), this._16(), this._17(), this._18(), this._19(), this._20()) +fun Tuple20<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T16, T17, T18, T19, T20>.drop15(): Tuple5 = Tuple5(this._16(), this._17(), this._18(), this._19(), this._20()) +fun Tuple20<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T17, T18, T19, T20>.drop16(): Tuple4 = Tuple4(this._17(), this._18(), this._19(), this._20()) +fun Tuple20<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T18, T19, T20>.drop17(): Tuple3 = Tuple3(this._18(), this._19(), this._20()) +fun Tuple20<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T19, T20>.drop18(): Tuple2 = Tuple2(this._19(), this._20()) +fun Tuple20<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T20>.drop19(): Tuple1 = Tuple1(this._20()) +fun Tuple20<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.drop20(): EmptyTuple = EmptyTuple +fun Tuple21.drop0(): 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 Tuple21<*, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>.drop1(): Tuple20 = Tuple20(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<*, *, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>.drop2(): Tuple19 = Tuple19(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<*, *, *, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>.drop3(): Tuple18 = Tuple18(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<*, *, *, *, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>.drop4(): Tuple17 = Tuple17(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<*, *, *, *, *, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>.drop5(): Tuple16 = Tuple16(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<*, *, *, *, *, *, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>.drop6(): Tuple15 = Tuple15(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<*, *, *, *, *, *, *, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>.drop7(): Tuple14 = Tuple14(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<*, *, *, *, *, *, *, *, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>.drop8(): Tuple13 = Tuple13(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<*, *, *, *, *, *, *, *, *, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>.drop9(): Tuple12 = Tuple12(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<*, *, *, *, *, *, *, *, *, *, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>.drop10(): Tuple11 = Tuple11(this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20(), this._21()) +fun Tuple21<*, *, *, *, *, *, *, *, *, *, *, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>.drop11(): Tuple10 = Tuple10(this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20(), this._21()) +fun Tuple21<*, *, *, *, *, *, *, *, *, *, *, *, T13, T14, T15, T16, T17, T18, T19, T20, T21>.drop12(): Tuple9 = Tuple9(this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20(), this._21()) +fun Tuple21<*, *, *, *, *, *, *, *, *, *, *, *, *, T14, T15, T16, T17, T18, T19, T20, T21>.drop13(): Tuple8 = Tuple8(this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20(), this._21()) +fun Tuple21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, T15, T16, T17, T18, T19, T20, T21>.drop14(): Tuple7 = Tuple7(this._15(), this._16(), this._17(), this._18(), this._19(), this._20(), this._21()) +fun Tuple21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T16, T17, T18, T19, T20, T21>.drop15(): Tuple6 = Tuple6(this._16(), this._17(), this._18(), this._19(), this._20(), this._21()) +fun Tuple21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T17, T18, T19, T20, T21>.drop16(): Tuple5 = Tuple5(this._17(), this._18(), this._19(), this._20(), this._21()) +fun Tuple21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T18, T19, T20, T21>.drop17(): Tuple4 = Tuple4(this._18(), this._19(), this._20(), this._21()) +fun Tuple21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T19, T20, T21>.drop18(): Tuple3 = Tuple3(this._19(), this._20(), this._21()) +fun Tuple21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T20, T21>.drop19(): Tuple2 = Tuple2(this._20(), this._21()) +fun Tuple21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T21>.drop20(): Tuple1 = Tuple1(this._21()) +fun Tuple21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.drop21(): EmptyTuple = EmptyTuple +fun Tuple22.drop0(): 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 Tuple22<*, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>.drop1(): Tuple21 = Tuple21(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<*, *, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>.drop2(): Tuple20 = Tuple20(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<*, *, *, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>.drop3(): Tuple19 = Tuple19(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<*, *, *, *, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>.drop4(): Tuple18 = Tuple18(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<*, *, *, *, *, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>.drop5(): Tuple17 = Tuple17(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<*, *, *, *, *, *, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>.drop6(): Tuple16 = Tuple16(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<*, *, *, *, *, *, *, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>.drop7(): Tuple15 = Tuple15(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<*, *, *, *, *, *, *, *, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>.drop8(): Tuple14 = Tuple14(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<*, *, *, *, *, *, *, *, *, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>.drop9(): Tuple13 = Tuple13(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<*, *, *, *, *, *, *, *, *, *, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>.drop10(): Tuple12 = Tuple12(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<*, *, *, *, *, *, *, *, *, *, *, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>.drop11(): Tuple11 = Tuple11(this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20(), this._21(), this._22()) +fun Tuple22<*, *, *, *, *, *, *, *, *, *, *, *, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>.drop12(): Tuple10 = Tuple10(this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20(), this._21(), this._22()) +fun Tuple22<*, *, *, *, *, *, *, *, *, *, *, *, *, T14, T15, T16, T17, T18, T19, T20, T21, T22>.drop13(): Tuple9 = Tuple9(this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20(), this._21(), this._22()) +fun Tuple22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, T15, T16, T17, T18, T19, T20, T21, T22>.drop14(): Tuple8 = Tuple8(this._15(), this._16(), this._17(), this._18(), this._19(), this._20(), this._21(), this._22()) +fun Tuple22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T16, T17, T18, T19, T20, T21, T22>.drop15(): Tuple7 = Tuple7(this._16(), this._17(), this._18(), this._19(), this._20(), this._21(), this._22()) +fun Tuple22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T17, T18, T19, T20, T21, T22>.drop16(): Tuple6 = Tuple6(this._17(), this._18(), this._19(), this._20(), this._21(), this._22()) +fun Tuple22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T18, T19, T20, T21, T22>.drop17(): Tuple5 = Tuple5(this._18(), this._19(), this._20(), this._21(), this._22()) +fun Tuple22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T19, T20, T21, T22>.drop18(): Tuple4 = Tuple4(this._19(), this._20(), this._21(), this._22()) +fun Tuple22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T20, T21, T22>.drop19(): Tuple3 = Tuple3(this._20(), this._21(), this._22()) +fun Tuple22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T21, T22>.drop20(): Tuple2 = Tuple2(this._21(), this._22()) +fun Tuple22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T22>.drop21(): Tuple1 = Tuple1(this._22()) +fun Tuple22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.drop22(): EmptyTuple = EmptyTuple + + + +fun Tuple1.dropLast0(): Tuple1 = Tuple1(this._1()) +fun Tuple1<*>.dropLast1(): EmptyTuple = EmptyTuple +fun Tuple2.dropLast0(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple2.dropLast1(): Tuple1 = Tuple1(this._1()) +fun Tuple2<*, *>.dropLast2(): EmptyTuple = EmptyTuple +fun Tuple3.dropLast0(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple3.dropLast1(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple3.dropLast2(): Tuple1 = Tuple1(this._1()) +fun Tuple3<*, *, *>.dropLast3(): EmptyTuple = EmptyTuple +fun Tuple4.dropLast0(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple4.dropLast1(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple4.dropLast2(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple4.dropLast3(): Tuple1 = Tuple1(this._1()) +fun Tuple4<*, *, *, *>.dropLast4(): EmptyTuple = EmptyTuple +fun Tuple5.dropLast0(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple5.dropLast1(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple5.dropLast2(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple5.dropLast3(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple5.dropLast4(): Tuple1 = Tuple1(this._1()) +fun Tuple5<*, *, *, *, *>.dropLast5(): EmptyTuple = EmptyTuple +fun Tuple6.dropLast0(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple6.dropLast1(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple6.dropLast2(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple6.dropLast3(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple6.dropLast4(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple6.dropLast5(): Tuple1 = Tuple1(this._1()) +fun Tuple6<*, *, *, *, *, *>.dropLast6(): EmptyTuple = EmptyTuple +fun Tuple7.dropLast0(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple7.dropLast1(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple7.dropLast2(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple7.dropLast3(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple7.dropLast4(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple7.dropLast5(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple7.dropLast6(): Tuple1 = Tuple1(this._1()) +fun Tuple7<*, *, *, *, *, *, *>.dropLast7(): EmptyTuple = EmptyTuple +fun Tuple8.dropLast0(): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple8.dropLast1(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple8.dropLast2(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple8.dropLast3(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple8.dropLast4(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple8.dropLast5(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple8.dropLast6(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple8.dropLast7(): Tuple1 = Tuple1(this._1()) +fun Tuple8<*, *, *, *, *, *, *, *>.dropLast8(): EmptyTuple = EmptyTuple +fun Tuple9.dropLast0(): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple9.dropLast1(): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple9.dropLast2(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple9.dropLast3(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple9.dropLast4(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple9.dropLast5(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple9.dropLast6(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple9.dropLast7(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple9.dropLast8(): Tuple1 = Tuple1(this._1()) +fun Tuple9<*, *, *, *, *, *, *, *, *>.dropLast9(): EmptyTuple = EmptyTuple +fun Tuple10.dropLast0(): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple10.dropLast1(): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple10.dropLast2(): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple10.dropLast3(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple10.dropLast4(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple10.dropLast5(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple10.dropLast6(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple10.dropLast7(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple10.dropLast8(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple10.dropLast9(): Tuple1 = Tuple1(this._1()) +fun Tuple10<*, *, *, *, *, *, *, *, *, *>.dropLast10(): EmptyTuple = EmptyTuple +fun Tuple11.dropLast0(): 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 Tuple11.dropLast1(): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple11.dropLast2(): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple11.dropLast3(): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple11.dropLast4(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple11.dropLast5(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple11.dropLast6(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple11.dropLast7(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple11.dropLast8(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple11.dropLast9(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple11.dropLast10(): Tuple1 = Tuple1(this._1()) +fun Tuple11<*, *, *, *, *, *, *, *, *, *, *>.dropLast11(): EmptyTuple = EmptyTuple +fun Tuple12.dropLast0(): 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 Tuple12.dropLast1(): 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 Tuple12.dropLast2(): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple12.dropLast3(): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple12.dropLast4(): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple12.dropLast5(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple12.dropLast6(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple12.dropLast7(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple12.dropLast8(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple12.dropLast9(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple12.dropLast10(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple12.dropLast11(): Tuple1 = Tuple1(this._1()) +fun Tuple12<*, *, *, *, *, *, *, *, *, *, *, *>.dropLast12(): EmptyTuple = EmptyTuple +fun Tuple13.dropLast0(): 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 Tuple13.dropLast1(): 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 Tuple13.dropLast2(): 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 Tuple13.dropLast3(): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple13.dropLast4(): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple13.dropLast5(): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple13.dropLast6(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple13.dropLast7(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple13.dropLast8(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple13.dropLast9(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple13.dropLast10(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple13.dropLast11(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple13.dropLast12(): Tuple1 = Tuple1(this._1()) +fun Tuple13<*, *, *, *, *, *, *, *, *, *, *, *, *>.dropLast13(): EmptyTuple = EmptyTuple +fun Tuple14.dropLast0(): 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 Tuple14.dropLast1(): 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 Tuple14.dropLast2(): 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 Tuple14.dropLast3(): 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 Tuple14.dropLast4(): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple14.dropLast5(): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple14.dropLast6(): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple14.dropLast7(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple14.dropLast8(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple14.dropLast9(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple14.dropLast10(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple14.dropLast11(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple14.dropLast12(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple14.dropLast13(): Tuple1 = Tuple1(this._1()) +fun Tuple14<*, *, *, *, *, *, *, *, *, *, *, *, *, *>.dropLast14(): EmptyTuple = EmptyTuple +fun Tuple15.dropLast0(): 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 Tuple15.dropLast1(): 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 Tuple15.dropLast2(): 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 Tuple15.dropLast3(): 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 Tuple15.dropLast4(): 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 Tuple15.dropLast5(): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple15.dropLast6(): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple15.dropLast7(): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple15.dropLast8(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple15.dropLast9(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple15.dropLast10(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple15.dropLast11(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple15.dropLast12(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple15.dropLast13(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple15.dropLast14(): Tuple1 = Tuple1(this._1()) +fun Tuple15<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.dropLast15(): EmptyTuple = EmptyTuple +fun Tuple16.dropLast0(): 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 Tuple16.dropLast1(): 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 Tuple16.dropLast2(): 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 Tuple16.dropLast3(): 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 Tuple16.dropLast4(): 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 Tuple16.dropLast5(): 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 Tuple16.dropLast6(): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple16.dropLast7(): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple16.dropLast8(): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple16.dropLast9(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple16.dropLast10(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple16.dropLast11(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple16.dropLast12(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple16.dropLast13(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple16.dropLast14(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple16.dropLast15(): Tuple1 = Tuple1(this._1()) +fun Tuple16<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.dropLast16(): EmptyTuple = EmptyTuple +fun Tuple17.dropLast0(): 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 Tuple17.dropLast1(): 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 Tuple17.dropLast2(): 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 Tuple17.dropLast3(): 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 Tuple17.dropLast4(): 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 Tuple17.dropLast5(): 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 Tuple17.dropLast6(): 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 Tuple17.dropLast7(): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple17.dropLast8(): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple17.dropLast9(): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple17.dropLast10(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple17.dropLast11(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple17.dropLast12(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple17.dropLast13(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple17.dropLast14(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple17.dropLast15(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple17.dropLast16(): Tuple1 = Tuple1(this._1()) +fun Tuple17<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.dropLast17(): EmptyTuple = EmptyTuple +fun Tuple18.dropLast0(): 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 Tuple18.dropLast1(): 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 Tuple18.dropLast2(): 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 Tuple18.dropLast3(): 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 Tuple18.dropLast4(): 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 Tuple18.dropLast5(): 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 Tuple18.dropLast6(): 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 Tuple18.dropLast7(): 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 Tuple18.dropLast8(): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple18.dropLast9(): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple18.dropLast10(): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple18.dropLast11(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple18.dropLast12(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple18.dropLast13(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple18.dropLast14(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple18.dropLast15(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple18.dropLast16(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple18.dropLast17(): Tuple1 = Tuple1(this._1()) +fun Tuple18<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.dropLast18(): EmptyTuple = EmptyTuple +fun Tuple19.dropLast0(): 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 Tuple19.dropLast1(): 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 Tuple19.dropLast2(): 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 Tuple19.dropLast3(): 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 Tuple19.dropLast4(): 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 Tuple19.dropLast5(): 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 Tuple19.dropLast6(): 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 Tuple19.dropLast7(): 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 Tuple19.dropLast8(): 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 Tuple19.dropLast9(): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple19.dropLast10(): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple19.dropLast11(): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple19.dropLast12(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple19.dropLast13(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple19.dropLast14(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple19.dropLast15(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple19.dropLast16(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple19.dropLast17(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple19.dropLast18(): Tuple1 = Tuple1(this._1()) +fun Tuple19<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.dropLast19(): EmptyTuple = EmptyTuple +fun Tuple20.dropLast0(): 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 Tuple20.dropLast1(): 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 Tuple20.dropLast2(): 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 Tuple20.dropLast3(): 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 Tuple20.dropLast4(): 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 Tuple20.dropLast5(): 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 Tuple20.dropLast6(): 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 Tuple20.dropLast7(): 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 Tuple20.dropLast8(): 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 Tuple20.dropLast9(): 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 Tuple20.dropLast10(): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple20.dropLast11(): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple20.dropLast12(): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple20.dropLast13(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple20.dropLast14(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple20.dropLast15(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple20.dropLast16(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple20.dropLast17(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple20.dropLast18(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple20.dropLast19(): Tuple1 = Tuple1(this._1()) +fun Tuple20<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.dropLast20(): EmptyTuple = EmptyTuple +fun Tuple21.dropLast0(): 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 Tuple21.dropLast1(): 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 Tuple21.dropLast2(): 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 Tuple21.dropLast3(): 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 Tuple21.dropLast4(): 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 Tuple21.dropLast5(): 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 Tuple21.dropLast6(): 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 Tuple21.dropLast7(): 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 Tuple21.dropLast8(): 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 Tuple21.dropLast9(): 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 Tuple21.dropLast10(): 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 Tuple21.dropLast11(): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple21.dropLast12(): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple21.dropLast13(): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple21.dropLast14(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple21.dropLast15(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple21.dropLast16(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple21.dropLast17(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple21.dropLast18(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple21.dropLast19(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple21.dropLast20(): Tuple1 = Tuple1(this._1()) +fun Tuple21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.dropLast21(): EmptyTuple = EmptyTuple +fun Tuple22.dropLast0(): 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 Tuple22.dropLast1(): 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 Tuple22.dropLast2(): 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 Tuple22.dropLast3(): 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 Tuple22.dropLast4(): 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 Tuple22.dropLast5(): 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 Tuple22.dropLast6(): 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 Tuple22.dropLast7(): 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 Tuple22.dropLast8(): 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 Tuple22.dropLast9(): 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 Tuple22.dropLast10(): 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 Tuple22.dropLast11(): 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 Tuple22.dropLast12(): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple22.dropLast13(): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple22.dropLast14(): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple22.dropLast15(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple22.dropLast16(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple22.dropLast17(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple22.dropLast18(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple22.dropLast19(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple22.dropLast20(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple22.dropLast21(): Tuple1 = Tuple1(this._1()) +fun Tuple22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.dropLast22(): EmptyTuple = EmptyTuple diff --git a/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/TupleExtending.kt b/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/TupleExtending.kt new file mode 100644 index 00000000..595d1883 --- /dev/null +++ b/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/TupleExtending.kt @@ -0,0 +1,161 @@ +/*- + * =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("FunctionName", "RemoveExplicitTypeArguments") +package org.jetbrains.kotlinx.spark.api.tuples + +import scala.Tuple1 +import scala.Tuple2 +import scala.Tuple3 +import scala.Tuple4 +import scala.Tuple5 +import scala.Tuple6 +import scala.Tuple7 +import scala.Tuple8 +import scala.Tuple9 +import scala.Tuple10 +import scala.Tuple11 +import scala.Tuple12 +import scala.Tuple13 +import scala.Tuple14 +import scala.Tuple15 +import scala.Tuple16 +import scala.Tuple17 +import scala.Tuple18 +import scala.Tuple19 +import scala.Tuple20 +import scala.Tuple21 +import scala.Tuple22 + +/** + * This file provides functions to easily extend Scala Tuples. + * + * This means you can easily create a new tuple appended-, or prepended by a new value or tuple. + * + * For example (using tupleOf() to create a new tuple): + * ```tupleOf(a, b).appendedBy(c) == tupleOf(a, b, c)``` + * and + * ```tupleOf(a, b).prependedBy(c) == tupleOf(c, a, b)``` + * + * or in shorthand: + * ```tupleOf(a, b) + c == tupleOf(a, b, c)``` + * and + * ```c + tupleOf(a, b) == tupleOf(c, a, b)``` + * + * Note that ```tupleOf(a, b) + tupleOf(c, d)``` will merge the two into ```tupleOf(a, b, c, d)```: + * If you mean to create ```tupleOf(a, b, tupleOf(c, d))``` or ```tupleOf(tupleOf(a, b), c, d)```, + * use [appendedBy] and [prependedBy] explicitly. + * + * Note that [String.plus] concatenates any object to the string, so prepending it like ```myString + myTuple``` won't work. + * + * For concatenating two tuples, see [org.jetbrains.kotlinx.spark.api.tuples.concat]. + * + */ + +fun EmptyTuple.appendedBy(other: T1): Tuple1 = Tuple1(other) +fun Tuple1.appendedBy(other: T2): Tuple2 = Tuple2(this._1(), other) +fun Tuple2.appendedBy(other: T3): Tuple3 = Tuple3(this._1(), this._2(), other) +fun Tuple3.appendedBy(other: T4): Tuple4 = Tuple4(this._1(), this._2(), this._3(), other) +fun Tuple4.appendedBy(other: T5): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), other) +fun Tuple5.appendedBy(other: T6): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), other) +fun Tuple6.appendedBy(other: T7): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), other) +fun Tuple7.appendedBy(other: T8): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), other) +fun Tuple8.appendedBy(other: T9): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), other) +fun Tuple9.appendedBy(other: T10): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), other) +fun Tuple10.appendedBy(other: T11): Tuple11 = Tuple11(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), other) +fun Tuple11.appendedBy(other: T12): Tuple12 = Tuple12(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), other) +fun Tuple12.appendedBy(other: T13): 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(), other) +fun Tuple13.appendedBy(other: T14): 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(), other) +fun Tuple14.appendedBy(other: T15): 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(), other) +fun Tuple15.appendedBy(other: T16): 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(), other) +fun Tuple16.appendedBy(other: T17): 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(), other) +fun Tuple17.appendedBy(other: T18): 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(), other) +fun Tuple18.appendedBy(other: T19): 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(), other) +fun Tuple19.appendedBy(other: T20): 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(), other) +fun Tuple20.appendedBy(other: T21): 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(), other) +fun Tuple21.appendedBy(other: T22): 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(), other) + +fun EmptyTuple.prependedBy(other: T1): Tuple1 = Tuple1(other) +fun Tuple1.prependedBy(other: T1): Tuple2 = Tuple2(other, this._1()) +fun Tuple2.prependedBy(other: T1): Tuple3 = Tuple3(other, this._1(), this._2()) +fun Tuple3.prependedBy(other: T1): Tuple4 = Tuple4(other, this._1(), this._2(), this._3()) +fun Tuple4.prependedBy(other: T1): Tuple5 = Tuple5(other, this._1(), this._2(), this._3(), this._4()) +fun Tuple5.prependedBy(other: T1): Tuple6 = Tuple6(other, this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple6.prependedBy(other: T1): Tuple7 = Tuple7(other, this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple7.prependedBy(other: T1): Tuple8 = Tuple8(other, this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple8.prependedBy(other: T1): Tuple9 = Tuple9(other, this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple9.prependedBy(other: T1): Tuple10 = Tuple10(other, this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple10.prependedBy(other: T1): Tuple11 = Tuple11(other, this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple11.prependedBy(other: T1): Tuple12 = Tuple12(other, this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11()) +fun Tuple12.prependedBy(other: T1): Tuple13 = Tuple13(other, 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 Tuple13.prependedBy(other: T1): Tuple14 = Tuple14(other, 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 Tuple14.prependedBy(other: T1): Tuple15 = Tuple15(other, 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 Tuple15.prependedBy(other: T1): Tuple16 = Tuple16(other, 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 Tuple16.prependedBy(other: T1): Tuple17 = Tuple17(other, 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 Tuple17.prependedBy(other: T1): Tuple18 = Tuple18(other, 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 Tuple18.prependedBy(other: T1): Tuple19 = Tuple19(other, 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 Tuple19.prependedBy(other: T1): Tuple20 = Tuple20(other, 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 Tuple20.prependedBy(other: T1): Tuple21 = Tuple21(other, 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 Tuple21.prependedBy(other: T1): Tuple22 = Tuple22(other, 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()) + +operator fun EmptyTuple.plus(other: T1): Tuple1 = Tuple1(other) +operator fun Tuple1.plus(other: T2): Tuple2 = this.appendedBy(other) +operator fun Tuple2.plus(other: T3): Tuple3 = this.appendedBy(other) +operator fun Tuple3.plus(other: T4): Tuple4 = this.appendedBy(other) +operator fun Tuple4.plus(other: T5): Tuple5 = this.appendedBy(other) +operator fun Tuple5.plus(other: T6): Tuple6 = this.appendedBy(other) +operator fun Tuple6.plus(other: T7): Tuple7 = this.appendedBy(other) +operator fun Tuple7.plus(other: T8): Tuple8 = this.appendedBy(other) +operator fun Tuple8.plus(other: T9): Tuple9 = this.appendedBy(other) +operator fun Tuple9.plus(other: T10): Tuple10 = this.appendedBy(other) +operator fun Tuple10.plus(other: T11): Tuple11 = this.appendedBy(other) +operator fun Tuple11.plus(other: T12): Tuple12 = this.appendedBy(other) +operator fun Tuple12.plus(other: T13): Tuple13 = this.appendedBy(other) +operator fun Tuple13.plus(other: T14): Tuple14 = this.appendedBy(other) +operator fun Tuple14.plus(other: T15): Tuple15 = this.appendedBy(other) +operator fun Tuple15.plus(other: T16): Tuple16 = this.appendedBy(other) +operator fun Tuple16.plus(other: T17): Tuple17 = this.appendedBy(other) +operator fun Tuple17.plus(other: T18): Tuple18 = this.appendedBy(other) +operator fun Tuple18.plus(other: T19): Tuple19 = this.appendedBy(other) +operator fun Tuple19.plus(other: T20): Tuple20 = this.appendedBy(other) +operator fun Tuple20.plus(other: T21): Tuple21 = this.appendedBy(other) +operator fun Tuple21.plus(other: T22): Tuple22 = this.appendedBy(other) + +operator fun T1.plus(other: EmptyTuple): Tuple1 = Tuple1(this) +operator fun T1.plus(other: Tuple1): Tuple2 = other.prependedBy(this) +operator fun T1.plus(other: Tuple2): Tuple3 = other.prependedBy(this) +operator fun T1.plus(other: Tuple3): Tuple4 = other.prependedBy(this) +operator fun T1.plus(other: Tuple4): Tuple5 = other.prependedBy(this) +operator fun T1.plus(other: Tuple5): Tuple6 = other.prependedBy(this) +operator fun T1.plus(other: Tuple6): Tuple7 = other.prependedBy(this) +operator fun T1.plus(other: Tuple7): Tuple8 = other.prependedBy(this) +operator fun T1.plus(other: Tuple8): Tuple9 = other.prependedBy(this) +operator fun T1.plus(other: Tuple9): Tuple10 = other.prependedBy(this) +operator fun T1.plus(other: Tuple10): Tuple11 = other.prependedBy(this) +operator fun T1.plus(other: Tuple11): Tuple12 = other.prependedBy(this) +operator fun T1.plus(other: Tuple12): Tuple13 = other.prependedBy(this) +operator fun T1.plus(other: Tuple13): Tuple14 = other.prependedBy(this) +operator fun T1.plus(other: Tuple14): Tuple15 = other.prependedBy(this) +operator fun T1.plus(other: Tuple15): Tuple16 = other.prependedBy(this) +operator fun T1.plus(other: Tuple16): Tuple17 = other.prependedBy(this) +operator fun T1.plus(other: Tuple17): Tuple18 = other.prependedBy(this) +operator fun T1.plus(other: Tuple18): Tuple19 = other.prependedBy(this) +operator fun T1.plus(other: Tuple19): Tuple20 = other.prependedBy(this) +operator fun T1.plus(other: Tuple20): Tuple21 = other.prependedBy(this) +operator fun T1.plus(other: Tuple21): Tuple22 = other.prependedBy(this) diff --git a/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/TupleSplit.kt b/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/TupleSplit.kt new file mode 100644 index 00000000..984bd213 --- /dev/null +++ b/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/TupleSplit.kt @@ -0,0 +1,310 @@ +/*- + * =LICENSE= + * Kotlin Spark API: Scala Tuples in Kotlin + * ---------- + * 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.tuples + +import scala.* + +/** + * Given a tuple `t(a1, ..., am)`, returns a [Tuple2] of the tuple `t(a1, ..., an)` + * consisting of the first n elements, and the tuple `t(an+1, ..., am)` consisting + * of the remaining elements. + * Splitting at 0 or at n results in `t(t(), myTuple)` or `t(myTuple, t())` respectively. + * + * For example: + * ```kotlin + * t(1, 2, 3, 4, 5).splitAt2() == t(t(1, 2), t(3, 4, 5)) + * ``` + */ + +fun Tuple1.splitAt0(): Tuple2> = Tuple2>(EmptyTuple, Tuple1(this._1())) +fun Tuple1.splitAt1(): Tuple2, EmptyTuple> = Tuple2, EmptyTuple>(Tuple1(this._1()), EmptyTuple) +fun Tuple2.splitAt0(): Tuple2> = Tuple2>(EmptyTuple, Tuple2(this._1(), this._2())) +fun Tuple2.splitAt1(): Tuple2, Tuple1> = Tuple2, Tuple1>(Tuple1(this._1()), Tuple1(this._2())) +fun Tuple2.splitAt2(): Tuple2, EmptyTuple> = Tuple2, EmptyTuple>(Tuple2(this._1(), this._2()), EmptyTuple) +fun Tuple3.splitAt0(): Tuple2> = Tuple2>(EmptyTuple, Tuple3(this._1(), this._2(), this._3())) +fun Tuple3.splitAt1(): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple1(this._1()), Tuple2(this._2(), this._3())) +fun Tuple3.splitAt2(): Tuple2, Tuple1> = Tuple2, Tuple1>(Tuple2(this._1(), this._2()), Tuple1(this._3())) +fun Tuple3.splitAt3(): Tuple2, EmptyTuple> = Tuple2, EmptyTuple>(Tuple3(this._1(), this._2(), this._3()), EmptyTuple) +fun Tuple4.splitAt0(): Tuple2> = Tuple2>(EmptyTuple, Tuple4(this._1(), this._2(), this._3(), this._4())) +fun Tuple4.splitAt1(): Tuple2, Tuple3> = Tuple2, Tuple3>(Tuple1(this._1()), Tuple3(this._2(), this._3(), this._4())) +fun Tuple4.splitAt2(): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), this._2()), Tuple2(this._3(), this._4())) +fun Tuple4.splitAt3(): Tuple2, Tuple1> = Tuple2, Tuple1>(Tuple3(this._1(), this._2(), this._3()), Tuple1(this._4())) +fun Tuple4.splitAt4(): Tuple2, EmptyTuple> = Tuple2, EmptyTuple>(Tuple4(this._1(), this._2(), this._3(), this._4()), EmptyTuple) +fun Tuple5.splitAt0(): Tuple2> = Tuple2>(EmptyTuple, Tuple5(this._1(), this._2(), this._3(), this._4(), this._5())) +fun Tuple5.splitAt1(): Tuple2, Tuple4> = Tuple2, Tuple4>(Tuple1(this._1()), Tuple4(this._2(), this._3(), this._4(), this._5())) +fun Tuple5.splitAt2(): Tuple2, Tuple3> = Tuple2, Tuple3>(Tuple2(this._1(), this._2()), Tuple3(this._3(), this._4(), this._5())) +fun Tuple5.splitAt3(): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple3(this._1(), this._2(), this._3()), Tuple2(this._4(), this._5())) +fun Tuple5.splitAt4(): Tuple2, Tuple1> = Tuple2, Tuple1>(Tuple4(this._1(), this._2(), this._3(), this._4()), Tuple1(this._5())) +fun Tuple5.splitAt5(): Tuple2, EmptyTuple> = Tuple2, EmptyTuple>(Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()), EmptyTuple) +fun Tuple6.splitAt0(): Tuple2> = Tuple2>(EmptyTuple, Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6())) +fun Tuple6.splitAt1(): Tuple2, Tuple5> = Tuple2, Tuple5>(Tuple1(this._1()), Tuple5(this._2(), this._3(), this._4(), this._5(), this._6())) +fun Tuple6.splitAt2(): Tuple2, Tuple4> = Tuple2, Tuple4>(Tuple2(this._1(), this._2()), Tuple4(this._3(), this._4(), this._5(), this._6())) +fun Tuple6.splitAt3(): Tuple2, Tuple3> = Tuple2, Tuple3>(Tuple3(this._1(), this._2(), this._3()), Tuple3(this._4(), this._5(), this._6())) +fun Tuple6.splitAt4(): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple4(this._1(), this._2(), this._3(), this._4()), Tuple2(this._5(), this._6())) +fun Tuple6.splitAt5(): Tuple2, Tuple1> = Tuple2, Tuple1>(Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()), Tuple1(this._6())) +fun Tuple6.splitAt6(): Tuple2, EmptyTuple> = Tuple2, EmptyTuple>(Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()), EmptyTuple) +fun Tuple7.splitAt0(): Tuple2> = Tuple2>(EmptyTuple, Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7())) +fun Tuple7.splitAt1(): Tuple2, Tuple6> = Tuple2, Tuple6>(Tuple1(this._1()), Tuple6(this._2(), this._3(), this._4(), this._5(), this._6(), this._7())) +fun Tuple7.splitAt2(): Tuple2, Tuple5> = Tuple2, Tuple5>(Tuple2(this._1(), this._2()), Tuple5(this._3(), this._4(), this._5(), this._6(), this._7())) +fun Tuple7.splitAt3(): Tuple2, Tuple4> = Tuple2, Tuple4>(Tuple3(this._1(), this._2(), this._3()), Tuple4(this._4(), this._5(), this._6(), this._7())) +fun Tuple7.splitAt4(): Tuple2, Tuple3> = Tuple2, Tuple3>(Tuple4(this._1(), this._2(), this._3(), this._4()), Tuple3(this._5(), this._6(), this._7())) +fun Tuple7.splitAt5(): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()), Tuple2(this._6(), this._7())) +fun Tuple7.splitAt6(): Tuple2, Tuple1> = Tuple2, Tuple1>(Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()), Tuple1(this._7())) +fun Tuple7.splitAt7(): Tuple2, EmptyTuple> = Tuple2, EmptyTuple>(Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()), EmptyTuple) +fun Tuple8.splitAt0(): Tuple2> = Tuple2>(EmptyTuple, Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8())) +fun Tuple8.splitAt1(): Tuple2, Tuple7> = Tuple2, Tuple7>(Tuple1(this._1()), Tuple7(this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8())) +fun Tuple8.splitAt2(): Tuple2, Tuple6> = Tuple2, Tuple6>(Tuple2(this._1(), this._2()), Tuple6(this._3(), this._4(), this._5(), this._6(), this._7(), this._8())) +fun Tuple8.splitAt3(): Tuple2, Tuple5> = Tuple2, Tuple5>(Tuple3(this._1(), this._2(), this._3()), Tuple5(this._4(), this._5(), this._6(), this._7(), this._8())) +fun Tuple8.splitAt4(): Tuple2, Tuple4> = Tuple2, Tuple4>(Tuple4(this._1(), this._2(), this._3(), this._4()), Tuple4(this._5(), this._6(), this._7(), this._8())) +fun Tuple8.splitAt5(): Tuple2, Tuple3> = Tuple2, Tuple3>(Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()), Tuple3(this._6(), this._7(), this._8())) +fun Tuple8.splitAt6(): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()), Tuple2(this._7(), this._8())) +fun Tuple8.splitAt7(): Tuple2, Tuple1> = Tuple2, Tuple1>(Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()), Tuple1(this._8())) +fun Tuple8.splitAt8(): Tuple2, EmptyTuple> = Tuple2, EmptyTuple>(Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()), EmptyTuple) +fun Tuple9.splitAt0(): Tuple2> = Tuple2>(EmptyTuple, Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9())) +fun Tuple9.splitAt1(): Tuple2, Tuple8> = Tuple2, Tuple8>(Tuple1(this._1()), Tuple8(this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9())) +fun Tuple9.splitAt2(): Tuple2, Tuple7> = Tuple2, Tuple7>(Tuple2(this._1(), this._2()), Tuple7(this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9())) +fun Tuple9.splitAt3(): Tuple2, Tuple6> = Tuple2, Tuple6>(Tuple3(this._1(), this._2(), this._3()), Tuple6(this._4(), this._5(), this._6(), this._7(), this._8(), this._9())) +fun Tuple9.splitAt4(): Tuple2, Tuple5> = Tuple2, Tuple5>(Tuple4(this._1(), this._2(), this._3(), this._4()), Tuple5(this._5(), this._6(), this._7(), this._8(), this._9())) +fun Tuple9.splitAt5(): Tuple2, Tuple4> = Tuple2, Tuple4>(Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()), Tuple4(this._6(), this._7(), this._8(), this._9())) +fun Tuple9.splitAt6(): Tuple2, Tuple3> = Tuple2, Tuple3>(Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()), Tuple3(this._7(), this._8(), this._9())) +fun Tuple9.splitAt7(): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()), Tuple2(this._8(), this._9())) +fun Tuple9.splitAt8(): Tuple2, Tuple1> = Tuple2, Tuple1>(Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()), Tuple1(this._9())) +fun Tuple9.splitAt9(): Tuple2, EmptyTuple> = Tuple2, EmptyTuple>(Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()), EmptyTuple) +fun Tuple10.splitAt0(): Tuple2> = Tuple2>(EmptyTuple, Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10())) +fun Tuple10.splitAt1(): Tuple2, Tuple9> = Tuple2, Tuple9>(Tuple1(this._1()), Tuple9(this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10())) +fun Tuple10.splitAt2(): Tuple2, Tuple8> = Tuple2, Tuple8>(Tuple2(this._1(), this._2()), Tuple8(this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10())) +fun Tuple10.splitAt3(): Tuple2, Tuple7> = Tuple2, Tuple7>(Tuple3(this._1(), this._2(), this._3()), Tuple7(this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10())) +fun Tuple10.splitAt4(): Tuple2, Tuple6> = Tuple2, Tuple6>(Tuple4(this._1(), this._2(), this._3(), this._4()), Tuple6(this._5(), this._6(), this._7(), this._8(), this._9(), this._10())) +fun Tuple10.splitAt5(): Tuple2, Tuple5> = Tuple2, Tuple5>(Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()), Tuple5(this._6(), this._7(), this._8(), this._9(), this._10())) +fun Tuple10.splitAt6(): Tuple2, Tuple4> = Tuple2, Tuple4>(Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()), Tuple4(this._7(), this._8(), this._9(), this._10())) +fun Tuple10.splitAt7(): Tuple2, Tuple3> = Tuple2, Tuple3>(Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()), Tuple3(this._8(), this._9(), this._10())) +fun Tuple10.splitAt8(): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()), Tuple2(this._9(), this._10())) +fun Tuple10.splitAt9(): Tuple2, Tuple1> = Tuple2, Tuple1>(Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()), Tuple1(this._10())) +fun Tuple10.splitAt10(): Tuple2, EmptyTuple> = Tuple2, EmptyTuple>(Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()), EmptyTuple) +fun Tuple11.splitAt0(): Tuple2> = Tuple2>(EmptyTuple, Tuple11(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11())) +fun Tuple11.splitAt1(): Tuple2, Tuple10> = Tuple2, Tuple10>(Tuple1(this._1()), Tuple10(this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11())) +fun Tuple11.splitAt2(): Tuple2, Tuple9> = Tuple2, Tuple9>(Tuple2(this._1(), this._2()), Tuple9(this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11())) +fun Tuple11.splitAt3(): Tuple2, Tuple8> = Tuple2, Tuple8>(Tuple3(this._1(), this._2(), this._3()), Tuple8(this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11())) +fun Tuple11.splitAt4(): Tuple2, Tuple7> = Tuple2, Tuple7>(Tuple4(this._1(), this._2(), this._3(), this._4()), Tuple7(this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11())) +fun Tuple11.splitAt5(): Tuple2, Tuple6> = Tuple2, Tuple6>(Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()), Tuple6(this._6(), this._7(), this._8(), this._9(), this._10(), this._11())) +fun Tuple11.splitAt6(): Tuple2, Tuple5> = Tuple2, Tuple5>(Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()), Tuple5(this._7(), this._8(), this._9(), this._10(), this._11())) +fun Tuple11.splitAt7(): Tuple2, Tuple4> = Tuple2, Tuple4>(Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()), Tuple4(this._8(), this._9(), this._10(), this._11())) +fun Tuple11.splitAt8(): Tuple2, Tuple3> = Tuple2, Tuple3>(Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()), Tuple3(this._9(), this._10(), this._11())) +fun Tuple11.splitAt9(): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()), Tuple2(this._10(), this._11())) +fun Tuple11.splitAt10(): Tuple2, Tuple1> = Tuple2, Tuple1>(Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()), Tuple1(this._11())) +fun Tuple11.splitAt11(): Tuple2, EmptyTuple> = Tuple2, EmptyTuple>(Tuple11(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11()), EmptyTuple) +fun Tuple12.splitAt0(): Tuple2> = Tuple2>(EmptyTuple, 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 Tuple12.splitAt1(): Tuple2, Tuple11> = Tuple2, Tuple11>(Tuple1(this._1()), Tuple11(this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12())) +fun Tuple12.splitAt2(): Tuple2, Tuple10> = Tuple2, Tuple10>(Tuple2(this._1(), this._2()), Tuple10(this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12())) +fun Tuple12.splitAt3(): Tuple2, Tuple9> = Tuple2, Tuple9>(Tuple3(this._1(), this._2(), this._3()), Tuple9(this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12())) +fun Tuple12.splitAt4(): Tuple2, Tuple8> = Tuple2, Tuple8>(Tuple4(this._1(), this._2(), this._3(), this._4()), Tuple8(this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12())) +fun Tuple12.splitAt5(): Tuple2, Tuple7> = Tuple2, Tuple7>(Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()), Tuple7(this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12())) +fun Tuple12.splitAt6(): Tuple2, Tuple6> = Tuple2, Tuple6>(Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()), Tuple6(this._7(), this._8(), this._9(), this._10(), this._11(), this._12())) +fun Tuple12.splitAt7(): Tuple2, Tuple5> = Tuple2, Tuple5>(Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()), Tuple5(this._8(), this._9(), this._10(), this._11(), this._12())) +fun Tuple12.splitAt8(): Tuple2, Tuple4> = Tuple2, Tuple4>(Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()), Tuple4(this._9(), this._10(), this._11(), this._12())) +fun Tuple12.splitAt9(): Tuple2, Tuple3> = Tuple2, Tuple3>(Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()), Tuple3(this._10(), this._11(), this._12())) +fun Tuple12.splitAt10(): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()), Tuple2(this._11(), this._12())) +fun Tuple12.splitAt11(): Tuple2, Tuple1> = Tuple2, Tuple1>(Tuple11(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11()), Tuple1(this._12())) +fun Tuple12.splitAt12(): Tuple2, EmptyTuple> = Tuple2, EmptyTuple>(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()), EmptyTuple) +fun Tuple13.splitAt0(): Tuple2> = Tuple2>(EmptyTuple, 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 Tuple13.splitAt1(): Tuple2, Tuple12> = Tuple2, Tuple12>(Tuple1(this._1()), Tuple12(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.splitAt2(): Tuple2, Tuple11> = Tuple2, Tuple11>(Tuple2(this._1(), this._2()), Tuple11(this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13())) +fun Tuple13.splitAt3(): Tuple2, Tuple10> = Tuple2, Tuple10>(Tuple3(this._1(), this._2(), this._3()), Tuple10(this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13())) +fun Tuple13.splitAt4(): Tuple2, Tuple9> = Tuple2, Tuple9>(Tuple4(this._1(), this._2(), this._3(), this._4()), Tuple9(this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13())) +fun Tuple13.splitAt5(): Tuple2, Tuple8> = Tuple2, Tuple8>(Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()), Tuple8(this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13())) +fun Tuple13.splitAt6(): Tuple2, Tuple7> = Tuple2, Tuple7>(Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()), Tuple7(this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13())) +fun Tuple13.splitAt7(): Tuple2, Tuple6> = Tuple2, Tuple6>(Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()), Tuple6(this._8(), this._9(), this._10(), this._11(), this._12(), this._13())) +fun Tuple13.splitAt8(): Tuple2, Tuple5> = Tuple2, Tuple5>(Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()), Tuple5(this._9(), this._10(), this._11(), this._12(), this._13())) +fun Tuple13.splitAt9(): Tuple2, Tuple4> = Tuple2, Tuple4>(Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()), Tuple4(this._10(), this._11(), this._12(), this._13())) +fun Tuple13.splitAt10(): Tuple2, Tuple3> = Tuple2, Tuple3>(Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()), Tuple3(this._11(), this._12(), this._13())) +fun Tuple13.splitAt11(): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple11(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11()), Tuple2(this._12(), this._13())) +fun Tuple13.splitAt12(): Tuple2, Tuple1> = Tuple2, Tuple1>(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()), Tuple1(this._13())) +fun Tuple13.splitAt13(): Tuple2, EmptyTuple> = Tuple2, EmptyTuple>(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()), EmptyTuple) +fun Tuple14.splitAt0(): Tuple2> = Tuple2>(EmptyTuple, 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 Tuple14.splitAt1(): Tuple2, Tuple13> = Tuple2, Tuple13>(Tuple1(this._1()), Tuple13(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.splitAt2(): Tuple2, Tuple12> = Tuple2, Tuple12>(Tuple2(this._1(), this._2()), Tuple12(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.splitAt3(): Tuple2, Tuple11> = Tuple2, Tuple11>(Tuple3(this._1(), this._2(), this._3()), Tuple11(this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14())) +fun Tuple14.splitAt4(): Tuple2, Tuple10> = Tuple2, Tuple10>(Tuple4(this._1(), this._2(), this._3(), this._4()), Tuple10(this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14())) +fun Tuple14.splitAt5(): Tuple2, Tuple9> = Tuple2, Tuple9>(Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()), Tuple9(this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14())) +fun Tuple14.splitAt6(): Tuple2, Tuple8> = Tuple2, Tuple8>(Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()), Tuple8(this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14())) +fun Tuple14.splitAt7(): Tuple2, Tuple7> = Tuple2, Tuple7>(Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()), Tuple7(this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14())) +fun Tuple14.splitAt8(): Tuple2, Tuple6> = Tuple2, Tuple6>(Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()), Tuple6(this._9(), this._10(), this._11(), this._12(), this._13(), this._14())) +fun Tuple14.splitAt9(): Tuple2, Tuple5> = Tuple2, Tuple5>(Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()), Tuple5(this._10(), this._11(), this._12(), this._13(), this._14())) +fun Tuple14.splitAt10(): Tuple2, Tuple4> = Tuple2, Tuple4>(Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()), Tuple4(this._11(), this._12(), this._13(), this._14())) +fun Tuple14.splitAt11(): Tuple2, Tuple3> = Tuple2, Tuple3>(Tuple11(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11()), Tuple3(this._12(), this._13(), this._14())) +fun Tuple14.splitAt12(): Tuple2, Tuple2> = Tuple2, Tuple2>(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()), Tuple2(this._13(), this._14())) +fun Tuple14.splitAt13(): Tuple2, Tuple1> = Tuple2, Tuple1>(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()), Tuple1(this._14())) +fun Tuple14.splitAt14(): Tuple2, EmptyTuple> = Tuple2, EmptyTuple>(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()), EmptyTuple) +fun Tuple15.splitAt0(): Tuple2> = Tuple2>(EmptyTuple, 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 Tuple15.splitAt1(): Tuple2, Tuple14> = Tuple2, Tuple14>(Tuple1(this._1()), Tuple14(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.splitAt2(): Tuple2, Tuple13> = Tuple2, Tuple13>(Tuple2(this._1(), this._2()), Tuple13(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.splitAt3(): Tuple2, Tuple12> = Tuple2, Tuple12>(Tuple3(this._1(), this._2(), this._3()), Tuple12(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.splitAt4(): Tuple2, Tuple11> = Tuple2, Tuple11>(Tuple4(this._1(), this._2(), this._3(), this._4()), Tuple11(this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15())) +fun Tuple15.splitAt5(): Tuple2, Tuple10> = Tuple2, Tuple10>(Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()), Tuple10(this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15())) +fun Tuple15.splitAt6(): Tuple2, Tuple9> = Tuple2, Tuple9>(Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()), Tuple9(this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15())) +fun Tuple15.splitAt7(): Tuple2, Tuple8> = Tuple2, Tuple8>(Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()), Tuple8(this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15())) +fun Tuple15.splitAt8(): Tuple2, Tuple7> = Tuple2, Tuple7>(Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()), Tuple7(this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15())) +fun Tuple15.splitAt9(): Tuple2, Tuple6> = Tuple2, Tuple6>(Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()), Tuple6(this._10(), this._11(), this._12(), this._13(), this._14(), this._15())) +fun Tuple15.splitAt10(): Tuple2, Tuple5> = Tuple2, Tuple5>(Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()), Tuple5(this._11(), this._12(), this._13(), this._14(), this._15())) +fun Tuple15.splitAt11(): Tuple2, Tuple4> = Tuple2, Tuple4>(Tuple11(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11()), Tuple4(this._12(), this._13(), this._14(), this._15())) +fun Tuple15.splitAt12(): Tuple2, Tuple3> = Tuple2, Tuple3>(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()), Tuple3(this._13(), this._14(), this._15())) +fun Tuple15.splitAt13(): Tuple2, Tuple2> = Tuple2, Tuple2>(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()), Tuple2(this._14(), this._15())) +fun Tuple15.splitAt14(): Tuple2, Tuple1> = Tuple2, Tuple1>(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()), Tuple1(this._15())) +fun Tuple15.splitAt15(): Tuple2, EmptyTuple> = Tuple2, EmptyTuple>(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()), EmptyTuple) +fun Tuple16.splitAt0(): Tuple2> = Tuple2>(EmptyTuple, 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 Tuple16.splitAt1(): Tuple2, Tuple15> = Tuple2, Tuple15>(Tuple1(this._1()), Tuple15(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.splitAt2(): Tuple2, Tuple14> = Tuple2, Tuple14>(Tuple2(this._1(), this._2()), Tuple14(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.splitAt3(): Tuple2, Tuple13> = Tuple2, Tuple13>(Tuple3(this._1(), this._2(), this._3()), Tuple13(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.splitAt4(): Tuple2, Tuple12> = Tuple2, Tuple12>(Tuple4(this._1(), this._2(), this._3(), this._4()), Tuple12(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.splitAt5(): Tuple2, Tuple11> = Tuple2, Tuple11>(Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()), Tuple11(this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16())) +fun Tuple16.splitAt6(): Tuple2, Tuple10> = Tuple2, Tuple10>(Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()), Tuple10(this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16())) +fun Tuple16.splitAt7(): Tuple2, Tuple9> = Tuple2, Tuple9>(Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()), Tuple9(this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16())) +fun Tuple16.splitAt8(): Tuple2, Tuple8> = Tuple2, Tuple8>(Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()), Tuple8(this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16())) +fun Tuple16.splitAt9(): Tuple2, Tuple7> = Tuple2, Tuple7>(Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()), Tuple7(this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16())) +fun Tuple16.splitAt10(): Tuple2, Tuple6> = Tuple2, Tuple6>(Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()), Tuple6(this._11(), this._12(), this._13(), this._14(), this._15(), this._16())) +fun Tuple16.splitAt11(): Tuple2, Tuple5> = Tuple2, Tuple5>(Tuple11(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11()), Tuple5(this._12(), this._13(), this._14(), this._15(), this._16())) +fun Tuple16.splitAt12(): Tuple2, Tuple4> = Tuple2, Tuple4>(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()), Tuple4(this._13(), this._14(), this._15(), this._16())) +fun Tuple16.splitAt13(): Tuple2, Tuple3> = Tuple2, Tuple3>(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()), Tuple3(this._14(), this._15(), this._16())) +fun Tuple16.splitAt14(): Tuple2, Tuple2> = Tuple2, Tuple2>(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()), Tuple2(this._15(), this._16())) +fun Tuple16.splitAt15(): Tuple2, Tuple1> = Tuple2, Tuple1>(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()), Tuple1(this._16())) +fun Tuple16.splitAt16(): Tuple2, EmptyTuple> = Tuple2, EmptyTuple>(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()), EmptyTuple) +fun Tuple17.splitAt0(): Tuple2> = Tuple2>(EmptyTuple, 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 Tuple17.splitAt1(): Tuple2, Tuple16> = Tuple2, Tuple16>(Tuple1(this._1()), Tuple16(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.splitAt2(): Tuple2, Tuple15> = Tuple2, Tuple15>(Tuple2(this._1(), this._2()), Tuple15(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.splitAt3(): Tuple2, Tuple14> = Tuple2, Tuple14>(Tuple3(this._1(), this._2(), this._3()), Tuple14(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.splitAt4(): Tuple2, Tuple13> = Tuple2, Tuple13>(Tuple4(this._1(), this._2(), this._3(), this._4()), Tuple13(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.splitAt5(): Tuple2, Tuple12> = Tuple2, Tuple12>(Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()), Tuple12(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.splitAt6(): Tuple2, Tuple11> = Tuple2, Tuple11>(Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()), Tuple11(this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17())) +fun Tuple17.splitAt7(): Tuple2, Tuple10> = Tuple2, Tuple10>(Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()), Tuple10(this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17())) +fun Tuple17.splitAt8(): Tuple2, Tuple9> = Tuple2, Tuple9>(Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()), Tuple9(this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17())) +fun Tuple17.splitAt9(): Tuple2, Tuple8> = Tuple2, Tuple8>(Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()), Tuple8(this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17())) +fun Tuple17.splitAt10(): Tuple2, Tuple7> = Tuple2, Tuple7>(Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()), Tuple7(this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17())) +fun Tuple17.splitAt11(): Tuple2, Tuple6> = Tuple2, Tuple6>(Tuple11(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11()), Tuple6(this._12(), this._13(), this._14(), this._15(), this._16(), this._17())) +fun Tuple17.splitAt12(): Tuple2, Tuple5> = Tuple2, Tuple5>(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()), Tuple5(this._13(), this._14(), this._15(), this._16(), this._17())) +fun Tuple17.splitAt13(): Tuple2, Tuple4> = Tuple2, Tuple4>(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()), Tuple4(this._14(), this._15(), this._16(), this._17())) +fun Tuple17.splitAt14(): Tuple2, Tuple3> = Tuple2, Tuple3>(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()), Tuple3(this._15(), this._16(), this._17())) +fun Tuple17.splitAt15(): Tuple2, Tuple2> = Tuple2, Tuple2>(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()), Tuple2(this._16(), this._17())) +fun Tuple17.splitAt16(): Tuple2, Tuple1> = Tuple2, Tuple1>(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()), Tuple1(this._17())) +fun Tuple17.splitAt17(): Tuple2, EmptyTuple> = Tuple2, EmptyTuple>(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()), EmptyTuple) +fun Tuple18.splitAt0(): Tuple2> = Tuple2>(EmptyTuple, 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 Tuple18.splitAt1(): Tuple2, Tuple17> = Tuple2, Tuple17>(Tuple1(this._1()), Tuple17(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.splitAt2(): Tuple2, Tuple16> = Tuple2, Tuple16>(Tuple2(this._1(), this._2()), Tuple16(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.splitAt3(): Tuple2, Tuple15> = Tuple2, Tuple15>(Tuple3(this._1(), this._2(), this._3()), Tuple15(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.splitAt4(): Tuple2, Tuple14> = Tuple2, Tuple14>(Tuple4(this._1(), this._2(), this._3(), this._4()), Tuple14(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.splitAt5(): Tuple2, Tuple13> = Tuple2, Tuple13>(Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()), Tuple13(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.splitAt6(): Tuple2, Tuple12> = Tuple2, Tuple12>(Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()), Tuple12(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.splitAt7(): Tuple2, Tuple11> = Tuple2, Tuple11>(Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()), Tuple11(this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18())) +fun Tuple18.splitAt8(): Tuple2, Tuple10> = Tuple2, Tuple10>(Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()), Tuple10(this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18())) +fun Tuple18.splitAt9(): Tuple2, Tuple9> = Tuple2, Tuple9>(Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()), Tuple9(this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18())) +fun Tuple18.splitAt10(): Tuple2, Tuple8> = Tuple2, Tuple8>(Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()), Tuple8(this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18())) +fun Tuple18.splitAt11(): Tuple2, Tuple7> = Tuple2, Tuple7>(Tuple11(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11()), Tuple7(this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18())) +fun Tuple18.splitAt12(): Tuple2, Tuple6> = Tuple2, Tuple6>(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()), Tuple6(this._13(), this._14(), this._15(), this._16(), this._17(), this._18())) +fun Tuple18.splitAt13(): Tuple2, Tuple5> = Tuple2, Tuple5>(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()), Tuple5(this._14(), this._15(), this._16(), this._17(), this._18())) +fun Tuple18.splitAt14(): Tuple2, Tuple4> = Tuple2, Tuple4>(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()), Tuple4(this._15(), this._16(), this._17(), this._18())) +fun Tuple18.splitAt15(): Tuple2, Tuple3> = Tuple2, Tuple3>(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()), Tuple3(this._16(), this._17(), this._18())) +fun Tuple18.splitAt16(): Tuple2, Tuple2> = Tuple2, Tuple2>(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()), Tuple2(this._17(), this._18())) +fun Tuple18.splitAt17(): Tuple2, Tuple1> = Tuple2, Tuple1>(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()), Tuple1(this._18())) +fun Tuple18.splitAt18(): Tuple2, EmptyTuple> = Tuple2, EmptyTuple>(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()), EmptyTuple) +fun Tuple19.splitAt0(): Tuple2> = Tuple2>(EmptyTuple, 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 Tuple19.splitAt1(): Tuple2, Tuple18> = Tuple2, Tuple18>(Tuple1(this._1()), Tuple18(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.splitAt2(): Tuple2, Tuple17> = Tuple2, Tuple17>(Tuple2(this._1(), this._2()), Tuple17(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.splitAt3(): Tuple2, Tuple16> = Tuple2, Tuple16>(Tuple3(this._1(), this._2(), this._3()), Tuple16(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.splitAt4(): Tuple2, Tuple15> = Tuple2, Tuple15>(Tuple4(this._1(), this._2(), this._3(), this._4()), Tuple15(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.splitAt5(): Tuple2, Tuple14> = Tuple2, Tuple14>(Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()), Tuple14(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.splitAt6(): Tuple2, Tuple13> = Tuple2, Tuple13>(Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()), Tuple13(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.splitAt7(): Tuple2, Tuple12> = Tuple2, Tuple12>(Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()), Tuple12(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.splitAt8(): Tuple2, Tuple11> = Tuple2, Tuple11>(Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()), Tuple11(this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19())) +fun Tuple19.splitAt9(): Tuple2, Tuple10> = Tuple2, Tuple10>(Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()), Tuple10(this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19())) +fun Tuple19.splitAt10(): Tuple2, Tuple9> = Tuple2, Tuple9>(Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()), Tuple9(this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19())) +fun Tuple19.splitAt11(): Tuple2, Tuple8> = Tuple2, Tuple8>(Tuple11(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11()), Tuple8(this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19())) +fun Tuple19.splitAt12(): Tuple2, Tuple7> = Tuple2, Tuple7>(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()), Tuple7(this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19())) +fun Tuple19.splitAt13(): Tuple2, Tuple6> = Tuple2, Tuple6>(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()), Tuple6(this._14(), this._15(), this._16(), this._17(), this._18(), this._19())) +fun Tuple19.splitAt14(): Tuple2, Tuple5> = Tuple2, Tuple5>(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()), Tuple5(this._15(), this._16(), this._17(), this._18(), this._19())) +fun Tuple19.splitAt15(): Tuple2, Tuple4> = Tuple2, Tuple4>(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()), Tuple4(this._16(), this._17(), this._18(), this._19())) +fun Tuple19.splitAt16(): Tuple2, Tuple3> = Tuple2, Tuple3>(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()), Tuple3(this._17(), this._18(), this._19())) +fun Tuple19.splitAt17(): Tuple2, Tuple2> = Tuple2, Tuple2>(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()), Tuple2(this._18(), this._19())) +fun Tuple19.splitAt18(): Tuple2, Tuple1> = Tuple2, Tuple1>(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()), Tuple1(this._19())) +fun Tuple19.splitAt19(): Tuple2, EmptyTuple> = Tuple2, EmptyTuple>(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()), EmptyTuple) +fun Tuple20.splitAt0(): Tuple2> = Tuple2>(EmptyTuple, 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 Tuple20.splitAt1(): Tuple2, Tuple19> = Tuple2, Tuple19>(Tuple1(this._1()), Tuple19(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.splitAt2(): Tuple2, Tuple18> = Tuple2, Tuple18>(Tuple2(this._1(), this._2()), Tuple18(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.splitAt3(): Tuple2, Tuple17> = Tuple2, Tuple17>(Tuple3(this._1(), this._2(), this._3()), Tuple17(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.splitAt4(): Tuple2, Tuple16> = Tuple2, Tuple16>(Tuple4(this._1(), this._2(), this._3(), this._4()), Tuple16(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.splitAt5(): Tuple2, Tuple15> = Tuple2, Tuple15>(Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()), Tuple15(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.splitAt6(): Tuple2, Tuple14> = Tuple2, Tuple14>(Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()), Tuple14(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.splitAt7(): Tuple2, Tuple13> = Tuple2, Tuple13>(Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()), Tuple13(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.splitAt8(): Tuple2, Tuple12> = Tuple2, Tuple12>(Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()), Tuple12(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.splitAt9(): Tuple2, Tuple11> = Tuple2, Tuple11>(Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()), Tuple11(this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20())) +fun Tuple20.splitAt10(): Tuple2, Tuple10> = Tuple2, Tuple10>(Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()), Tuple10(this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20())) +fun Tuple20.splitAt11(): Tuple2, Tuple9> = Tuple2, Tuple9>(Tuple11(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11()), Tuple9(this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20())) +fun Tuple20.splitAt12(): Tuple2, Tuple8> = Tuple2, Tuple8>(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()), Tuple8(this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20())) +fun Tuple20.splitAt13(): Tuple2, Tuple7> = Tuple2, Tuple7>(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()), Tuple7(this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20())) +fun Tuple20.splitAt14(): Tuple2, Tuple6> = Tuple2, Tuple6>(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()), Tuple6(this._15(), this._16(), this._17(), this._18(), this._19(), this._20())) +fun Tuple20.splitAt15(): Tuple2, Tuple5> = Tuple2, Tuple5>(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()), Tuple5(this._16(), this._17(), this._18(), this._19(), this._20())) +fun Tuple20.splitAt16(): Tuple2, Tuple4> = Tuple2, Tuple4>(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()), Tuple4(this._17(), this._18(), this._19(), this._20())) +fun Tuple20.splitAt17(): Tuple2, Tuple3> = Tuple2, Tuple3>(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()), Tuple3(this._18(), this._19(), this._20())) +fun Tuple20.splitAt18(): Tuple2, Tuple2> = Tuple2, Tuple2>(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()), Tuple2(this._19(), this._20())) +fun Tuple20.splitAt19(): Tuple2, Tuple1> = Tuple2, Tuple1>(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()), Tuple1(this._20())) +fun Tuple20.splitAt20(): Tuple2, EmptyTuple> = Tuple2, EmptyTuple>(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()), EmptyTuple) +fun Tuple21.splitAt0(): Tuple2> = Tuple2>(EmptyTuple, 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 Tuple21.splitAt1(): Tuple2, Tuple20> = Tuple2, Tuple20>(Tuple1(this._1()), Tuple20(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.splitAt2(): Tuple2, Tuple19> = Tuple2, Tuple19>(Tuple2(this._1(), this._2()), Tuple19(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.splitAt3(): Tuple2, Tuple18> = Tuple2, Tuple18>(Tuple3(this._1(), this._2(), this._3()), Tuple18(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.splitAt4(): Tuple2, Tuple17> = Tuple2, Tuple17>(Tuple4(this._1(), this._2(), this._3(), this._4()), Tuple17(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.splitAt5(): Tuple2, Tuple16> = Tuple2, Tuple16>(Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()), Tuple16(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.splitAt6(): Tuple2, Tuple15> = Tuple2, Tuple15>(Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()), Tuple15(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.splitAt7(): Tuple2, Tuple14> = Tuple2, Tuple14>(Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()), Tuple14(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.splitAt8(): Tuple2, Tuple13> = Tuple2, Tuple13>(Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()), Tuple13(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.splitAt9(): Tuple2, Tuple12> = Tuple2, Tuple12>(Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()), Tuple12(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.splitAt10(): Tuple2, Tuple11> = Tuple2, Tuple11>(Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()), Tuple11(this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20(), this._21())) +fun Tuple21.splitAt11(): Tuple2, Tuple10> = Tuple2, Tuple10>(Tuple11(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11()), Tuple10(this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20(), this._21())) +fun Tuple21.splitAt12(): Tuple2, Tuple9> = Tuple2, Tuple9>(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()), Tuple9(this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20(), this._21())) +fun Tuple21.splitAt13(): Tuple2, Tuple8> = Tuple2, Tuple8>(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()), Tuple8(this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20(), this._21())) +fun Tuple21.splitAt14(): Tuple2, Tuple7> = Tuple2, Tuple7>(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()), Tuple7(this._15(), this._16(), this._17(), this._18(), this._19(), this._20(), this._21())) +fun Tuple21.splitAt15(): Tuple2, Tuple6> = Tuple2, Tuple6>(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()), Tuple6(this._16(), this._17(), this._18(), this._19(), this._20(), this._21())) +fun Tuple21.splitAt16(): Tuple2, Tuple5> = Tuple2, Tuple5>(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()), Tuple5(this._17(), this._18(), this._19(), this._20(), this._21())) +fun Tuple21.splitAt17(): Tuple2, Tuple4> = Tuple2, Tuple4>(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()), Tuple4(this._18(), this._19(), this._20(), this._21())) +fun Tuple21.splitAt18(): Tuple2, Tuple3> = Tuple2, Tuple3>(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()), Tuple3(this._19(), this._20(), this._21())) +fun Tuple21.splitAt19(): Tuple2, Tuple2> = Tuple2, Tuple2>(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()), Tuple2(this._20(), this._21())) +fun Tuple21.splitAt20(): Tuple2, Tuple1> = Tuple2, Tuple1>(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()), Tuple1(this._21())) +fun Tuple21.splitAt21(): Tuple2, EmptyTuple> = Tuple2, EmptyTuple>(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()), EmptyTuple) +fun Tuple22.splitAt0(): Tuple2> = Tuple2>(EmptyTuple, 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 Tuple22.splitAt1(): Tuple2, Tuple21> = Tuple2, Tuple21>(Tuple1(this._1()), Tuple21(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.splitAt2(): Tuple2, Tuple20> = Tuple2, Tuple20>(Tuple2(this._1(), this._2()), Tuple20(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.splitAt3(): Tuple2, Tuple19> = Tuple2, Tuple19>(Tuple3(this._1(), this._2(), this._3()), Tuple19(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.splitAt4(): Tuple2, Tuple18> = Tuple2, Tuple18>(Tuple4(this._1(), this._2(), this._3(), this._4()), Tuple18(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.splitAt5(): Tuple2, Tuple17> = Tuple2, Tuple17>(Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()), Tuple17(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.splitAt6(): Tuple2, Tuple16> = Tuple2, Tuple16>(Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()), Tuple16(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.splitAt7(): Tuple2, Tuple15> = Tuple2, Tuple15>(Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()), Tuple15(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.splitAt8(): Tuple2, Tuple14> = Tuple2, Tuple14>(Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()), Tuple14(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.splitAt9(): Tuple2, Tuple13> = Tuple2, Tuple13>(Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()), Tuple13(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.splitAt10(): Tuple2, Tuple12> = Tuple2, Tuple12>(Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()), Tuple12(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.splitAt11(): Tuple2, Tuple11> = Tuple2, Tuple11>(Tuple11(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11()), Tuple11(this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20(), this._21(), this._22())) +fun Tuple22.splitAt12(): Tuple2, Tuple10> = Tuple2, Tuple10>(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()), Tuple10(this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20(), this._21(), this._22())) +fun Tuple22.splitAt13(): Tuple2, Tuple9> = Tuple2, Tuple9>(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()), Tuple9(this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20(), this._21(), this._22())) +fun Tuple22.splitAt14(): Tuple2, Tuple8> = Tuple2, Tuple8>(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()), Tuple8(this._15(), this._16(), this._17(), this._18(), this._19(), this._20(), this._21(), this._22())) +fun Tuple22.splitAt15(): Tuple2, Tuple7> = Tuple2, Tuple7>(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()), Tuple7(this._16(), this._17(), this._18(), this._19(), this._20(), this._21(), this._22())) +fun Tuple22.splitAt16(): Tuple2, Tuple6> = Tuple2, Tuple6>(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()), Tuple6(this._17(), this._18(), this._19(), this._20(), this._21(), this._22())) +fun Tuple22.splitAt17(): Tuple2, Tuple5> = Tuple2, Tuple5>(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()), Tuple5(this._18(), this._19(), this._20(), this._21(), this._22())) +fun Tuple22.splitAt18(): Tuple2, Tuple4> = Tuple2, Tuple4>(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()), Tuple4(this._19(), this._20(), this._21(), this._22())) +fun Tuple22.splitAt19(): Tuple2, Tuple3> = Tuple2, Tuple3>(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()), Tuple3(this._20(), this._21(), this._22())) +fun Tuple22.splitAt20(): Tuple2, Tuple2> = Tuple2, Tuple2>(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()), Tuple2(this._21(), this._22())) +fun Tuple22.splitAt21(): Tuple2, Tuple1> = Tuple2, Tuple1>(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()), Tuple1(this._22())) +fun Tuple22.splitAt22(): Tuple2, EmptyTuple> = Tuple2, EmptyTuple>(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()), EmptyTuple) diff --git a/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/TupleTake.kt b/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/TupleTake.kt new file mode 100644 index 00000000..ffe7f18d --- /dev/null +++ b/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/TupleTake.kt @@ -0,0 +1,587 @@ +/*- + * =LICENSE= + * Kotlin Spark API: Scala Tuples in Kotlin + * ---------- + * 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.tuples + +import scala.* + +/** + * This file contains all functions to take N items from the beginning or end of a Tuple. + * If 0 items are taken, the result will be [EmptyTuple]. + * + * For example: + * ```kotlin + * tupleOf(1, 2, 3, 4).take2() == tupleOf(1, 2) + * tupleOf(1, 2, 3, 4).takeLast2() == tupleOf(3, 4) + * ``` + */ + +fun Tuple1<*>.take0(): EmptyTuple = EmptyTuple +fun Tuple1.take1(): Tuple1 = Tuple1(this._1()) +fun Tuple2<*, *>.take0(): EmptyTuple = EmptyTuple +fun Tuple2.take1(): Tuple1 = Tuple1(this._1()) +fun Tuple2.take2(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple3<*, *, *>.take0(): EmptyTuple = EmptyTuple +fun Tuple3.take1(): Tuple1 = Tuple1(this._1()) +fun Tuple3.take2(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple3.take3(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple4<*, *, *, *>.take0(): EmptyTuple = EmptyTuple +fun Tuple4.take1(): Tuple1 = Tuple1(this._1()) +fun Tuple4.take2(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple4.take3(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple4.take4(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple5<*, *, *, *, *>.take0(): EmptyTuple = EmptyTuple +fun Tuple5.take1(): Tuple1 = Tuple1(this._1()) +fun Tuple5.take2(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple5.take3(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple5.take4(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple5.take5(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple6<*, *, *, *, *, *>.take0(): EmptyTuple = EmptyTuple +fun Tuple6.take1(): Tuple1 = Tuple1(this._1()) +fun Tuple6.take2(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple6.take3(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple6.take4(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple6.take5(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple6.take6(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple7<*, *, *, *, *, *, *>.take0(): EmptyTuple = EmptyTuple +fun Tuple7.take1(): Tuple1 = Tuple1(this._1()) +fun Tuple7.take2(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple7.take3(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple7.take4(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple7.take5(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple7.take6(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple7.take7(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple8<*, *, *, *, *, *, *, *>.take0(): EmptyTuple = EmptyTuple +fun Tuple8.take1(): Tuple1 = Tuple1(this._1()) +fun Tuple8.take2(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple8.take3(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple8.take4(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple8.take5(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple8.take6(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple8.take7(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple8.take8(): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple9<*, *, *, *, *, *, *, *, *>.take0(): EmptyTuple = EmptyTuple +fun Tuple9.take1(): Tuple1 = Tuple1(this._1()) +fun Tuple9.take2(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple9.take3(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple9.take4(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple9.take5(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple9.take6(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple9.take7(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple9.take8(): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple9.take9(): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple10<*, *, *, *, *, *, *, *, *, *>.take0(): EmptyTuple = EmptyTuple +fun Tuple10.take1(): Tuple1 = Tuple1(this._1()) +fun Tuple10.take2(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple10.take3(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple10.take4(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple10.take5(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple10.take6(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple10.take7(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple10.take8(): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple10.take9(): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple10.take10(): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple11<*, *, *, *, *, *, *, *, *, *, *>.take0(): EmptyTuple = EmptyTuple +fun Tuple11.take1(): Tuple1 = Tuple1(this._1()) +fun Tuple11.take2(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple11.take3(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple11.take4(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple11.take5(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple11.take6(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple11.take7(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple11.take8(): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple11.take9(): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple11.take10(): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple11.take11(): 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 Tuple12<*, *, *, *, *, *, *, *, *, *, *, *>.take0(): EmptyTuple = EmptyTuple +fun Tuple12.take1(): Tuple1 = Tuple1(this._1()) +fun Tuple12.take2(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple12.take3(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple12.take4(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple12.take5(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple12.take6(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple12.take7(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple12.take8(): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple12.take9(): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple12.take10(): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple12.take11(): 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 Tuple12.take12(): 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 Tuple13<*, *, *, *, *, *, *, *, *, *, *, *, *>.take0(): EmptyTuple = EmptyTuple +fun Tuple13.take1(): Tuple1 = Tuple1(this._1()) +fun Tuple13.take2(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple13.take3(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple13.take4(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple13.take5(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple13.take6(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple13.take7(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple13.take8(): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple13.take9(): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple13.take10(): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple13.take11(): 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 Tuple13.take12(): 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 Tuple13.take13(): 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 Tuple14<*, *, *, *, *, *, *, *, *, *, *, *, *, *>.take0(): EmptyTuple = EmptyTuple +fun Tuple14.take1(): Tuple1 = Tuple1(this._1()) +fun Tuple14.take2(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple14.take3(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple14.take4(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple14.take5(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple14.take6(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple14.take7(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple14.take8(): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple14.take9(): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple14.take10(): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple14.take11(): 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 Tuple14.take12(): 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 Tuple14.take13(): 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 Tuple14.take14(): 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 Tuple15<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.take0(): EmptyTuple = EmptyTuple +fun Tuple15.take1(): Tuple1 = Tuple1(this._1()) +fun Tuple15.take2(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple15.take3(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple15.take4(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple15.take5(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple15.take6(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple15.take7(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple15.take8(): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple15.take9(): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple15.take10(): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple15.take11(): 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 Tuple15.take12(): 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 Tuple15.take13(): 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 Tuple15.take14(): 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 Tuple15.take15(): 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 Tuple16<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.take0(): EmptyTuple = EmptyTuple +fun Tuple16.take1(): Tuple1 = Tuple1(this._1()) +fun Tuple16.take2(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple16.take3(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple16.take4(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple16.take5(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple16.take6(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple16.take7(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple16.take8(): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple16.take9(): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple16.take10(): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple16.take11(): 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 Tuple16.take12(): 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 Tuple16.take13(): 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 Tuple16.take14(): 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 Tuple16.take15(): 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 Tuple16.take16(): 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 Tuple17<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.take0(): EmptyTuple = EmptyTuple +fun Tuple17.take1(): Tuple1 = Tuple1(this._1()) +fun Tuple17.take2(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple17.take3(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple17.take4(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple17.take5(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple17.take6(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple17.take7(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple17.take8(): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple17.take9(): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple17.take10(): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple17.take11(): 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 Tuple17.take12(): 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 Tuple17.take13(): 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 Tuple17.take14(): 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 Tuple17.take15(): 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 Tuple17.take16(): 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 Tuple17.take17(): 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 Tuple18<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.take0(): EmptyTuple = EmptyTuple +fun Tuple18.take1(): Tuple1 = Tuple1(this._1()) +fun Tuple18.take2(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple18.take3(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple18.take4(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple18.take5(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple18.take6(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple18.take7(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple18.take8(): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple18.take9(): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple18.take10(): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple18.take11(): 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 Tuple18.take12(): 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 Tuple18.take13(): 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 Tuple18.take14(): 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 Tuple18.take15(): 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 Tuple18.take16(): 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 Tuple18.take17(): 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 Tuple18.take18(): 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 Tuple19<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.take0(): EmptyTuple = EmptyTuple +fun Tuple19.take1(): Tuple1 = Tuple1(this._1()) +fun Tuple19.take2(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple19.take3(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple19.take4(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple19.take5(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple19.take6(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple19.take7(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple19.take8(): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple19.take9(): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple19.take10(): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple19.take11(): 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 Tuple19.take12(): 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 Tuple19.take13(): 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 Tuple19.take14(): 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 Tuple19.take15(): 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 Tuple19.take16(): 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 Tuple19.take17(): 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 Tuple19.take18(): 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 Tuple19.take19(): 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 Tuple20<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.take0(): EmptyTuple = EmptyTuple +fun Tuple20.take1(): Tuple1 = Tuple1(this._1()) +fun Tuple20.take2(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple20.take3(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple20.take4(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple20.take5(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple20.take6(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple20.take7(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple20.take8(): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple20.take9(): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple20.take10(): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple20.take11(): 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 Tuple20.take12(): 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 Tuple20.take13(): 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 Tuple20.take14(): 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 Tuple20.take15(): 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 Tuple20.take16(): 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 Tuple20.take17(): 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 Tuple20.take18(): 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 Tuple20.take19(): 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 Tuple20.take20(): 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 Tuple21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.take0(): EmptyTuple = EmptyTuple +fun Tuple21.take1(): Tuple1 = Tuple1(this._1()) +fun Tuple21.take2(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple21.take3(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple21.take4(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple21.take5(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple21.take6(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple21.take7(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple21.take8(): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple21.take9(): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple21.take10(): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple21.take11(): 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 Tuple21.take12(): 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 Tuple21.take13(): 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 Tuple21.take14(): 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 Tuple21.take15(): 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 Tuple21.take16(): 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 Tuple21.take17(): 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 Tuple21.take18(): 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 Tuple21.take19(): 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 Tuple21.take20(): 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 Tuple21.take21(): 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 Tuple22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.take0(): EmptyTuple = EmptyTuple +fun Tuple22.take1(): Tuple1 = Tuple1(this._1()) +fun Tuple22.take2(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple22.take3(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple22.take4(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple22.take5(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple22.take6(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple22.take7(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple22.take8(): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple22.take9(): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple22.take10(): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple22.take11(): 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 Tuple22.take12(): 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 Tuple22.take13(): 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 Tuple22.take14(): 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 Tuple22.take15(): 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 Tuple22.take16(): 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 Tuple22.take17(): 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 Tuple22.take18(): 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 Tuple22.take19(): 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 Tuple22.take20(): 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 Tuple22.take21(): 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 Tuple22.take22(): 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 Tuple1<*>.takeLast0(): EmptyTuple = EmptyTuple +fun Tuple1.takeLast1(): Tuple1 = Tuple1(this._1()) +fun Tuple2<*, *>.takeLast0(): EmptyTuple = EmptyTuple +fun Tuple2.takeLast2(): Tuple2 = Tuple2(this._1(), this._2()) +fun Tuple2<*, T2>.takeLast1(): Tuple1 = Tuple1(this._2()) +fun Tuple3<*, *, *>.takeLast0(): EmptyTuple = EmptyTuple +fun Tuple3.takeLast3(): Tuple3 = Tuple3(this._1(), this._2(), this._3()) +fun Tuple3<*, T2, T3>.takeLast2(): Tuple2 = Tuple2(this._2(), this._3()) +fun Tuple3<*, *, T3>.takeLast1(): Tuple1 = Tuple1(this._3()) +fun Tuple4<*, *, *, *>.takeLast0(): EmptyTuple = EmptyTuple +fun Tuple4.takeLast4(): Tuple4 = Tuple4(this._1(), this._2(), this._3(), this._4()) +fun Tuple4<*, T2, T3, T4>.takeLast3(): Tuple3 = Tuple3(this._2(), this._3(), this._4()) +fun Tuple4<*, *, T3, T4>.takeLast2(): Tuple2 = Tuple2(this._3(), this._4()) +fun Tuple4<*, *, *, T4>.takeLast1(): Tuple1 = Tuple1(this._4()) +fun Tuple5<*, *, *, *, *>.takeLast0(): EmptyTuple = EmptyTuple +fun Tuple5.takeLast5(): Tuple5 = Tuple5(this._1(), this._2(), this._3(), this._4(), this._5()) +fun Tuple5<*, T2, T3, T4, T5>.takeLast4(): Tuple4 = Tuple4(this._2(), this._3(), this._4(), this._5()) +fun Tuple5<*, *, T3, T4, T5>.takeLast3(): Tuple3 = Tuple3(this._3(), this._4(), this._5()) +fun Tuple5<*, *, *, T4, T5>.takeLast2(): Tuple2 = Tuple2(this._4(), this._5()) +fun Tuple5<*, *, *, *, T5>.takeLast1(): Tuple1 = Tuple1(this._5()) +fun Tuple6<*, *, *, *, *, *>.takeLast0(): EmptyTuple = EmptyTuple +fun Tuple6.takeLast6(): Tuple6 = Tuple6(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple6<*, T2, T3, T4, T5, T6>.takeLast5(): Tuple5 = Tuple5(this._2(), this._3(), this._4(), this._5(), this._6()) +fun Tuple6<*, *, T3, T4, T5, T6>.takeLast4(): Tuple4 = Tuple4(this._3(), this._4(), this._5(), this._6()) +fun Tuple6<*, *, *, T4, T5, T6>.takeLast3(): Tuple3 = Tuple3(this._4(), this._5(), this._6()) +fun Tuple6<*, *, *, *, T5, T6>.takeLast2(): Tuple2 = Tuple2(this._5(), this._6()) +fun Tuple6<*, *, *, *, *, T6>.takeLast1(): Tuple1 = Tuple1(this._6()) +fun Tuple7<*, *, *, *, *, *, *>.takeLast0(): EmptyTuple = EmptyTuple +fun Tuple7.takeLast7(): Tuple7 = Tuple7(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple7<*, T2, T3, T4, T5, T6, T7>.takeLast6(): Tuple6 = Tuple6(this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple7<*, *, T3, T4, T5, T6, T7>.takeLast5(): Tuple5 = Tuple5(this._3(), this._4(), this._5(), this._6(), this._7()) +fun Tuple7<*, *, *, T4, T5, T6, T7>.takeLast4(): Tuple4 = Tuple4(this._4(), this._5(), this._6(), this._7()) +fun Tuple7<*, *, *, *, T5, T6, T7>.takeLast3(): Tuple3 = Tuple3(this._5(), this._6(), this._7()) +fun Tuple7<*, *, *, *, *, T6, T7>.takeLast2(): Tuple2 = Tuple2(this._6(), this._7()) +fun Tuple7<*, *, *, *, *, *, T7>.takeLast1(): Tuple1 = Tuple1(this._7()) +fun Tuple8<*, *, *, *, *, *, *, *>.takeLast0(): EmptyTuple = EmptyTuple +fun Tuple8.takeLast8(): Tuple8 = Tuple8(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple8<*, T2, T3, T4, T5, T6, T7, T8>.takeLast7(): Tuple7 = Tuple7(this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple8<*, *, T3, T4, T5, T6, T7, T8>.takeLast6(): Tuple6 = Tuple6(this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple8<*, *, *, T4, T5, T6, T7, T8>.takeLast5(): Tuple5 = Tuple5(this._4(), this._5(), this._6(), this._7(), this._8()) +fun Tuple8<*, *, *, *, T5, T6, T7, T8>.takeLast4(): Tuple4 = Tuple4(this._5(), this._6(), this._7(), this._8()) +fun Tuple8<*, *, *, *, *, T6, T7, T8>.takeLast3(): Tuple3 = Tuple3(this._6(), this._7(), this._8()) +fun Tuple8<*, *, *, *, *, *, T7, T8>.takeLast2(): Tuple2 = Tuple2(this._7(), this._8()) +fun Tuple8<*, *, *, *, *, *, *, T8>.takeLast1(): Tuple1 = Tuple1(this._8()) +fun Tuple9<*, *, *, *, *, *, *, *, *>.takeLast0(): EmptyTuple = EmptyTuple +fun Tuple9.takeLast9(): Tuple9 = Tuple9(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple9<*, T2, T3, T4, T5, T6, T7, T8, T9>.takeLast8(): Tuple8 = Tuple8(this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple9<*, *, T3, T4, T5, T6, T7, T8, T9>.takeLast7(): Tuple7 = Tuple7(this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple9<*, *, *, T4, T5, T6, T7, T8, T9>.takeLast6(): Tuple6 = Tuple6(this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple9<*, *, *, *, T5, T6, T7, T8, T9>.takeLast5(): Tuple5 = Tuple5(this._5(), this._6(), this._7(), this._8(), this._9()) +fun Tuple9<*, *, *, *, *, T6, T7, T8, T9>.takeLast4(): Tuple4 = Tuple4(this._6(), this._7(), this._8(), this._9()) +fun Tuple9<*, *, *, *, *, *, T7, T8, T9>.takeLast3(): Tuple3 = Tuple3(this._7(), this._8(), this._9()) +fun Tuple9<*, *, *, *, *, *, *, T8, T9>.takeLast2(): Tuple2 = Tuple2(this._8(), this._9()) +fun Tuple9<*, *, *, *, *, *, *, *, T9>.takeLast1(): Tuple1 = Tuple1(this._9()) +fun Tuple10<*, *, *, *, *, *, *, *, *, *>.takeLast0(): EmptyTuple = EmptyTuple +fun Tuple10.takeLast10(): Tuple10 = Tuple10(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple10<*, T2, T3, T4, T5, T6, T7, T8, T9, T10>.takeLast9(): Tuple9 = Tuple9(this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple10<*, *, T3, T4, T5, T6, T7, T8, T9, T10>.takeLast8(): Tuple8 = Tuple8(this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple10<*, *, *, T4, T5, T6, T7, T8, T9, T10>.takeLast7(): Tuple7 = Tuple7(this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple10<*, *, *, *, T5, T6, T7, T8, T9, T10>.takeLast6(): Tuple6 = Tuple6(this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple10<*, *, *, *, *, T6, T7, T8, T9, T10>.takeLast5(): Tuple5 = Tuple5(this._6(), this._7(), this._8(), this._9(), this._10()) +fun Tuple10<*, *, *, *, *, *, T7, T8, T9, T10>.takeLast4(): Tuple4 = Tuple4(this._7(), this._8(), this._9(), this._10()) +fun Tuple10<*, *, *, *, *, *, *, T8, T9, T10>.takeLast3(): Tuple3 = Tuple3(this._8(), this._9(), this._10()) +fun Tuple10<*, *, *, *, *, *, *, *, T9, T10>.takeLast2(): Tuple2 = Tuple2(this._9(), this._10()) +fun Tuple10<*, *, *, *, *, *, *, *, *, T10>.takeLast1(): Tuple1 = Tuple1(this._10()) +fun Tuple11<*, *, *, *, *, *, *, *, *, *, *>.takeLast0(): EmptyTuple = EmptyTuple +fun Tuple11.takeLast11(): 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 Tuple11<*, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11>.takeLast10(): Tuple10 = Tuple10(this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11()) +fun Tuple11<*, *, T3, T4, T5, T6, T7, T8, T9, T10, T11>.takeLast9(): Tuple9 = Tuple9(this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11()) +fun Tuple11<*, *, *, T4, T5, T6, T7, T8, T9, T10, T11>.takeLast8(): Tuple8 = Tuple8(this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11()) +fun Tuple11<*, *, *, *, T5, T6, T7, T8, T9, T10, T11>.takeLast7(): Tuple7 = Tuple7(this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11()) +fun Tuple11<*, *, *, *, *, T6, T7, T8, T9, T10, T11>.takeLast6(): Tuple6 = Tuple6(this._6(), this._7(), this._8(), this._9(), this._10(), this._11()) +fun Tuple11<*, *, *, *, *, *, T7, T8, T9, T10, T11>.takeLast5(): Tuple5 = Tuple5(this._7(), this._8(), this._9(), this._10(), this._11()) +fun Tuple11<*, *, *, *, *, *, *, T8, T9, T10, T11>.takeLast4(): Tuple4 = Tuple4(this._8(), this._9(), this._10(), this._11()) +fun Tuple11<*, *, *, *, *, *, *, *, T9, T10, T11>.takeLast3(): Tuple3 = Tuple3(this._9(), this._10(), this._11()) +fun Tuple11<*, *, *, *, *, *, *, *, *, T10, T11>.takeLast2(): Tuple2 = Tuple2(this._10(), this._11()) +fun Tuple11<*, *, *, *, *, *, *, *, *, *, T11>.takeLast1(): Tuple1 = Tuple1(this._11()) +fun Tuple12<*, *, *, *, *, *, *, *, *, *, *, *>.takeLast0(): EmptyTuple = EmptyTuple +fun Tuple12.takeLast12(): 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 Tuple12<*, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12>.takeLast11(): Tuple11 = Tuple11(this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12()) +fun Tuple12<*, *, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12>.takeLast10(): Tuple10 = Tuple10(this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12()) +fun Tuple12<*, *, *, T4, T5, T6, T7, T8, T9, T10, T11, T12>.takeLast9(): Tuple9 = Tuple9(this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12()) +fun Tuple12<*, *, *, *, T5, T6, T7, T8, T9, T10, T11, T12>.takeLast8(): Tuple8 = Tuple8(this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12()) +fun Tuple12<*, *, *, *, *, T6, T7, T8, T9, T10, T11, T12>.takeLast7(): Tuple7 = Tuple7(this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12()) +fun Tuple12<*, *, *, *, *, *, T7, T8, T9, T10, T11, T12>.takeLast6(): Tuple6 = Tuple6(this._7(), this._8(), this._9(), this._10(), this._11(), this._12()) +fun Tuple12<*, *, *, *, *, *, *, T8, T9, T10, T11, T12>.takeLast5(): Tuple5 = Tuple5(this._8(), this._9(), this._10(), this._11(), this._12()) +fun Tuple12<*, *, *, *, *, *, *, *, T9, T10, T11, T12>.takeLast4(): Tuple4 = Tuple4(this._9(), this._10(), this._11(), this._12()) +fun Tuple12<*, *, *, *, *, *, *, *, *, T10, T11, T12>.takeLast3(): Tuple3 = Tuple3(this._10(), this._11(), this._12()) +fun Tuple12<*, *, *, *, *, *, *, *, *, *, T11, T12>.takeLast2(): Tuple2 = Tuple2(this._11(), this._12()) +fun Tuple12<*, *, *, *, *, *, *, *, *, *, *, T12>.takeLast1(): Tuple1 = Tuple1(this._12()) +fun Tuple13<*, *, *, *, *, *, *, *, *, *, *, *, *>.takeLast0(): EmptyTuple = EmptyTuple +fun Tuple13.takeLast13(): 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 Tuple13<*, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>.takeLast12(): Tuple12 = Tuple12(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<*, *, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>.takeLast11(): Tuple11 = Tuple11(this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13()) +fun Tuple13<*, *, *, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>.takeLast10(): Tuple10 = Tuple10(this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13()) +fun Tuple13<*, *, *, *, T5, T6, T7, T8, T9, T10, T11, T12, T13>.takeLast9(): Tuple9 = Tuple9(this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13()) +fun Tuple13<*, *, *, *, *, T6, T7, T8, T9, T10, T11, T12, T13>.takeLast8(): Tuple8 = Tuple8(this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13()) +fun Tuple13<*, *, *, *, *, *, T7, T8, T9, T10, T11, T12, T13>.takeLast7(): Tuple7 = Tuple7(this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13()) +fun Tuple13<*, *, *, *, *, *, *, T8, T9, T10, T11, T12, T13>.takeLast6(): Tuple6 = Tuple6(this._8(), this._9(), this._10(), this._11(), this._12(), this._13()) +fun Tuple13<*, *, *, *, *, *, *, *, T9, T10, T11, T12, T13>.takeLast5(): Tuple5 = Tuple5(this._9(), this._10(), this._11(), this._12(), this._13()) +fun Tuple13<*, *, *, *, *, *, *, *, *, T10, T11, T12, T13>.takeLast4(): Tuple4 = Tuple4(this._10(), this._11(), this._12(), this._13()) +fun Tuple13<*, *, *, *, *, *, *, *, *, *, T11, T12, T13>.takeLast3(): Tuple3 = Tuple3(this._11(), this._12(), this._13()) +fun Tuple13<*, *, *, *, *, *, *, *, *, *, *, T12, T13>.takeLast2(): Tuple2 = Tuple2(this._12(), this._13()) +fun Tuple13<*, *, *, *, *, *, *, *, *, *, *, *, T13>.takeLast1(): Tuple1 = Tuple1(this._13()) +fun Tuple14<*, *, *, *, *, *, *, *, *, *, *, *, *, *>.takeLast0(): EmptyTuple = EmptyTuple +fun Tuple14.takeLast14(): 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 Tuple14<*, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>.takeLast13(): Tuple13 = Tuple13(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<*, *, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>.takeLast12(): Tuple12 = Tuple12(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<*, *, *, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>.takeLast11(): Tuple11 = Tuple11(this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14()) +fun Tuple14<*, *, *, *, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>.takeLast10(): Tuple10 = Tuple10(this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14()) +fun Tuple14<*, *, *, *, *, T6, T7, T8, T9, T10, T11, T12, T13, T14>.takeLast9(): Tuple9 = Tuple9(this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14()) +fun Tuple14<*, *, *, *, *, *, T7, T8, T9, T10, T11, T12, T13, T14>.takeLast8(): Tuple8 = Tuple8(this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14()) +fun Tuple14<*, *, *, *, *, *, *, T8, T9, T10, T11, T12, T13, T14>.takeLast7(): Tuple7 = Tuple7(this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14()) +fun Tuple14<*, *, *, *, *, *, *, *, T9, T10, T11, T12, T13, T14>.takeLast6(): Tuple6 = Tuple6(this._9(), this._10(), this._11(), this._12(), this._13(), this._14()) +fun Tuple14<*, *, *, *, *, *, *, *, *, T10, T11, T12, T13, T14>.takeLast5(): Tuple5 = Tuple5(this._10(), this._11(), this._12(), this._13(), this._14()) +fun Tuple14<*, *, *, *, *, *, *, *, *, *, T11, T12, T13, T14>.takeLast4(): Tuple4 = Tuple4(this._11(), this._12(), this._13(), this._14()) +fun Tuple14<*, *, *, *, *, *, *, *, *, *, *, T12, T13, T14>.takeLast3(): Tuple3 = Tuple3(this._12(), this._13(), this._14()) +fun Tuple14<*, *, *, *, *, *, *, *, *, *, *, *, T13, T14>.takeLast2(): Tuple2 = Tuple2(this._13(), this._14()) +fun Tuple14<*, *, *, *, *, *, *, *, *, *, *, *, *, T14>.takeLast1(): Tuple1 = Tuple1(this._14()) +fun Tuple15<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.takeLast0(): EmptyTuple = EmptyTuple +fun Tuple15.takeLast15(): 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 Tuple15<*, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>.takeLast14(): Tuple14 = Tuple14(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<*, *, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>.takeLast13(): Tuple13 = Tuple13(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<*, *, *, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>.takeLast12(): Tuple12 = Tuple12(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<*, *, *, *, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>.takeLast11(): Tuple11 = Tuple11(this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15()) +fun Tuple15<*, *, *, *, *, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>.takeLast10(): Tuple10 = Tuple10(this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15()) +fun Tuple15<*, *, *, *, *, *, T7, T8, T9, T10, T11, T12, T13, T14, T15>.takeLast9(): Tuple9 = Tuple9(this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15()) +fun Tuple15<*, *, *, *, *, *, *, T8, T9, T10, T11, T12, T13, T14, T15>.takeLast8(): Tuple8 = Tuple8(this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15()) +fun Tuple15<*, *, *, *, *, *, *, *, T9, T10, T11, T12, T13, T14, T15>.takeLast7(): Tuple7 = Tuple7(this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15()) +fun Tuple15<*, *, *, *, *, *, *, *, *, T10, T11, T12, T13, T14, T15>.takeLast6(): Tuple6 = Tuple6(this._10(), this._11(), this._12(), this._13(), this._14(), this._15()) +fun Tuple15<*, *, *, *, *, *, *, *, *, *, T11, T12, T13, T14, T15>.takeLast5(): Tuple5 = Tuple5(this._11(), this._12(), this._13(), this._14(), this._15()) +fun Tuple15<*, *, *, *, *, *, *, *, *, *, *, T12, T13, T14, T15>.takeLast4(): Tuple4 = Tuple4(this._12(), this._13(), this._14(), this._15()) +fun Tuple15<*, *, *, *, *, *, *, *, *, *, *, *, T13, T14, T15>.takeLast3(): Tuple3 = Tuple3(this._13(), this._14(), this._15()) +fun Tuple15<*, *, *, *, *, *, *, *, *, *, *, *, *, T14, T15>.takeLast2(): Tuple2 = Tuple2(this._14(), this._15()) +fun Tuple15<*, *, *, *, *, *, *, *, *, *, *, *, *, *, T15>.takeLast1(): Tuple1 = Tuple1(this._15()) +fun Tuple16<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.takeLast0(): EmptyTuple = EmptyTuple +fun Tuple16.takeLast16(): 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 Tuple16<*, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>.takeLast15(): Tuple15 = Tuple15(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<*, *, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>.takeLast14(): Tuple14 = Tuple14(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<*, *, *, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>.takeLast13(): Tuple13 = Tuple13(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<*, *, *, *, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>.takeLast12(): Tuple12 = Tuple12(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<*, *, *, *, *, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>.takeLast11(): Tuple11 = Tuple11(this._6(), this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16()) +fun Tuple16<*, *, *, *, *, *, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>.takeLast10(): Tuple10 = Tuple10(this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16()) +fun Tuple16<*, *, *, *, *, *, *, T8, T9, T10, T11, T12, T13, T14, T15, T16>.takeLast9(): Tuple9 = Tuple9(this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16()) +fun Tuple16<*, *, *, *, *, *, *, *, T9, T10, T11, T12, T13, T14, T15, T16>.takeLast8(): Tuple8 = Tuple8(this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16()) +fun Tuple16<*, *, *, *, *, *, *, *, *, T10, T11, T12, T13, T14, T15, T16>.takeLast7(): Tuple7 = Tuple7(this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16()) +fun Tuple16<*, *, *, *, *, *, *, *, *, *, T11, T12, T13, T14, T15, T16>.takeLast6(): Tuple6 = Tuple6(this._11(), this._12(), this._13(), this._14(), this._15(), this._16()) +fun Tuple16<*, *, *, *, *, *, *, *, *, *, *, T12, T13, T14, T15, T16>.takeLast5(): Tuple5 = Tuple5(this._12(), this._13(), this._14(), this._15(), this._16()) +fun Tuple16<*, *, *, *, *, *, *, *, *, *, *, *, T13, T14, T15, T16>.takeLast4(): Tuple4 = Tuple4(this._13(), this._14(), this._15(), this._16()) +fun Tuple16<*, *, *, *, *, *, *, *, *, *, *, *, *, T14, T15, T16>.takeLast3(): Tuple3 = Tuple3(this._14(), this._15(), this._16()) +fun Tuple16<*, *, *, *, *, *, *, *, *, *, *, *, *, *, T15, T16>.takeLast2(): Tuple2 = Tuple2(this._15(), this._16()) +fun Tuple16<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T16>.takeLast1(): Tuple1 = Tuple1(this._16()) +fun Tuple17<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.takeLast0(): EmptyTuple = EmptyTuple +fun Tuple17.takeLast17(): 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 Tuple17<*, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>.takeLast16(): Tuple16 = Tuple16(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<*, *, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>.takeLast15(): Tuple15 = Tuple15(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<*, *, *, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>.takeLast14(): Tuple14 = Tuple14(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<*, *, *, *, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>.takeLast13(): Tuple13 = Tuple13(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<*, *, *, *, *, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>.takeLast12(): Tuple12 = Tuple12(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<*, *, *, *, *, *, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>.takeLast11(): Tuple11 = Tuple11(this._7(), this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17()) +fun Tuple17<*, *, *, *, *, *, *, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>.takeLast10(): Tuple10 = Tuple10(this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17()) +fun Tuple17<*, *, *, *, *, *, *, *, T9, T10, T11, T12, T13, T14, T15, T16, T17>.takeLast9(): Tuple9 = Tuple9(this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17()) +fun Tuple17<*, *, *, *, *, *, *, *, *, T10, T11, T12, T13, T14, T15, T16, T17>.takeLast8(): Tuple8 = Tuple8(this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17()) +fun Tuple17<*, *, *, *, *, *, *, *, *, *, T11, T12, T13, T14, T15, T16, T17>.takeLast7(): Tuple7 = Tuple7(this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17()) +fun Tuple17<*, *, *, *, *, *, *, *, *, *, *, T12, T13, T14, T15, T16, T17>.takeLast6(): Tuple6 = Tuple6(this._12(), this._13(), this._14(), this._15(), this._16(), this._17()) +fun Tuple17<*, *, *, *, *, *, *, *, *, *, *, *, T13, T14, T15, T16, T17>.takeLast5(): Tuple5 = Tuple5(this._13(), this._14(), this._15(), this._16(), this._17()) +fun Tuple17<*, *, *, *, *, *, *, *, *, *, *, *, *, T14, T15, T16, T17>.takeLast4(): Tuple4 = Tuple4(this._14(), this._15(), this._16(), this._17()) +fun Tuple17<*, *, *, *, *, *, *, *, *, *, *, *, *, *, T15, T16, T17>.takeLast3(): Tuple3 = Tuple3(this._15(), this._16(), this._17()) +fun Tuple17<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T16, T17>.takeLast2(): Tuple2 = Tuple2(this._16(), this._17()) +fun Tuple17<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T17>.takeLast1(): Tuple1 = Tuple1(this._17()) +fun Tuple18<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.takeLast0(): EmptyTuple = EmptyTuple +fun Tuple18.takeLast18(): 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 Tuple18<*, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>.takeLast17(): Tuple17 = Tuple17(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<*, *, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>.takeLast16(): Tuple16 = Tuple16(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<*, *, *, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>.takeLast15(): Tuple15 = Tuple15(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<*, *, *, *, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>.takeLast14(): Tuple14 = Tuple14(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<*, *, *, *, *, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>.takeLast13(): Tuple13 = Tuple13(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<*, *, *, *, *, *, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>.takeLast12(): Tuple12 = Tuple12(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<*, *, *, *, *, *, *, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>.takeLast11(): Tuple11 = Tuple11(this._8(), this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18()) +fun Tuple18<*, *, *, *, *, *, *, *, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>.takeLast10(): Tuple10 = Tuple10(this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18()) +fun Tuple18<*, *, *, *, *, *, *, *, *, T10, T11, T12, T13, T14, T15, T16, T17, T18>.takeLast9(): Tuple9 = Tuple9(this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18()) +fun Tuple18<*, *, *, *, *, *, *, *, *, *, T11, T12, T13, T14, T15, T16, T17, T18>.takeLast8(): Tuple8 = Tuple8(this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18()) +fun Tuple18<*, *, *, *, *, *, *, *, *, *, *, T12, T13, T14, T15, T16, T17, T18>.takeLast7(): Tuple7 = Tuple7(this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18()) +fun Tuple18<*, *, *, *, *, *, *, *, *, *, *, *, T13, T14, T15, T16, T17, T18>.takeLast6(): Tuple6 = Tuple6(this._13(), this._14(), this._15(), this._16(), this._17(), this._18()) +fun Tuple18<*, *, *, *, *, *, *, *, *, *, *, *, *, T14, T15, T16, T17, T18>.takeLast5(): Tuple5 = Tuple5(this._14(), this._15(), this._16(), this._17(), this._18()) +fun Tuple18<*, *, *, *, *, *, *, *, *, *, *, *, *, *, T15, T16, T17, T18>.takeLast4(): Tuple4 = Tuple4(this._15(), this._16(), this._17(), this._18()) +fun Tuple18<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T16, T17, T18>.takeLast3(): Tuple3 = Tuple3(this._16(), this._17(), this._18()) +fun Tuple18<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T17, T18>.takeLast2(): Tuple2 = Tuple2(this._17(), this._18()) +fun Tuple18<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T18>.takeLast1(): Tuple1 = Tuple1(this._18()) +fun Tuple19<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.takeLast0(): EmptyTuple = EmptyTuple +fun Tuple19.takeLast19(): 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 Tuple19<*, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>.takeLast18(): Tuple18 = Tuple18(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<*, *, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>.takeLast17(): Tuple17 = Tuple17(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<*, *, *, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>.takeLast16(): Tuple16 = Tuple16(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<*, *, *, *, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>.takeLast15(): Tuple15 = Tuple15(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<*, *, *, *, *, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>.takeLast14(): Tuple14 = Tuple14(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<*, *, *, *, *, *, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>.takeLast13(): Tuple13 = Tuple13(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<*, *, *, *, *, *, *, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>.takeLast12(): Tuple12 = Tuple12(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<*, *, *, *, *, *, *, *, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>.takeLast11(): Tuple11 = Tuple11(this._9(), this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19()) +fun Tuple19<*, *, *, *, *, *, *, *, *, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>.takeLast10(): Tuple10 = Tuple10(this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19()) +fun Tuple19<*, *, *, *, *, *, *, *, *, *, T11, T12, T13, T14, T15, T16, T17, T18, T19>.takeLast9(): Tuple9 = Tuple9(this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19()) +fun Tuple19<*, *, *, *, *, *, *, *, *, *, *, T12, T13, T14, T15, T16, T17, T18, T19>.takeLast8(): Tuple8 = Tuple8(this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19()) +fun Tuple19<*, *, *, *, *, *, *, *, *, *, *, *, T13, T14, T15, T16, T17, T18, T19>.takeLast7(): Tuple7 = Tuple7(this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19()) +fun Tuple19<*, *, *, *, *, *, *, *, *, *, *, *, *, T14, T15, T16, T17, T18, T19>.takeLast6(): Tuple6 = Tuple6(this._14(), this._15(), this._16(), this._17(), this._18(), this._19()) +fun Tuple19<*, *, *, *, *, *, *, *, *, *, *, *, *, *, T15, T16, T17, T18, T19>.takeLast5(): Tuple5 = Tuple5(this._15(), this._16(), this._17(), this._18(), this._19()) +fun Tuple19<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T16, T17, T18, T19>.takeLast4(): Tuple4 = Tuple4(this._16(), this._17(), this._18(), this._19()) +fun Tuple19<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T17, T18, T19>.takeLast3(): Tuple3 = Tuple3(this._17(), this._18(), this._19()) +fun Tuple19<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T18, T19>.takeLast2(): Tuple2 = Tuple2(this._18(), this._19()) +fun Tuple19<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T19>.takeLast1(): Tuple1 = Tuple1(this._19()) +fun Tuple20<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.takeLast0(): EmptyTuple = EmptyTuple +fun Tuple20.takeLast20(): 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 Tuple20<*, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>.takeLast19(): Tuple19 = Tuple19(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<*, *, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>.takeLast18(): Tuple18 = Tuple18(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<*, *, *, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>.takeLast17(): Tuple17 = Tuple17(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<*, *, *, *, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>.takeLast16(): Tuple16 = Tuple16(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<*, *, *, *, *, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>.takeLast15(): Tuple15 = Tuple15(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<*, *, *, *, *, *, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>.takeLast14(): Tuple14 = Tuple14(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<*, *, *, *, *, *, *, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>.takeLast13(): Tuple13 = Tuple13(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<*, *, *, *, *, *, *, *, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>.takeLast12(): Tuple12 = Tuple12(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<*, *, *, *, *, *, *, *, *, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>.takeLast11(): Tuple11 = Tuple11(this._10(), this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20()) +fun Tuple20<*, *, *, *, *, *, *, *, *, *, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>.takeLast10(): Tuple10 = Tuple10(this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20()) +fun Tuple20<*, *, *, *, *, *, *, *, *, *, *, T12, T13, T14, T15, T16, T17, T18, T19, T20>.takeLast9(): Tuple9 = Tuple9(this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20()) +fun Tuple20<*, *, *, *, *, *, *, *, *, *, *, *, T13, T14, T15, T16, T17, T18, T19, T20>.takeLast8(): Tuple8 = Tuple8(this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20()) +fun Tuple20<*, *, *, *, *, *, *, *, *, *, *, *, *, T14, T15, T16, T17, T18, T19, T20>.takeLast7(): Tuple7 = Tuple7(this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20()) +fun Tuple20<*, *, *, *, *, *, *, *, *, *, *, *, *, *, T15, T16, T17, T18, T19, T20>.takeLast6(): Tuple6 = Tuple6(this._15(), this._16(), this._17(), this._18(), this._19(), this._20()) +fun Tuple20<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T16, T17, T18, T19, T20>.takeLast5(): Tuple5 = Tuple5(this._16(), this._17(), this._18(), this._19(), this._20()) +fun Tuple20<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T17, T18, T19, T20>.takeLast4(): Tuple4 = Tuple4(this._17(), this._18(), this._19(), this._20()) +fun Tuple20<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T18, T19, T20>.takeLast3(): Tuple3 = Tuple3(this._18(), this._19(), this._20()) +fun Tuple20<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T19, T20>.takeLast2(): Tuple2 = Tuple2(this._19(), this._20()) +fun Tuple20<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T20>.takeLast1(): Tuple1 = Tuple1(this._20()) +fun Tuple21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.takeLast0(): EmptyTuple = EmptyTuple +fun Tuple21.takeLast21(): 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 Tuple21<*, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>.takeLast20(): Tuple20 = Tuple20(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<*, *, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>.takeLast19(): Tuple19 = Tuple19(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<*, *, *, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>.takeLast18(): Tuple18 = Tuple18(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<*, *, *, *, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>.takeLast17(): Tuple17 = Tuple17(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<*, *, *, *, *, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>.takeLast16(): Tuple16 = Tuple16(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<*, *, *, *, *, *, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>.takeLast15(): Tuple15 = Tuple15(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<*, *, *, *, *, *, *, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>.takeLast14(): Tuple14 = Tuple14(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<*, *, *, *, *, *, *, *, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>.takeLast13(): Tuple13 = Tuple13(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<*, *, *, *, *, *, *, *, *, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>.takeLast12(): Tuple12 = Tuple12(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<*, *, *, *, *, *, *, *, *, *, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>.takeLast11(): Tuple11 = Tuple11(this._11(), this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20(), this._21()) +fun Tuple21<*, *, *, *, *, *, *, *, *, *, *, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>.takeLast10(): Tuple10 = Tuple10(this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20(), this._21()) +fun Tuple21<*, *, *, *, *, *, *, *, *, *, *, *, T13, T14, T15, T16, T17, T18, T19, T20, T21>.takeLast9(): Tuple9 = Tuple9(this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20(), this._21()) +fun Tuple21<*, *, *, *, *, *, *, *, *, *, *, *, *, T14, T15, T16, T17, T18, T19, T20, T21>.takeLast8(): Tuple8 = Tuple8(this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20(), this._21()) +fun Tuple21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, T15, T16, T17, T18, T19, T20, T21>.takeLast7(): Tuple7 = Tuple7(this._15(), this._16(), this._17(), this._18(), this._19(), this._20(), this._21()) +fun Tuple21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T16, T17, T18, T19, T20, T21>.takeLast6(): Tuple6 = Tuple6(this._16(), this._17(), this._18(), this._19(), this._20(), this._21()) +fun Tuple21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T17, T18, T19, T20, T21>.takeLast5(): Tuple5 = Tuple5(this._17(), this._18(), this._19(), this._20(), this._21()) +fun Tuple21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T18, T19, T20, T21>.takeLast4(): Tuple4 = Tuple4(this._18(), this._19(), this._20(), this._21()) +fun Tuple21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T19, T20, T21>.takeLast3(): Tuple3 = Tuple3(this._19(), this._20(), this._21()) +fun Tuple21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T20, T21>.takeLast2(): Tuple2 = Tuple2(this._20(), this._21()) +fun Tuple21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T21>.takeLast1(): Tuple1 = Tuple1(this._21()) +fun Tuple22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.takeLast0(): EmptyTuple = EmptyTuple +fun Tuple22.takeLast22(): 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 Tuple22<*, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>.takeLast21(): Tuple21 = Tuple21(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<*, *, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>.takeLast20(): Tuple20 = Tuple20(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<*, *, *, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>.takeLast19(): Tuple19 = Tuple19(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<*, *, *, *, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>.takeLast18(): Tuple18 = Tuple18(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<*, *, *, *, *, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>.takeLast17(): Tuple17 = Tuple17(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<*, *, *, *, *, *, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>.takeLast16(): Tuple16 = Tuple16(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<*, *, *, *, *, *, *, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>.takeLast15(): Tuple15 = Tuple15(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<*, *, *, *, *, *, *, *, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>.takeLast14(): Tuple14 = Tuple14(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<*, *, *, *, *, *, *, *, *, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>.takeLast13(): Tuple13 = Tuple13(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<*, *, *, *, *, *, *, *, *, *, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>.takeLast12(): Tuple12 = Tuple12(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<*, *, *, *, *, *, *, *, *, *, *, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>.takeLast11(): Tuple11 = Tuple11(this._12(), this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20(), this._21(), this._22()) +fun Tuple22<*, *, *, *, *, *, *, *, *, *, *, *, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>.takeLast10(): Tuple10 = Tuple10(this._13(), this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20(), this._21(), this._22()) +fun Tuple22<*, *, *, *, *, *, *, *, *, *, *, *, *, T14, T15, T16, T17, T18, T19, T20, T21, T22>.takeLast9(): Tuple9 = Tuple9(this._14(), this._15(), this._16(), this._17(), this._18(), this._19(), this._20(), this._21(), this._22()) +fun Tuple22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, T15, T16, T17, T18, T19, T20, T21, T22>.takeLast8(): Tuple8 = Tuple8(this._15(), this._16(), this._17(), this._18(), this._19(), this._20(), this._21(), this._22()) +fun Tuple22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T16, T17, T18, T19, T20, T21, T22>.takeLast7(): Tuple7 = Tuple7(this._16(), this._17(), this._18(), this._19(), this._20(), this._21(), this._22()) +fun Tuple22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T17, T18, T19, T20, T21, T22>.takeLast6(): Tuple6 = Tuple6(this._17(), this._18(), this._19(), this._20(), this._21(), this._22()) +fun Tuple22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T18, T19, T20, T21, T22>.takeLast5(): Tuple5 = Tuple5(this._18(), this._19(), this._20(), this._21(), this._22()) +fun Tuple22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T19, T20, T21, T22>.takeLast4(): Tuple4 = Tuple4(this._19(), this._20(), this._21(), this._22()) +fun Tuple22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T20, T21, T22>.takeLast3(): Tuple3 = Tuple3(this._20(), this._21(), this._22()) +fun Tuple22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T21, T22>.takeLast2(): Tuple2 = Tuple2(this._21(), this._22()) +fun Tuple22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, T22>.takeLast1(): Tuple1 = Tuple1(this._22()) diff --git a/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/TupleZip.kt b/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/TupleZip.kt new file mode 100644 index 00000000..c4930996 --- /dev/null +++ b/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/TupleZip.kt @@ -0,0 +1,559 @@ +/*- + * =LICENSE= + * Kotlin Spark API: Scala Tuples in Kotlin + * ---------- + * 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.tuples + +import scala.* + +/** + * This file provides zip-functions to all Tuple variants. + * Given two tuples, `t(a1, ..., an) zip t(a1, ..., an)`, returns a tuple + * `t(t(a1, b1), ..., t(an, bn))`. If the two tuples have different sizes, + * the extra elements of the larger tuple will be disregarded. + * The result is typed as `TupleX, ..., Tuple2>`. + */ + +infix fun Tuple1<*>.zip(other: EmptyTuple): EmptyTuple = EmptyTuple +infix fun EmptyTuple.zip(other: Tuple1<*>): EmptyTuple = EmptyTuple +infix fun Tuple1.zip(other: Tuple1): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple1.zip(other: Tuple2): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple1.zip(other: Tuple3): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple1.zip(other: Tuple4): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple1.zip(other: Tuple5): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple1.zip(other: Tuple6): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple1.zip(other: Tuple7): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple1.zip(other: Tuple8): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple1.zip(other: Tuple9): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple1.zip(other: Tuple10): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple1.zip(other: Tuple11): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple1.zip(other: Tuple12): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple1.zip(other: Tuple13): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple1.zip(other: Tuple14): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple1.zip(other: Tuple15): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple1.zip(other: Tuple16): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple1.zip(other: Tuple17): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple1.zip(other: Tuple18): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple1.zip(other: Tuple19): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple1.zip(other: Tuple20): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple1.zip(other: Tuple21): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple1.zip(other: Tuple22): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple2<*, *>.zip(other: EmptyTuple): EmptyTuple = EmptyTuple +infix fun EmptyTuple.zip(other: Tuple2<*, *>): EmptyTuple = EmptyTuple +infix fun Tuple2.zip(other: Tuple1): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple2.zip(other: Tuple2): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple2.zip(other: Tuple3): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple2.zip(other: Tuple4): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple2.zip(other: Tuple5): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple2.zip(other: Tuple6): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple2.zip(other: Tuple7): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple2.zip(other: Tuple8): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple2.zip(other: Tuple9): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple2.zip(other: Tuple10): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple2.zip(other: Tuple11): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple2.zip(other: Tuple12): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple2.zip(other: Tuple13): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple2.zip(other: Tuple14): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple2.zip(other: Tuple15): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple2.zip(other: Tuple16): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple2.zip(other: Tuple17): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple2.zip(other: Tuple18): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple2.zip(other: Tuple19): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple2.zip(other: Tuple20): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple2.zip(other: Tuple21): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple2.zip(other: Tuple22): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple3<*, *, *>.zip(other: EmptyTuple): EmptyTuple = EmptyTuple +infix fun EmptyTuple.zip(other: Tuple3<*, *, *>): EmptyTuple = EmptyTuple +infix fun Tuple3.zip(other: Tuple1): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple3.zip(other: Tuple2): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple3.zip(other: Tuple3): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple3.zip(other: Tuple4): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple3.zip(other: Tuple5): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple3.zip(other: Tuple6): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple3.zip(other: Tuple7): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple3.zip(other: Tuple8): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple3.zip(other: Tuple9): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple3.zip(other: Tuple10): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple3.zip(other: Tuple11): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple3.zip(other: Tuple12): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple3.zip(other: Tuple13): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple3.zip(other: Tuple14): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple3.zip(other: Tuple15): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple3.zip(other: Tuple16): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple3.zip(other: Tuple17): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple3.zip(other: Tuple18): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple3.zip(other: Tuple19): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple3.zip(other: Tuple20): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple3.zip(other: Tuple21): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple3.zip(other: Tuple22): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple4<*, *, *, *>.zip(other: EmptyTuple): EmptyTuple = EmptyTuple +infix fun EmptyTuple.zip(other: Tuple4<*, *, *, *>): EmptyTuple = EmptyTuple +infix fun Tuple4.zip(other: Tuple1): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple4.zip(other: Tuple2): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple4.zip(other: Tuple3): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple4.zip(other: Tuple4): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple4.zip(other: Tuple5): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple4.zip(other: Tuple6): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple4.zip(other: Tuple7): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple4.zip(other: Tuple8): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple4.zip(other: Tuple9): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple4.zip(other: Tuple10): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple4.zip(other: Tuple11): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple4.zip(other: Tuple12): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple4.zip(other: Tuple13): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple4.zip(other: Tuple14): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple4.zip(other: Tuple15): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple4.zip(other: Tuple16): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple4.zip(other: Tuple17): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple4.zip(other: Tuple18): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple4.zip(other: Tuple19): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple4.zip(other: Tuple20): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple4.zip(other: Tuple21): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple4.zip(other: Tuple22): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple5<*, *, *, *, *>.zip(other: EmptyTuple): EmptyTuple = EmptyTuple +infix fun EmptyTuple.zip(other: Tuple5<*, *, *, *, *>): EmptyTuple = EmptyTuple +infix fun Tuple5.zip(other: Tuple1): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple5.zip(other: Tuple2): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple5.zip(other: Tuple3): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple5.zip(other: Tuple4): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple5.zip(other: Tuple5): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple5.zip(other: Tuple6): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple5.zip(other: Tuple7): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple5.zip(other: Tuple8): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple5.zip(other: Tuple9): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple5.zip(other: Tuple10): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple5.zip(other: Tuple11): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple5.zip(other: Tuple12): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple5.zip(other: Tuple13): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple5.zip(other: Tuple14): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple5.zip(other: Tuple15): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple5.zip(other: Tuple16): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple5.zip(other: Tuple17): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple5.zip(other: Tuple18): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple5.zip(other: Tuple19): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple5.zip(other: Tuple20): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple5.zip(other: Tuple21): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple5.zip(other: Tuple22): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple6<*, *, *, *, *, *>.zip(other: EmptyTuple): EmptyTuple = EmptyTuple +infix fun EmptyTuple.zip(other: Tuple6<*, *, *, *, *, *>): EmptyTuple = EmptyTuple +infix fun Tuple6.zip(other: Tuple1): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple6.zip(other: Tuple2): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple6.zip(other: Tuple3): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple6.zip(other: Tuple4): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple6.zip(other: Tuple5): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple6.zip(other: Tuple6): Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6())) +infix fun Tuple6.zip(other: Tuple7): Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6())) +infix fun Tuple6.zip(other: Tuple8): Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6())) +infix fun Tuple6.zip(other: Tuple9): Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6())) +infix fun Tuple6.zip(other: Tuple10): Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6())) +infix fun Tuple6.zip(other: Tuple11): Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6())) +infix fun Tuple6.zip(other: Tuple12): Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6())) +infix fun Tuple6.zip(other: Tuple13): Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6())) +infix fun Tuple6.zip(other: Tuple14): Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6())) +infix fun Tuple6.zip(other: Tuple15): Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6())) +infix fun Tuple6.zip(other: Tuple16): Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6())) +infix fun Tuple6.zip(other: Tuple17): Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6())) +infix fun Tuple6.zip(other: Tuple18): Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6())) +infix fun Tuple6.zip(other: Tuple19): Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6())) +infix fun Tuple6.zip(other: Tuple20): Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6())) +infix fun Tuple6.zip(other: Tuple21): Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6())) +infix fun Tuple6.zip(other: Tuple22): Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6())) +infix fun Tuple7<*, *, *, *, *, *, *>.zip(other: EmptyTuple): EmptyTuple = EmptyTuple +infix fun EmptyTuple.zip(other: Tuple7<*, *, *, *, *, *, *>): EmptyTuple = EmptyTuple +infix fun Tuple7.zip(other: Tuple1): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple7.zip(other: Tuple2): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple7.zip(other: Tuple3): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple7.zip(other: Tuple4): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple7.zip(other: Tuple5): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple7.zip(other: Tuple6): Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6())) +infix fun Tuple7.zip(other: Tuple7): Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7())) +infix fun Tuple7.zip(other: Tuple8): Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7())) +infix fun Tuple7.zip(other: Tuple9): Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7())) +infix fun Tuple7.zip(other: Tuple10): Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7())) +infix fun Tuple7.zip(other: Tuple11): Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7())) +infix fun Tuple7.zip(other: Tuple12): Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7())) +infix fun Tuple7.zip(other: Tuple13): Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7())) +infix fun Tuple7.zip(other: Tuple14): Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7())) +infix fun Tuple7.zip(other: Tuple15): Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7())) +infix fun Tuple7.zip(other: Tuple16): Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7())) +infix fun Tuple7.zip(other: Tuple17): Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7())) +infix fun Tuple7.zip(other: Tuple18): Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7())) +infix fun Tuple7.zip(other: Tuple19): Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7())) +infix fun Tuple7.zip(other: Tuple20): Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7())) +infix fun Tuple7.zip(other: Tuple21): Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7())) +infix fun Tuple7.zip(other: Tuple22): Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7())) +infix fun Tuple8<*, *, *, *, *, *, *, *>.zip(other: EmptyTuple): EmptyTuple = EmptyTuple +infix fun EmptyTuple.zip(other: Tuple8<*, *, *, *, *, *, *, *>): EmptyTuple = EmptyTuple +infix fun Tuple8.zip(other: Tuple1): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple8.zip(other: Tuple2): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple8.zip(other: Tuple3): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple8.zip(other: Tuple4): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple8.zip(other: Tuple5): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple8.zip(other: Tuple6): Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6())) +infix fun Tuple8.zip(other: Tuple7): Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7())) +infix fun Tuple8.zip(other: Tuple8): Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8())) +infix fun Tuple8.zip(other: Tuple9): Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8())) +infix fun Tuple8.zip(other: Tuple10): Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8())) +infix fun Tuple8.zip(other: Tuple11): Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8())) +infix fun Tuple8.zip(other: Tuple12): Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8())) +infix fun Tuple8.zip(other: Tuple13): Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8())) +infix fun Tuple8.zip(other: Tuple14): Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8())) +infix fun Tuple8.zip(other: Tuple15): Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8())) +infix fun Tuple8.zip(other: Tuple16): Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8())) +infix fun Tuple8.zip(other: Tuple17): Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8())) +infix fun Tuple8.zip(other: Tuple18): Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8())) +infix fun Tuple8.zip(other: Tuple19): Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8())) +infix fun Tuple8.zip(other: Tuple20): Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8())) +infix fun Tuple8.zip(other: Tuple21): Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8())) +infix fun Tuple8.zip(other: Tuple22): Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8())) +infix fun Tuple9<*, *, *, *, *, *, *, *, *>.zip(other: EmptyTuple): EmptyTuple = EmptyTuple +infix fun EmptyTuple.zip(other: Tuple9<*, *, *, *, *, *, *, *, *>): EmptyTuple = EmptyTuple +infix fun Tuple9.zip(other: Tuple1): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple9.zip(other: Tuple2): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple9.zip(other: Tuple3): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple9.zip(other: Tuple4): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple9.zip(other: Tuple5): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple9.zip(other: Tuple6): Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6())) +infix fun Tuple9.zip(other: Tuple7): Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7())) +infix fun Tuple9.zip(other: Tuple8): Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8())) +infix fun Tuple9.zip(other: Tuple9): Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9())) +infix fun Tuple9.zip(other: Tuple10): Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9())) +infix fun Tuple9.zip(other: Tuple11): Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9())) +infix fun Tuple9.zip(other: Tuple12): Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9())) +infix fun Tuple9.zip(other: Tuple13): Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9())) +infix fun Tuple9.zip(other: Tuple14): Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9())) +infix fun Tuple9.zip(other: Tuple15): Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9())) +infix fun Tuple9.zip(other: Tuple16): Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9())) +infix fun Tuple9.zip(other: Tuple17): Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9())) +infix fun Tuple9.zip(other: Tuple18): Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9())) +infix fun Tuple9.zip(other: Tuple19): Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9())) +infix fun Tuple9.zip(other: Tuple20): Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9())) +infix fun Tuple9.zip(other: Tuple21): Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9())) +infix fun Tuple9.zip(other: Tuple22): Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9())) +infix fun Tuple10<*, *, *, *, *, *, *, *, *, *>.zip(other: EmptyTuple): EmptyTuple = EmptyTuple +infix fun EmptyTuple.zip(other: Tuple10<*, *, *, *, *, *, *, *, *, *>): EmptyTuple = EmptyTuple +infix fun Tuple10.zip(other: Tuple1): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple10.zip(other: Tuple2): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple10.zip(other: Tuple3): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple10.zip(other: Tuple4): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple10.zip(other: Tuple5): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple10.zip(other: Tuple6): Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6())) +infix fun Tuple10.zip(other: Tuple7): Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7())) +infix fun Tuple10.zip(other: Tuple8): Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8())) +infix fun Tuple10.zip(other: Tuple9): Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9())) +infix fun Tuple10.zip(other: Tuple10): Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10())) +infix fun Tuple10.zip(other: Tuple11): Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10())) +infix fun Tuple10.zip(other: Tuple12): Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10())) +infix fun Tuple10.zip(other: Tuple13): Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10())) +infix fun Tuple10.zip(other: Tuple14): Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10())) +infix fun Tuple10.zip(other: Tuple15): Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10())) +infix fun Tuple10.zip(other: Tuple16): Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10())) +infix fun Tuple10.zip(other: Tuple17): Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10())) +infix fun Tuple10.zip(other: Tuple18): Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10())) +infix fun Tuple10.zip(other: Tuple19): Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10())) +infix fun Tuple10.zip(other: Tuple20): Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10())) +infix fun Tuple10.zip(other: Tuple21): Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10())) +infix fun Tuple10.zip(other: Tuple22): Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10())) +infix fun Tuple11<*, *, *, *, *, *, *, *, *, *, *>.zip(other: EmptyTuple): EmptyTuple = EmptyTuple +infix fun EmptyTuple.zip(other: Tuple11<*, *, *, *, *, *, *, *, *, *, *>): EmptyTuple = EmptyTuple +infix fun Tuple11.zip(other: Tuple1): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple11.zip(other: Tuple2): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple11.zip(other: Tuple3): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple11.zip(other: Tuple4): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple11.zip(other: Tuple5): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple11.zip(other: Tuple6): Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6())) +infix fun Tuple11.zip(other: Tuple7): Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7())) +infix fun Tuple11.zip(other: Tuple8): Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8())) +infix fun Tuple11.zip(other: Tuple9): Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9())) +infix fun Tuple11.zip(other: Tuple10): Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10())) +infix fun Tuple11.zip(other: Tuple11): Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11())) +infix fun Tuple11.zip(other: Tuple12): Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11())) +infix fun Tuple11.zip(other: Tuple13): Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11())) +infix fun Tuple11.zip(other: Tuple14): Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11())) +infix fun Tuple11.zip(other: Tuple15): Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11())) +infix fun Tuple11.zip(other: Tuple16): Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11())) +infix fun Tuple11.zip(other: Tuple17): Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11())) +infix fun Tuple11.zip(other: Tuple18): Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11())) +infix fun Tuple11.zip(other: Tuple19): Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11())) +infix fun Tuple11.zip(other: Tuple20): Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11())) +infix fun Tuple11.zip(other: Tuple21): Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11())) +infix fun Tuple11.zip(other: Tuple22): Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11())) +infix fun Tuple12<*, *, *, *, *, *, *, *, *, *, *, *>.zip(other: EmptyTuple): EmptyTuple = EmptyTuple +infix fun EmptyTuple.zip(other: Tuple12<*, *, *, *, *, *, *, *, *, *, *, *>): EmptyTuple = EmptyTuple +infix fun Tuple12.zip(other: Tuple1): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple12.zip(other: Tuple2): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple12.zip(other: Tuple3): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple12.zip(other: Tuple4): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple12.zip(other: Tuple5): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple12.zip(other: Tuple6): Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6())) +infix fun Tuple12.zip(other: Tuple7): Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7())) +infix fun Tuple12.zip(other: Tuple8): Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8())) +infix fun Tuple12.zip(other: Tuple9): Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9())) +infix fun Tuple12.zip(other: Tuple10): Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10())) +infix fun Tuple12.zip(other: Tuple11): Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11())) +infix fun Tuple12.zip(other: Tuple12): Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12())) +infix fun Tuple12.zip(other: Tuple13): Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12())) +infix fun Tuple12.zip(other: Tuple14): Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12())) +infix fun Tuple12.zip(other: Tuple15): Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12())) +infix fun Tuple12.zip(other: Tuple16): Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12())) +infix fun Tuple12.zip(other: Tuple17): Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12())) +infix fun Tuple12.zip(other: Tuple18): Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12())) +infix fun Tuple12.zip(other: Tuple19): Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12())) +infix fun Tuple12.zip(other: Tuple20): Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12())) +infix fun Tuple12.zip(other: Tuple21): Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12())) +infix fun Tuple12.zip(other: Tuple22): Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12())) +infix fun Tuple13<*, *, *, *, *, *, *, *, *, *, *, *, *>.zip(other: EmptyTuple): EmptyTuple = EmptyTuple +infix fun EmptyTuple.zip(other: Tuple13<*, *, *, *, *, *, *, *, *, *, *, *, *>): EmptyTuple = EmptyTuple +infix fun Tuple13.zip(other: Tuple1): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple13.zip(other: Tuple2): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple13.zip(other: Tuple3): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple13.zip(other: Tuple4): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple13.zip(other: Tuple5): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple13.zip(other: Tuple6): Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6())) +infix fun Tuple13.zip(other: Tuple7): Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7())) +infix fun Tuple13.zip(other: Tuple8): Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8())) +infix fun Tuple13.zip(other: Tuple9): Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9())) +infix fun Tuple13.zip(other: Tuple10): Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10())) +infix fun Tuple13.zip(other: Tuple11): Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11())) +infix fun Tuple13.zip(other: Tuple12): Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12())) +infix fun Tuple13.zip(other: Tuple13): Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13())) +infix fun Tuple13.zip(other: Tuple14): Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13())) +infix fun Tuple13.zip(other: Tuple15): Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13())) +infix fun Tuple13.zip(other: Tuple16): Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13())) +infix fun Tuple13.zip(other: Tuple17): Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13())) +infix fun Tuple13.zip(other: Tuple18): Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13())) +infix fun Tuple13.zip(other: Tuple19): Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13())) +infix fun Tuple13.zip(other: Tuple20): Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13())) +infix fun Tuple13.zip(other: Tuple21): Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13())) +infix fun Tuple13.zip(other: Tuple22): Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13())) +infix fun Tuple14<*, *, *, *, *, *, *, *, *, *, *, *, *, *>.zip(other: EmptyTuple): EmptyTuple = EmptyTuple +infix fun EmptyTuple.zip(other: Tuple14<*, *, *, *, *, *, *, *, *, *, *, *, *, *>): EmptyTuple = EmptyTuple +infix fun Tuple14.zip(other: Tuple1): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple14.zip(other: Tuple2): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple14.zip(other: Tuple3): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple14.zip(other: Tuple4): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple14.zip(other: Tuple5): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple14.zip(other: Tuple6): Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6())) +infix fun Tuple14.zip(other: Tuple7): Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7())) +infix fun Tuple14.zip(other: Tuple8): Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8())) +infix fun Tuple14.zip(other: Tuple9): Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9())) +infix fun Tuple14.zip(other: Tuple10): Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10())) +infix fun Tuple14.zip(other: Tuple11): Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11())) +infix fun Tuple14.zip(other: Tuple12): Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12())) +infix fun Tuple14.zip(other: Tuple13): Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13())) +infix fun Tuple14.zip(other: Tuple14): Tuple14, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple14, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14())) +infix fun Tuple14.zip(other: Tuple15): Tuple14, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple14, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14())) +infix fun Tuple14.zip(other: Tuple16): Tuple14, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple14, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14())) +infix fun Tuple14.zip(other: Tuple17): Tuple14, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple14, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14())) +infix fun Tuple14.zip(other: Tuple18): Tuple14, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple14, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14())) +infix fun Tuple14.zip(other: Tuple19): Tuple14, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple14, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14())) +infix fun Tuple14.zip(other: Tuple20): Tuple14, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple14, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14())) +infix fun Tuple14.zip(other: Tuple21): Tuple14, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple14, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14())) +infix fun Tuple14.zip(other: Tuple22): Tuple14, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple14, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14())) +infix fun Tuple15<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.zip(other: EmptyTuple): EmptyTuple = EmptyTuple +infix fun EmptyTuple.zip(other: Tuple15<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *>): EmptyTuple = EmptyTuple +infix fun Tuple15.zip(other: Tuple1): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple15.zip(other: Tuple2): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple15.zip(other: Tuple3): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple15.zip(other: Tuple4): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple15.zip(other: Tuple5): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple15.zip(other: Tuple6): Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6())) +infix fun Tuple15.zip(other: Tuple7): Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7())) +infix fun Tuple15.zip(other: Tuple8): Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8())) +infix fun Tuple15.zip(other: Tuple9): Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9())) +infix fun Tuple15.zip(other: Tuple10): Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10())) +infix fun Tuple15.zip(other: Tuple11): Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11())) +infix fun Tuple15.zip(other: Tuple12): Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12())) +infix fun Tuple15.zip(other: Tuple13): Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13())) +infix fun Tuple15.zip(other: Tuple14): Tuple14, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple14, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14())) +infix fun Tuple15.zip(other: Tuple15): Tuple15, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple15, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15())) +infix fun Tuple15.zip(other: Tuple16): Tuple15, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple15, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15())) +infix fun Tuple15.zip(other: Tuple17): Tuple15, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple15, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15())) +infix fun Tuple15.zip(other: Tuple18): Tuple15, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple15, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15())) +infix fun Tuple15.zip(other: Tuple19): Tuple15, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple15, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15())) +infix fun Tuple15.zip(other: Tuple20): Tuple15, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple15, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15())) +infix fun Tuple15.zip(other: Tuple21): Tuple15, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple15, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15())) +infix fun Tuple15.zip(other: Tuple22): Tuple15, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple15, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15())) +infix fun Tuple16<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.zip(other: EmptyTuple): EmptyTuple = EmptyTuple +infix fun EmptyTuple.zip(other: Tuple16<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>): EmptyTuple = EmptyTuple +infix fun Tuple16.zip(other: Tuple1): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple16.zip(other: Tuple2): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple16.zip(other: Tuple3): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple16.zip(other: Tuple4): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple16.zip(other: Tuple5): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple16.zip(other: Tuple6): Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6())) +infix fun Tuple16.zip(other: Tuple7): Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7())) +infix fun Tuple16.zip(other: Tuple8): Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8())) +infix fun Tuple16.zip(other: Tuple9): Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9())) +infix fun Tuple16.zip(other: Tuple10): Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10())) +infix fun Tuple16.zip(other: Tuple11): Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11())) +infix fun Tuple16.zip(other: Tuple12): Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12())) +infix fun Tuple16.zip(other: Tuple13): Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13())) +infix fun Tuple16.zip(other: Tuple14): Tuple14, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple14, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14())) +infix fun Tuple16.zip(other: Tuple15): Tuple15, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple15, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15())) +infix fun Tuple16.zip(other: Tuple16): Tuple16, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple16, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16())) +infix fun Tuple16.zip(other: Tuple17): Tuple16, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple16, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16())) +infix fun Tuple16.zip(other: Tuple18): Tuple16, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple16, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16())) +infix fun Tuple16.zip(other: Tuple19): Tuple16, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple16, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16())) +infix fun Tuple16.zip(other: Tuple20): Tuple16, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple16, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16())) +infix fun Tuple16.zip(other: Tuple21): Tuple16, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple16, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16())) +infix fun Tuple16.zip(other: Tuple22): Tuple16, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple16, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16())) +infix fun Tuple17<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.zip(other: EmptyTuple): EmptyTuple = EmptyTuple +infix fun EmptyTuple.zip(other: Tuple17<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>): EmptyTuple = EmptyTuple +infix fun Tuple17.zip(other: Tuple1): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple17.zip(other: Tuple2): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple17.zip(other: Tuple3): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple17.zip(other: Tuple4): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple17.zip(other: Tuple5): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple17.zip(other: Tuple6): Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6())) +infix fun Tuple17.zip(other: Tuple7): Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7())) +infix fun Tuple17.zip(other: Tuple8): Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8())) +infix fun Tuple17.zip(other: Tuple9): Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9())) +infix fun Tuple17.zip(other: Tuple10): Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10())) +infix fun Tuple17.zip(other: Tuple11): Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11())) +infix fun Tuple17.zip(other: Tuple12): Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12())) +infix fun Tuple17.zip(other: Tuple13): Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13())) +infix fun Tuple17.zip(other: Tuple14): Tuple14, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple14, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14())) +infix fun Tuple17.zip(other: Tuple15): Tuple15, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple15, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15())) +infix fun Tuple17.zip(other: Tuple16): Tuple16, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple16, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16())) +infix fun Tuple17.zip(other: Tuple17): Tuple17, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple17, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17())) +infix fun Tuple17.zip(other: Tuple18): Tuple17, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple17, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17())) +infix fun Tuple17.zip(other: Tuple19): Tuple17, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple17, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17())) +infix fun Tuple17.zip(other: Tuple20): Tuple17, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple17, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17())) +infix fun Tuple17.zip(other: Tuple21): Tuple17, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple17, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17())) +infix fun Tuple17.zip(other: Tuple22): Tuple17, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple17, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17())) +infix fun Tuple18<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.zip(other: EmptyTuple): EmptyTuple = EmptyTuple +infix fun EmptyTuple.zip(other: Tuple18<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>): EmptyTuple = EmptyTuple +infix fun Tuple18.zip(other: Tuple1): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple18.zip(other: Tuple2): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple18.zip(other: Tuple3): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple18.zip(other: Tuple4): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple18.zip(other: Tuple5): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple18.zip(other: Tuple6): Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6())) +infix fun Tuple18.zip(other: Tuple7): Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7())) +infix fun Tuple18.zip(other: Tuple8): Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8())) +infix fun Tuple18.zip(other: Tuple9): Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9())) +infix fun Tuple18.zip(other: Tuple10): Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10())) +infix fun Tuple18.zip(other: Tuple11): Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11())) +infix fun Tuple18.zip(other: Tuple12): Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12())) +infix fun Tuple18.zip(other: Tuple13): Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13())) +infix fun Tuple18.zip(other: Tuple14): Tuple14, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple14, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14())) +infix fun Tuple18.zip(other: Tuple15): Tuple15, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple15, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15())) +infix fun Tuple18.zip(other: Tuple16): Tuple16, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple16, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16())) +infix fun Tuple18.zip(other: Tuple17): Tuple17, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple17, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17())) +infix fun Tuple18.zip(other: Tuple18): Tuple18, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple18, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17()), Tuple2(this._18(), other._18())) +infix fun Tuple18.zip(other: Tuple19): Tuple18, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple18, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17()), Tuple2(this._18(), other._18())) +infix fun Tuple18.zip(other: Tuple20): Tuple18, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple18, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17()), Tuple2(this._18(), other._18())) +infix fun Tuple18.zip(other: Tuple21): Tuple18, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple18, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17()), Tuple2(this._18(), other._18())) +infix fun Tuple18.zip(other: Tuple22): Tuple18, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple18, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17()), Tuple2(this._18(), other._18())) +infix fun Tuple19<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.zip(other: EmptyTuple): EmptyTuple = EmptyTuple +infix fun EmptyTuple.zip(other: Tuple19<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>): EmptyTuple = EmptyTuple +infix fun Tuple19.zip(other: Tuple1): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple19.zip(other: Tuple2): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple19.zip(other: Tuple3): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple19.zip(other: Tuple4): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple19.zip(other: Tuple5): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple19.zip(other: Tuple6): Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6())) +infix fun Tuple19.zip(other: Tuple7): Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7())) +infix fun Tuple19.zip(other: Tuple8): Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8())) +infix fun Tuple19.zip(other: Tuple9): Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9())) +infix fun Tuple19.zip(other: Tuple10): Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10())) +infix fun Tuple19.zip(other: Tuple11): Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11())) +infix fun Tuple19.zip(other: Tuple12): Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12())) +infix fun Tuple19.zip(other: Tuple13): Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13())) +infix fun Tuple19.zip(other: Tuple14): Tuple14, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple14, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14())) +infix fun Tuple19.zip(other: Tuple15): Tuple15, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple15, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15())) +infix fun Tuple19.zip(other: Tuple16): Tuple16, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple16, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16())) +infix fun Tuple19.zip(other: Tuple17): Tuple17, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple17, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17())) +infix fun Tuple19.zip(other: Tuple18): Tuple18, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple18, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17()), Tuple2(this._18(), other._18())) +infix fun Tuple19.zip(other: Tuple19): Tuple19, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple19, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17()), Tuple2(this._18(), other._18()), Tuple2(this._19(), other._19())) +infix fun Tuple19.zip(other: Tuple20): Tuple19, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple19, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17()), Tuple2(this._18(), other._18()), Tuple2(this._19(), other._19())) +infix fun Tuple19.zip(other: Tuple21): Tuple19, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple19, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17()), Tuple2(this._18(), other._18()), Tuple2(this._19(), other._19())) +infix fun Tuple19.zip(other: Tuple22): Tuple19, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple19, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17()), Tuple2(this._18(), other._18()), Tuple2(this._19(), other._19())) +infix fun Tuple20<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.zip(other: EmptyTuple): EmptyTuple = EmptyTuple +infix fun EmptyTuple.zip(other: Tuple20<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>): EmptyTuple = EmptyTuple +infix fun Tuple20.zip(other: Tuple1): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple20.zip(other: Tuple2): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple20.zip(other: Tuple3): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple20.zip(other: Tuple4): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple20.zip(other: Tuple5): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple20.zip(other: Tuple6): Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6())) +infix fun Tuple20.zip(other: Tuple7): Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7())) +infix fun Tuple20.zip(other: Tuple8): Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8())) +infix fun Tuple20.zip(other: Tuple9): Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9())) +infix fun Tuple20.zip(other: Tuple10): Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10())) +infix fun Tuple20.zip(other: Tuple11): Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11())) +infix fun Tuple20.zip(other: Tuple12): Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12())) +infix fun Tuple20.zip(other: Tuple13): Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13())) +infix fun Tuple20.zip(other: Tuple14): Tuple14, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple14, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14())) +infix fun Tuple20.zip(other: Tuple15): Tuple15, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple15, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15())) +infix fun Tuple20.zip(other: Tuple16): Tuple16, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple16, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16())) +infix fun Tuple20.zip(other: Tuple17): Tuple17, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple17, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17())) +infix fun Tuple20.zip(other: Tuple18): Tuple18, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple18, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17()), Tuple2(this._18(), other._18())) +infix fun Tuple20.zip(other: Tuple19): Tuple19, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple19, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17()), Tuple2(this._18(), other._18()), Tuple2(this._19(), other._19())) +infix fun Tuple20.zip(other: Tuple20): Tuple20, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple20, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17()), Tuple2(this._18(), other._18()), Tuple2(this._19(), other._19()), Tuple2(this._20(), other._20())) +infix fun Tuple20.zip(other: Tuple21): Tuple20, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple20, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17()), Tuple2(this._18(), other._18()), Tuple2(this._19(), other._19()), Tuple2(this._20(), other._20())) +infix fun Tuple20.zip(other: Tuple22): Tuple20, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple20, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17()), Tuple2(this._18(), other._18()), Tuple2(this._19(), other._19()), Tuple2(this._20(), other._20())) +infix fun Tuple21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.zip(other: EmptyTuple): EmptyTuple = EmptyTuple +infix fun EmptyTuple.zip(other: Tuple21<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>): EmptyTuple = EmptyTuple +infix fun Tuple21.zip(other: Tuple1): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple21.zip(other: Tuple2): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple21.zip(other: Tuple3): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple21.zip(other: Tuple4): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple21.zip(other: Tuple5): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple21.zip(other: Tuple6): Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6())) +infix fun Tuple21.zip(other: Tuple7): Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7())) +infix fun Tuple21.zip(other: Tuple8): Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8())) +infix fun Tuple21.zip(other: Tuple9): Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9())) +infix fun Tuple21.zip(other: Tuple10): Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10())) +infix fun Tuple21.zip(other: Tuple11): Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11())) +infix fun Tuple21.zip(other: Tuple12): Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12())) +infix fun Tuple21.zip(other: Tuple13): Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13())) +infix fun Tuple21.zip(other: Tuple14): Tuple14, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple14, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14())) +infix fun Tuple21.zip(other: Tuple15): Tuple15, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple15, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15())) +infix fun Tuple21.zip(other: Tuple16): Tuple16, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple16, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16())) +infix fun Tuple21.zip(other: Tuple17): Tuple17, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple17, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17())) +infix fun Tuple21.zip(other: Tuple18): Tuple18, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple18, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17()), Tuple2(this._18(), other._18())) +infix fun Tuple21.zip(other: Tuple19): Tuple19, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple19, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17()), Tuple2(this._18(), other._18()), Tuple2(this._19(), other._19())) +infix fun Tuple21.zip(other: Tuple20): Tuple20, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple20, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17()), Tuple2(this._18(), other._18()), Tuple2(this._19(), other._19()), Tuple2(this._20(), other._20())) +infix fun Tuple21.zip(other: Tuple21): Tuple21, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple21, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17()), Tuple2(this._18(), other._18()), Tuple2(this._19(), other._19()), Tuple2(this._20(), other._20()), Tuple2(this._21(), other._21())) +infix fun Tuple21.zip(other: Tuple22): Tuple21, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple21, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17()), Tuple2(this._18(), other._18()), Tuple2(this._19(), other._19()), Tuple2(this._20(), other._20()), Tuple2(this._21(), other._21())) +infix fun Tuple22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>.zip(other: EmptyTuple): EmptyTuple = EmptyTuple +infix fun EmptyTuple.zip(other: Tuple22<*, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *, *>): EmptyTuple = EmptyTuple +infix fun Tuple22.zip(other: Tuple1): Tuple1> = Tuple1>(Tuple2(this._1(), other._1())) +infix fun Tuple22.zip(other: Tuple2): Tuple2, Tuple2> = Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2())) +infix fun Tuple22.zip(other: Tuple3): Tuple3, Tuple2, Tuple2> = Tuple3, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3())) +infix fun Tuple22.zip(other: Tuple4): Tuple4, Tuple2, Tuple2, Tuple2> = Tuple4, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4())) +infix fun Tuple22.zip(other: Tuple5): Tuple5, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple5, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5())) +infix fun Tuple22.zip(other: Tuple6): Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple6, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6())) +infix fun Tuple22.zip(other: Tuple7): Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple7, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7())) +infix fun Tuple22.zip(other: Tuple8): Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple8, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8())) +infix fun Tuple22.zip(other: Tuple9): Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple9, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9())) +infix fun Tuple22.zip(other: Tuple10): Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple10, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10())) +infix fun Tuple22.zip(other: Tuple11): Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple11, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11())) +infix fun Tuple22.zip(other: Tuple12): Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple12, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12())) +infix fun Tuple22.zip(other: Tuple13): Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple13, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13())) +infix fun Tuple22.zip(other: Tuple14): Tuple14, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple14, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14())) +infix fun Tuple22.zip(other: Tuple15): Tuple15, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple15, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15())) +infix fun Tuple22.zip(other: Tuple16): Tuple16, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple16, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16())) +infix fun Tuple22.zip(other: Tuple17): Tuple17, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple17, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17())) +infix fun Tuple22.zip(other: Tuple18): Tuple18, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple18, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17()), Tuple2(this._18(), other._18())) +infix fun Tuple22.zip(other: Tuple19): Tuple19, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple19, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17()), Tuple2(this._18(), other._18()), Tuple2(this._19(), other._19())) +infix fun Tuple22.zip(other: Tuple20): Tuple20, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple20, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17()), Tuple2(this._18(), other._18()), Tuple2(this._19(), other._19()), Tuple2(this._20(), other._20())) +infix fun Tuple22.zip(other: Tuple21): Tuple21, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple21, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17()), Tuple2(this._18(), other._18()), Tuple2(this._19(), other._19()), Tuple2(this._20(), other._20()), Tuple2(this._21(), other._21())) +infix fun Tuple22.zip(other: Tuple22): Tuple22, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2> = Tuple22, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2, Tuple2>(Tuple2(this._1(), other._1()), Tuple2(this._2(), other._2()), Tuple2(this._3(), other._3()), Tuple2(this._4(), other._4()), Tuple2(this._5(), other._5()), Tuple2(this._6(), other._6()), Tuple2(this._7(), other._7()), Tuple2(this._8(), other._8()), Tuple2(this._9(), other._9()), Tuple2(this._10(), other._10()), Tuple2(this._11(), other._11()), Tuple2(this._12(), other._12()), Tuple2(this._13(), other._13()), Tuple2(this._14(), other._14()), Tuple2(this._15(), other._15()), Tuple2(this._16(), other._16()), Tuple2(this._17(), other._17()), Tuple2(this._18(), other._18()), Tuple2(this._19(), other._19()), Tuple2(this._20(), other._20()), Tuple2(this._21(), other._21()), Tuple2(this._22(), other._22())) diff --git a/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/TypedProductExtensions.kt b/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/TypedProductExtensions.kt new file mode 100644 index 00000000..1f139cb1 --- /dev/null +++ b/scala-tuples-in-kotlin/src/main/kotlin/org/jetbrains/kotlinx/spark/api/tuples/TypedProductExtensions.kt @@ -0,0 +1,1047 @@ +/*- + * =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("UNCHECKED_CAST", "RemoveExplicitTypeArguments") + +package org.jetbrains.kotlinx.spark.api.tuples + +import scala.Product1 +import scala.Product2 +import scala.Product3 +import scala.Product4 +import scala.Product5 +import scala.Product6 +import scala.Product7 +import scala.Product8 +import scala.Product9 +import scala.Product10 +import scala.Product11 +import scala.Product12 +import scala.Product13 +import scala.Product14 +import scala.Product15 +import scala.Product16 +import scala.Product17 +import scala.Product18 +import scala.Product19 +import scala.Product20 +import scala.Product21 +import scala.Product22 +import scala.collection.JavaConverters + +/** + * This file provides quality of life extensions for Products/Tuples where each of its types is the same. + * This includes converting to [Iterable] or getting an [Iterator] of a Product/Tuple, + * as well as taking a single value or slice from a Tuple/Product. + * + */ + +/** Allows this product to be iterated over. Returns an iterator of type [T]. */ +operator fun Product1.iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) + +/** Allows this product to be iterated over. Returns an iterator of type [T]. */ +operator fun Product2.iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) + +/** Allows this product to be iterated over. Returns an iterator of type [T]. */ +operator fun Product3.iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) + +/** Allows this product to be iterated over. Returns an iterator of type [T]. */ +operator fun Product4.iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) + +/** Allows this product to be iterated over. Returns an iterator of type [T]. */ +operator fun Product5.iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) + +/** Allows this product to be iterated over. Returns an iterator of type [T]. */ +operator fun Product6.iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) + +/** Allows this product to be iterated over. Returns an iterator of type [T]. */ +operator fun Product7.iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) + +/** Allows this product to be iterated over. Returns an iterator of type [T]. */ +operator fun Product8.iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) + +/** Allows this product to be iterated over. Returns an iterator of type [T]. */ +operator fun Product9.iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) + +/** Allows this product to be iterated over. Returns an iterator of type [T]. */ +operator fun Product10.iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) + +/** Allows this product to be iterated over. Returns an iterator of type [T]. */ +operator fun Product11.iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) + +/** Allows this product to be iterated over. Returns an iterator of type [T]. */ +operator fun Product12.iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) + +/** Allows this product to be iterated over. Returns an iterator of type [T]. */ +operator fun Product13.iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) + +/** Allows this product to be iterated over. Returns an iterator of type [T]. */ +operator fun Product14.iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) + +/** Allows this product to be iterated over. Returns an iterator of type [T]. */ +operator fun Product15.iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) + +/** Allows this product to be iterated over. Returns an iterator of type [T]. */ +operator fun Product16.iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) + +/** Allows this product to be iterated over. Returns an iterator of type [T]. */ +operator fun Product17.iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) + +/** Allows this product to be iterated over. Returns an iterator of type [T]. */ +operator fun Product18.iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) + +/** Allows this product to be iterated over. Returns an iterator of type [T]. */ +operator fun Product19.iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) + +/** Allows this product to be iterated over. Returns an iterator of type [T]. */ +operator fun Product20.iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) + +/** Allows this product to be iterated over. Returns an iterator of type [T]. */ +operator fun Product21.iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) + +/** Allows this product to be iterated over. Returns an iterator of type [T]. */ +operator fun Product22.iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) + +/** Returns this product as an iterable of type [T]. */ +fun Product1.asIterable(): Iterable = object : Iterable { override fun iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) } + +/** Returns this product as an iterable of type [T]. */ +fun Product2.asIterable(): Iterable = object : Iterable { override fun iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) } + +/** Returns this product as an iterable of type [T]. */ +fun Product3.asIterable(): Iterable = object : Iterable { override fun iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) } + +/** Returns this product as an iterable of type [T]. */ +fun Product4.asIterable(): Iterable = object : Iterable { override fun iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) } + +/** Returns this product as an iterable of type [T]. */ +fun Product5.asIterable(): Iterable = object : Iterable { override fun iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) } + +/** Returns this product as an iterable of type [T]. */ +fun Product6.asIterable(): Iterable = object : Iterable { override fun iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) } + +/** Returns this product as an iterable of type [T]. */ +fun Product7.asIterable(): Iterable = object : Iterable { override fun iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) } + +/** Returns this product as an iterable of type [T]. */ +fun Product8.asIterable(): Iterable = object : Iterable { override fun iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) } + +/** Returns this product as an iterable of type [T]. */ +fun Product9.asIterable(): Iterable = object : Iterable { override fun iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) } + +/** Returns this product as an iterable of type [T]. */ +fun Product10.asIterable(): Iterable = object : Iterable { override fun iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) } + +/** Returns this product as an iterable of type [T]. */ +fun Product11.asIterable(): Iterable = object : Iterable { override fun iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) } + +/** Returns this product as an iterable of type [T]. */ +fun Product12.asIterable(): Iterable = object : Iterable { override fun iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) } + +/** Returns this product as an iterable of type [T]. */ +fun Product13.asIterable(): Iterable = object : Iterable { override fun iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) } + +/** Returns this product as an iterable of type [T]. */ +fun Product14.asIterable(): Iterable = object : Iterable { override fun iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) } + +/** Returns this product as an iterable of type [T]. */ +fun Product15.asIterable(): Iterable = object : Iterable { override fun iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) } + +/** Returns this product as an iterable of type [T]. */ +fun Product16.asIterable(): Iterable = object : Iterable { override fun iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) } + +/** Returns this product as an iterable of type [T]. */ +fun Product17.asIterable(): Iterable = object : Iterable { override fun iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) } + +/** Returns this product as an iterable of type [T]. */ +fun Product18.asIterable(): Iterable = object : Iterable { override fun iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) } + +/** Returns this product as an iterable of type [T]. */ +fun Product19.asIterable(): Iterable = object : Iterable { override fun iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) } + +/** Returns this product as an iterable of type [T]. */ +fun Product20.asIterable(): Iterable = object : Iterable { override fun iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) } + +/** Returns this product as an iterable of type [T]. */ +fun Product21.asIterable(): Iterable = object : Iterable { override fun iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) } + +/** Returns this product as an iterable of type [T]. */ +fun Product22.asIterable(): Iterable = object : Iterable { override fun iterator(): Iterator = JavaConverters.asJavaIterator(productIterator().map { it as T }) } + +/** Returns list of type [T] for this product. */ +fun Product1.toList(): List = listOf(this._1()) + +/** Returns list of type [T] for this product. */ +fun Product2.toList(): List = listOf(this._1(), this._2()) + +/** Returns list of type [T] for this product. */ +fun Product3.toList(): List = listOf(this._1(), this._2(), this._3()) + +/** Returns list of type [T] for this product. */ +fun Product4.toList(): List = listOf(this._1(), this._2(), this._3(), this._4()) + +/** Returns list of type [T] for this product. */ +fun Product5.toList(): List = listOf(this._1(), this._2(), this._3(), this._4(), this._5()) + +/** Returns list of type [T] for this product. */ +fun Product6.toList(): List = listOf(this._1(), this._2(), this._3(), this._4(), this._5(), this._6()) + +/** Returns list of type [T] for this product. */ +fun Product7.toList(): List = listOf(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7()) + +/** Returns list of type [T] for this product. */ +fun Product8.toList(): List = listOf(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8()) + +/** Returns list of type [T] for this product. */ +fun Product9.toList(): List = listOf(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9()) + +/** Returns list of type [T] for this product. */ +fun Product10.toList(): List = listOf(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10()) + +/** Returns list of type [T] for this product. */ +fun Product11.toList(): List = listOf(this._1(), this._2(), this._3(), this._4(), this._5(), this._6(), this._7(), this._8(), this._9(), this._10(), this._11()) + +/** Returns list of type [T] for this product. */ +fun Product12.toList(): List = listOf(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 list of type [T] for this product. */ +fun Product13.toList(): List = listOf(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 list of type [T] for this product. */ +fun Product14.toList(): List = listOf(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 list of type [T] for this product. */ +fun Product15.toList(): List = listOf(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 list of type [T] for this product. */ +fun Product16.toList(): List = listOf(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 list of type [T] for this product. */ +fun Product17.toList(): List = listOf(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 list of type [T] for this product. */ +fun Product18.toList(): List = listOf(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 list of type [T] for this product. */ +fun Product19.toList(): List = listOf(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 list of type [T] for this product. */ +fun Product20.toList(): List = listOf(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 list of type [T] for this product. */ +fun Product21.toList(): List = listOf(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 list of type [T] for this product. */ +fun Product22.toList(): List = listOf(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()) + + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @throws IndexOutOfBoundsException + * @return the element `n` elements after the first element + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product1.get(n: Int): T = productElement(n) as T + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @throws IndexOutOfBoundsException + * @return the element `n` elements after the first element + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product2.get(n: Int): T = productElement(n) as T + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @throws IndexOutOfBoundsException + * @return the element `n` elements after the first element + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product3.get(n: Int): T = productElement(n) as T + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @throws IndexOutOfBoundsException + * @return the element `n` elements after the first element + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product4.get(n: Int): T = productElement(n) as T + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @throws IndexOutOfBoundsException + * @return the element `n` elements after the first element + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product5.get(n: Int): T = productElement(n) as T + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @throws IndexOutOfBoundsException + * @return the element `n` elements after the first element + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product6.get(n: Int): T = productElement(n) as T + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @throws IndexOutOfBoundsException + * @return the element `n` elements after the first element + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product7.get(n: Int): T = productElement(n) as T + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @throws IndexOutOfBoundsException + * @return the element `n` elements after the first element + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product8.get(n: Int): T = productElement(n) as T + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @throws IndexOutOfBoundsException + * @return the element `n` elements after the first element + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product9.get(n: Int): T = productElement(n) as T + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @throws IndexOutOfBoundsException + * @return the element `n` elements after the first element + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product10.get(n: Int): T = productElement(n) as T + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @throws IndexOutOfBoundsException + * @return the element `n` elements after the first element + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product11.get(n: Int): T = productElement(n) as T + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @throws IndexOutOfBoundsException + * @return the element `n` elements after the first element + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product12.get(n: Int): T = productElement(n) as T + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @throws IndexOutOfBoundsException + * @return the element `n` elements after the first element + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product13.get(n: Int): T = productElement(n) as T + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @throws IndexOutOfBoundsException + * @return the element `n` elements after the first element + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product14.get(n: Int): T = productElement(n) as T + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @throws IndexOutOfBoundsException + * @return the element `n` elements after the first element + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product15.get(n: Int): T = productElement(n) as T + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @throws IndexOutOfBoundsException + * @return the element `n` elements after the first element + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product16.get(n: Int): T = productElement(n) as T + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @throws IndexOutOfBoundsException + * @return the element `n` elements after the first element + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product17.get(n: Int): T = productElement(n) as T + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @throws IndexOutOfBoundsException + * @return the element `n` elements after the first element + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product18.get(n: Int): T = productElement(n) as T + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @throws IndexOutOfBoundsException + * @return the element `n` elements after the first element + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product19.get(n: Int): T = productElement(n) as T + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @throws IndexOutOfBoundsException + * @return the element `n` elements after the first element + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product20.get(n: Int): T = productElement(n) as T + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @throws IndexOutOfBoundsException + * @return the element `n` elements after the first element + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product21.get(n: Int): T = productElement(n) as T + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @throws IndexOutOfBoundsException + * @return the element `n` elements after the first element + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product22.get(n: Int): T = productElement(n) as T + + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @return the element `n` elements after the first element, `null` if out of bounds + */ +fun Product1.getOrNull(n: Int): T? = (if (n in 0 until size) productElement(n) as T else null) + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @return the element `n` elements after the first element, `null` if out of bounds + */ +fun Product2.getOrNull(n: Int): T? = (if (n in 0 until size) productElement(n) as T else null) + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @return the element `n` elements after the first element, `null` if out of bounds + */ +fun Product3.getOrNull(n: Int): T? = (if (n in 0 until size) productElement(n) as T else null) + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @return the element `n` elements after the first element, `null` if out of bounds + */ +fun Product4.getOrNull(n: Int): T? = (if (n in 0 until size) productElement(n) as T else null) + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @return the element `n` elements after the first element, `null` if out of bounds + */ +fun Product5.getOrNull(n: Int): T? = (if (n in 0 until size) productElement(n) as T else null) + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @return the element `n` elements after the first element, `null` if out of bounds + */ +fun Product6.getOrNull(n: Int): T? = (if (n in 0 until size) productElement(n) as T else null) + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @return the element `n` elements after the first element, `null` if out of bounds + */ +fun Product7.getOrNull(n: Int): T? = (if (n in 0 until size) productElement(n) as T else null) + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @return the element `n` elements after the first element, `null` if out of bounds + */ +fun Product8.getOrNull(n: Int): T? = (if (n in 0 until size) productElement(n) as T else null) + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @return the element `n` elements after the first element, `null` if out of bounds + */ +fun Product9.getOrNull(n: Int): T? = (if (n in 0 until size) productElement(n) as T else null) + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @return the element `n` elements after the first element, `null` if out of bounds + */ +fun Product10.getOrNull(n: Int): T? = (if (n in 0 until size) productElement(n) as T else null) + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @return the element `n` elements after the first element, `null` if out of bounds + */ +fun Product11.getOrNull(n: Int): T? = (if (n in 0 until size) productElement(n) as T else null) + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @return the element `n` elements after the first element, `null` if out of bounds + */ +fun Product12.getOrNull(n: Int): T? = (if (n in 0 until size) productElement(n) as T else null) + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @return the element `n` elements after the first element, `null` if out of bounds + */ +fun Product13.getOrNull(n: Int): T? = (if (n in 0 until size) productElement(n) as T else null) + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @return the element `n` elements after the first element, `null` if out of bounds + */ +fun Product14.getOrNull(n: Int): T? = (if (n in 0 until size) productElement(n) as T else null) + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @return the element `n` elements after the first element, `null` if out of bounds + */ +fun Product15.getOrNull(n: Int): T? = (if (n in 0 until size) productElement(n) as T else null) + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @return the element `n` elements after the first element, `null` if out of bounds + */ +fun Product16.getOrNull(n: Int): T? = (if (n in 0 until size) productElement(n) as T else null) + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @return the element `n` elements after the first element, `null` if out of bounds + */ +fun Product17.getOrNull(n: Int): T? = (if (n in 0 until size) productElement(n) as T else null) + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @return the element `n` elements after the first element, `null` if out of bounds + */ +fun Product18.getOrNull(n: Int): T? = (if (n in 0 until size) productElement(n) as T else null) + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @return the element `n` elements after the first element, `null` if out of bounds + */ +fun Product19.getOrNull(n: Int): T? = (if (n in 0 until size) productElement(n) as T else null) + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @return the element `n` elements after the first element, `null` if out of bounds + */ +fun Product20.getOrNull(n: Int): T? = (if (n in 0 until size) productElement(n) as T else null) + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @return the element `n` elements after the first element, `null` if out of bounds + */ +fun Product21.getOrNull(n: Int): T? = (if (n in 0 until size) productElement(n) as T else null) + +/** The n'th element of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param n the index of the element to return + * @return the element `n` elements after the first element, `null` if out of bounds + */ +fun Product22.getOrNull(n: Int): T? = (if (n in 0 until size) productElement(n) as T else null) + + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @throws IndexOutOfBoundsException + * @return the elements in [indexRange] + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product1.get(indexRange: IntRange): List = indexRange.map(::get) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @throws IndexOutOfBoundsException + * @return the elements in [indexRange] + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product2.get(indexRange: IntRange): List = indexRange.map(::get) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @throws IndexOutOfBoundsException + * @return the elements in [indexRange] + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product3.get(indexRange: IntRange): List = indexRange.map(::get) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @throws IndexOutOfBoundsException + * @return the elements in [indexRange] + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product4.get(indexRange: IntRange): List = indexRange.map(::get) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @throws IndexOutOfBoundsException + * @return the elements in [indexRange] + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product5.get(indexRange: IntRange): List = indexRange.map(::get) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @throws IndexOutOfBoundsException + * @return the elements in [indexRange] + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product6.get(indexRange: IntRange): List = indexRange.map(::get) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @throws IndexOutOfBoundsException + * @return the elements in [indexRange] + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product7.get(indexRange: IntRange): List = indexRange.map(::get) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @throws IndexOutOfBoundsException + * @return the elements in [indexRange] + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product8.get(indexRange: IntRange): List = indexRange.map(::get) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @throws IndexOutOfBoundsException + * @return the elements in [indexRange] + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product9.get(indexRange: IntRange): List = indexRange.map(::get) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @throws IndexOutOfBoundsException + * @return the elements in [indexRange] + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product10.get(indexRange: IntRange): List = indexRange.map(::get) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @throws IndexOutOfBoundsException + * @return the elements in [indexRange] + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product11.get(indexRange: IntRange): List = indexRange.map(::get) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @throws IndexOutOfBoundsException + * @return the elements in [indexRange] + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product12.get(indexRange: IntRange): List = indexRange.map(::get) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @throws IndexOutOfBoundsException + * @return the elements in [indexRange] + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product13.get(indexRange: IntRange): List = indexRange.map(::get) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @throws IndexOutOfBoundsException + * @return the elements in [indexRange] + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product14.get(indexRange: IntRange): List = indexRange.map(::get) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @throws IndexOutOfBoundsException + * @return the elements in [indexRange] + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product15.get(indexRange: IntRange): List = indexRange.map(::get) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @throws IndexOutOfBoundsException + * @return the elements in [indexRange] + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product16.get(indexRange: IntRange): List = indexRange.map(::get) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @throws IndexOutOfBoundsException + * @return the elements in [indexRange] + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product17.get(indexRange: IntRange): List = indexRange.map(::get) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @throws IndexOutOfBoundsException + * @return the elements in [indexRange] + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product18.get(indexRange: IntRange): List = indexRange.map(::get) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @throws IndexOutOfBoundsException + * @return the elements in [indexRange] + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product19.get(indexRange: IntRange): List = indexRange.map(::get) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @throws IndexOutOfBoundsException + * @return the elements in [indexRange] + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product20.get(indexRange: IntRange): List = indexRange.map(::get) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @throws IndexOutOfBoundsException + * @return the elements in [indexRange] + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product21.get(indexRange: IntRange): List = indexRange.map(::get) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @throws IndexOutOfBoundsException + * @return the elements in [indexRange] + */ +@Throws(IndexOutOfBoundsException::class) +operator fun Product22.get(indexRange: IntRange): List = indexRange.map(::get) + + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @return the elements in [indexRange], `null` if out of bounds + */ +fun Product1.getOrNull(indexRange: IntRange): List = indexRange.map(::getOrNull) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @return the elements in [indexRange], `null` if out of bounds + */ +fun Product2.getOrNull(indexRange: IntRange): List = indexRange.map(::getOrNull) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @return the elements in [indexRange], `null` if out of bounds + */ +fun Product3.getOrNull(indexRange: IntRange): List = indexRange.map(::getOrNull) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @return the elements in [indexRange], `null` if out of bounds + */ +fun Product4.getOrNull(indexRange: IntRange): List = indexRange.map(::getOrNull) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @return the elements in [indexRange], `null` if out of bounds + */ +fun Product5.getOrNull(indexRange: IntRange): List = indexRange.map(::getOrNull) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @return the elements in [indexRange], `null` if out of bounds + */ +fun Product6.getOrNull(indexRange: IntRange): List = indexRange.map(::getOrNull) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @return the elements in [indexRange], `null` if out of bounds + */ +fun Product7.getOrNull(indexRange: IntRange): List = indexRange.map(::getOrNull) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @return the elements in [indexRange], `null` if out of bounds + */ +fun Product8.getOrNull(indexRange: IntRange): List = indexRange.map(::getOrNull) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @return the elements in [indexRange], `null` if out of bounds + */ +fun Product9.getOrNull(indexRange: IntRange): List = indexRange.map(::getOrNull) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @return the elements in [indexRange], `null` if out of bounds + */ +fun Product10.getOrNull(indexRange: IntRange): List = indexRange.map(::getOrNull) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @return the elements in [indexRange], `null` if out of bounds + */ +fun Product11.getOrNull(indexRange: IntRange): List = indexRange.map(::getOrNull) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @return the elements in [indexRange], `null` if out of bounds + */ +fun Product12.getOrNull(indexRange: IntRange): List = indexRange.map(::getOrNull) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @return the elements in [indexRange], `null` if out of bounds + */ +fun Product13.getOrNull(indexRange: IntRange): List = indexRange.map(::getOrNull) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @return the elements in [indexRange], `null` if out of bounds + */ +fun Product14.getOrNull(indexRange: IntRange): List = indexRange.map(::getOrNull) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @return the elements in [indexRange], `null` if out of bounds + */ +fun Product15.getOrNull(indexRange: IntRange): List = indexRange.map(::getOrNull) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @return the elements in [indexRange], `null` if out of bounds + */ +fun Product16.getOrNull(indexRange: IntRange): List = indexRange.map(::getOrNull) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @return the elements in [indexRange], `null` if out of bounds + */ +fun Product17.getOrNull(indexRange: IntRange): List = indexRange.map(::getOrNull) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @return the elements in [indexRange], `null` if out of bounds + */ +fun Product18.getOrNull(indexRange: IntRange): List = indexRange.map(::getOrNull) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @return the elements in [indexRange], `null` if out of bounds + */ +fun Product19.getOrNull(indexRange: IntRange): List = indexRange.map(::getOrNull) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @return the elements in [indexRange], `null` if out of bounds + */ +fun Product20.getOrNull(indexRange: IntRange): List = indexRange.map(::getOrNull) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @return the elements in [indexRange], `null` if out of bounds + */ +fun Product21.getOrNull(indexRange: IntRange): List = indexRange.map(::getOrNull) + +/** The range of n'th elements of this product, 0-based. In other words, for a + * product `A(x,,1,,, ..., x,,k,,)`, returns `x,,(n+1),,` where `0 <= n < k`. + * + * @param indexRange the indices of the elements to return + * @return the elements in [indexRange], `null` if out of bounds + */ +fun Product22.getOrNull(indexRange: IntRange): List = indexRange.map(::getOrNull) diff --git a/scala-tuples-in-kotlin/src/test/kotlin/org/jetbrains/kotlinx/spark/api/tuples/TuplesTest.kt b/scala-tuples-in-kotlin/src/test/kotlin/org/jetbrains/kotlinx/spark/api/tuples/TuplesTest.kt new file mode 100644 index 00000000..5cbf6c72 --- /dev/null +++ b/scala-tuples-in-kotlin/src/test/kotlin/org/jetbrains/kotlinx/spark/api/tuples/TuplesTest.kt @@ -0,0 +1,246 @@ +/*- + * =LICENSE= + * Kotlin Spark API: Scala Tuples in Kotlin + * ---------- + * 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.tuples + +import io.kotest.assertions.throwables.shouldThrow +import io.kotest.core.spec.style.ShouldSpec +import io.kotest.matchers.shouldBe +import io.kotest.matchers.shouldNotBe +import org.jetbrains.kotlinx.spark.api.tuples.* +import org.jetbrains.kotlinx.spark.api.* +import scala.Tuple3 +import scala.Tuple1 +import scala.Tuple2 +import kotlin.reflect.typeOf + +@Suppress("ShouldBeInstanceOfInspection", "RedundantLambdaArrow", "USELESS_IS_CHECK") +class TuplesTest : ShouldSpec({ + context("Test tuple extensions") { + + should("Support different ways to create tuples") { + listOf( + 1 X 2L X "3", + (1 X 2L) + "3", + 1 + (2L X "3"), + 1 + t() + 2L + "3", + t() + 1 + 2L + "3", + tupleOf() + 1 + 2L + "3", + EmptyTuple + 1 + 2L + "3", + emptyTuple() + 1 + 2L + "3", + t(1, 2L, "3"), + tupleOf(1, 2L, "3"), + t(1, 2L) + "3", + t(1, 2L) + t("3"), + t(1) + t(2L, "3"), + t(1) + t(2L) + t("3"), + t() + t(1, 2L, "3"), + t(1) + t() + t(2L, "3"), + t(1, 2L) + t() + t("3"), + t(1, 2L, "3") + t(), + 1 + t(2L) + "3", + 1 + t(2L, "3"), + Triple(1, 2L, "3").toTuple(), + (1 to 2L).toTuple().appendedBy("3"), + (2L to "3").toTuple().prependedBy(1), + ).forEach { + it shouldBe Tuple3(1, 2L, "3") + } + } + + should("Merge tuples with +, append/prepend other values") { + t() + 1 shouldBe t(1) + t(1) + 2L shouldBe t(1, 2L) + t(1, 2L) + "3" shouldBe t(1, 2L, "3") + 1 + t() shouldBe t(1) + 2L + t(1) shouldBe t(2L, 1) + 2L + t(1, "3") shouldBe t(2L, 1, "3") + + // NOTE! String.plus is a thing + "3" + t(1, 2L) shouldNotBe t("3", 1, 2L) + + t() + t(1) shouldBe t(1) + t(1) + t(2L) shouldBe t(1, 2L) + t(1, 2L) + t("3") shouldBe t(1, 2L, "3") + t(1) + t(2L, "3") shouldBe t(1, 2L, "3") + + t() concat t(1) shouldBe t(1) + t(1) concat t(2L) shouldBe t(1, 2L) + t(1, 2L) concat t("3") shouldBe t(1, 2L, "3") + t(1) concat t(2L, "3") shouldBe t(1, 2L, "3") + + // tuple inside other tuple + t().appendedBy(t(1)) shouldBe t(t(1)) + t() + t(t(1)) shouldBe t(t(1)) + t().prependedBy(t(1)) shouldBe t(t(1)) + t(t(1)) + t() shouldBe t(t(1)) + + t(1).appendedBy(t(2L)) shouldBe t(1, t(2L)) + t(1) + t(t(2L)) shouldBe t(1, t(2L)) + t(1).prependedBy(t(2L)) shouldBe t(t(2L), 1) + t(t(2L)) + t(1) shouldBe t(t(2L), 1) + } + + should("Have drop functions") { + t(1, 2L).dropLast() shouldBe t(1) + t(1, 2L).dropFirst() shouldBe t(2L) + t(1, 2L, "3").dropLast() shouldBe t(1, 2L) + t(1, 2L, "3").dropFirst() shouldBe t(2L, "3") + + t(1).dropLast() shouldBe emptyTuple() + t(1).dropFirst() shouldBe emptyTuple() + } + + should("Have Tuple destructuring") { + val (a: Int, b: Double, c: Long, d: String, e: Char, f: Float, g: Short, h: Byte, i: UInt, j: UByte, k: UShort, l: ULong) = + 1 X 2.0 X 3L X "4" X '5' X 6F X 7.toShort() X 8.toByte() X 9.toUInt() X 10.toUByte() X 11.toUShort() X 12.toULong() // etc... + a shouldBe 1 + b shouldBe 2.0 + c shouldBe 3L + d shouldBe "4" + e shouldBe '5' + f shouldBe 6F + g shouldBe 7.toShort() + h shouldBe 8.toByte() + i shouldBe 9.toUInt() + j shouldBe 10.toUByte() + k shouldBe 11.toUShort() + l shouldBe 12.toULong() + } + + should("Have other helpful extensions") { + (0 !in tupleOf()) shouldBe true + (1 in tupleOf(1, 2, 3)) shouldBe true + (0 !in tupleOf(1, 2, 3)) shouldBe true + tupleOf(1, 2, 3).iterator().asSequence().toSet() shouldBe setOf(1, 2, 3) + for (it in tupleOf(1, 1, 1)) { + it shouldBe 1 + } + tupleOf(1, 2, 3).toList().isNotEmpty() shouldBe true + tupleOf(1, 2, 3).asIterable().none { it > 4 } shouldBe true + tupleOf(1, 2, 3, 4, 5).size shouldBe 5 + tupleOf(1, 2, 3, 4)[0] shouldBe 1 + shouldThrow { tupleOf(1, 2L)[5] } + + tupleOf(1 to 3, arrayOf(1), A()).getOrNull(5).let { + (it is Any?) shouldBe true + it shouldBe null + } + + tupleOf(1, 2, 3).getOrNull(5).let { + (it is Int?) shouldBe true + it shouldBe null + } + + shouldThrow { tupleOf(1).getAs(5) } + shouldThrow { tupleOf(1).getAs(0) } + + tupleOf(1).getAsOrNull(5) shouldBe null + tupleOf(1).getAsOrNull(0) shouldBe null + + + tupleOf(1, 2, 3).toTriple() shouldBe Triple(1, 2, 3) + + tupleOf(1, 2, 3, 4, 5, 6, 7)[1..3].let { + (it is List) shouldBe true + it.containsAll(listOf(2, 3, 4)) shouldBe true + } + tupleOf(1, 1, 2)[1..2] shouldBe tupleOf(1, 2, 2)[0..1] + + tupleOf(1, 2, 3, 4, 5)[2] shouldBe 3 + + shouldThrow { tupleOf(1, 1, 2)[1..5] } + (null in tupleOf(1, 1, 2).getOrNull(1..5)) shouldBe true + + tupleOf(1, 2) shouldBe tupleOf(2, 1).swap() + tupleOf(1 to "Test") shouldBe tupleOf(1 to "Test") + val a: List = tupleOf(A(), B()).toList() + } + + should("Have copy methods for tuples, Kotlin data class style") { + t().copy() shouldBe t() + + t(1, 2).copy(_1 = 0) shouldBe t(0, 2) + t(1, 2).copy(_2 = 0) shouldBe t(1, 0) + + t(1, 2).copy() shouldBe t(1, 2) + t(1, 2, 3, 4, 5).copy() shouldBe t(1, 2, 3, 4, 5) + + // when specifying all parameters, the Scala version will be used + t(1, 2).copy(3, 4) shouldBe t(3, 4) + // unless explicitly giving parameters + t(1, 2).copy(_1 = 3, _2 = 4) shouldBe t(3, 4) + } + + should("Zip tuples") { + + (t(1, 2) zip t(3, 4)) shouldBe t(t(1, 3), t(2, 4)) + (t(1, 2, 3, 4, 5, 6) zip t("a", "b")) shouldBe t(t(1, "a"), t(2, "b")) + + (t(1, 2, 3, 4) zip t()) shouldBe t() + (t() zip t(1, 2, 3, 4)) shouldBe t() + + val a: Tuple2, Tuple2> = t("1", 2.0) zip t(3, 4L, "", "") + val b: Tuple3, Tuple2, Tuple2> = t("1", 2.0, 5f) zip t(3, 4L, "", "") + + val c: Tuple2, List>, Tuple2, Long>, Int>> = + t(1, mapOf(1 to "a")) zip t("13", 1L) zip t(listOf(null, 1), 1, 'c') + } + + should("Map tuples") { + t(1, 2.toShort(), 3L, 4.0, 5) + .map { it.toString() } + .shouldBe( + t("1", "2", "3", "4.0", "5") + ) + + shouldThrow { + t(1, "2", 3L).cast() + } + } + + should("Take n from tuples") { + t(1, 2, 3).take2() shouldBe t(1, 2) + t(1, 2, 3).takeLast2() shouldBe t(2, 3) + + t(1, 2, 3).take0() shouldBe t() + t(1, 2, 3).takeLast0() shouldBe t() + } + + should("Drop n from tuples") { + t(1, 2, 3).drop2() shouldBe t(3) + t(1, 2, 3).dropLast2() shouldBe t(1) + + t(1, 2, 3).drop0() shouldBe t(1, 2, 3) + t(1, 2, 3).dropLast0() shouldBe t(1, 2, 3) + } + + should("Split tuples") { + t(1, 2, 3, 4, 5).splitAt2() shouldBe t(t(1, 2), t(3, 4, 5)) + t(1, 2, 3, 4, 5).splitAt0() shouldBe t(t(), t(1, 2, 3, 4, 5)) + t(1, 2, 3, 4, 5).splitAt5() shouldBe t(t(1, 2, 3, 4, 5), t()) + } + + } +}) + +interface Super + +class A : Super +class B : Super