From f49a434bda351417441d196d6be3bf601a539b03 Mon Sep 17 00:00:00 2001 From: Flavio Brasil Date: Mon, 8 Jan 2018 10:26:47 -0800 Subject: [PATCH] add quotation to external APIs --- .../com/twitter/scalding/avro/package.scala | 7 +- .../commons/extensions/Checkpoint.scala | 3 +- .../source/VersionedKeyValSource.scala | 3 +- .../twitter/scalding/examples/KMeans.scala | 5 +- .../com/twitter/scalding/CumulativeSum.scala | 7 +- .../com/twitter/scalding/bdd/TBddDsl.scala | 3 + .../twitter/scalding/mathematics/Matrix.scala | 6 +- .../scalding/mathematics/Matrix2.scala | 97 ++++++----- .../mathematics/TypedSimilarity.scala | 25 +-- .../typed/GeneratedFlattenGroup.scala | 18 +- .../com/twitter/scalding/typed/Grouped.scala | 72 ++++---- .../twitter/scalding/typed/KeyedList.scala | 95 +++++----- .../twitter/scalding/typed/LookupJoin.scala | 9 +- .../twitter/scalding/typed/MultiJoin.scala | 128 +++++++------- .../com/twitter/scalding/typed/Sketched.scala | 6 +- .../com/twitter/scalding/typed/TDsl.scala | 5 +- .../twitter/scalding/typed/TypedPipe.scala | 162 +++++++++--------- .../scalding/typed/TypedPipeDiff.scala | 18 +- .../twitter/scalding/typed/ValuePipe.scala | 33 ++-- .../cascading_backend/CascadingBackend.scala | 11 +- .../scalding/ReferencedClassFinderTest.scala | 3 +- .../scalding/typed/InAnotherPackage.scala | 14 -- .../typed/NoStackLineNumberTest.scala | 51 ------ .../typed/OptimizationRulesTest.scala | 3 +- .../scalding/platform/PlatformTest.scala | 9 +- 25 files changed, 387 insertions(+), 406 deletions(-) delete mode 100644 scalding-core/src/test/scala/com/twitter/scalding/typed/InAnotherPackage.scala delete mode 100644 scalding-core/src/test/scala/com/twitter/scalding/typed/NoStackLineNumberTest.scala diff --git a/scalding-avro/src/main/scala/com/twitter/scalding/avro/package.scala b/scalding-avro/src/main/scala/com/twitter/scalding/avro/package.scala index c5bb868103..aad2e008ae 100644 --- a/scalding-avro/src/main/scala/com/twitter/scalding/avro/package.scala +++ b/scalding-avro/src/main/scala/com/twitter/scalding/avro/package.scala @@ -15,6 +15,7 @@ package com.twitter.scalding +import com.twitter.scalding.quotation.Quoted import cascading.flow.FlowDef import org.apache.avro.Schema import collection.JavaConverters._ @@ -26,7 +27,8 @@ package object avro { conv: TupleConverter[T], set: TupleSetter[T], flow: FlowDef, - mode: Mode): Unit = { + mode: Mode, + q: Quoted): Unit = { val sink = PackedAvroSource[T](path) pipe.write(sink) } @@ -35,7 +37,8 @@ package object avro { conv: TupleConverter[T], set: TupleSetter[T], flow: FlowDef, - mode: Mode): Unit = { + mode: Mode, + q: Quoted): Unit = { import Dsl._ val sink = UnpackedAvroSource[T](path, Some(schema)) val outFields = { diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/extensions/Checkpoint.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/extensions/Checkpoint.scala index 2dcdecb870..13764ddc51 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/extensions/Checkpoint.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/extensions/Checkpoint.scala @@ -17,6 +17,7 @@ limitations under the License. package com.twitter.scalding.commons.extensions import com.twitter.scalding._ +import com.twitter.scalding.quotation.Quoted import com.twitter.scalding.Dsl._ import cascading.flow.FlowDef @@ -111,7 +112,7 @@ object Checkpoint { // Wrapper for Checkpoint when using a TypedPipe def apply[A](checkpointName: String)(flow: => TypedPipe[A])(implicit args: Args, mode: Mode, flowDef: FlowDef, - conv: TupleConverter[A], setter: TupleSetter[A]): TypedPipe[A] = { + conv: TupleConverter[A], setter: TupleSetter[A], q: Quoted): TypedPipe[A] = { val rPipe = apply(checkpointName, Dsl.intFields(0 until conv.arity)) { flow.toPipe(Dsl.intFields(0 until conv.arity)) } diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/VersionedKeyValSource.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/VersionedKeyValSource.scala index 5c62b3687a..e20b8f335f 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/VersionedKeyValSource.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/VersionedKeyValSource.scala @@ -32,6 +32,7 @@ import com.twitter.scalding.commons.tap.VersionedTap.TapMode import com.twitter.scalding.source.{ CheckedInversion, MaxFailuresCheck } import com.twitter.scalding.typed.KeyedListLike import com.twitter.scalding.typed.TypedSink +import com.twitter.scalding.quotation.Quoted import org.apache.hadoop.mapred.JobConf import scala.collection.JavaConverters._ @@ -225,7 +226,7 @@ class TypedRichPipeEx[K: Ordering, V: Monoid](pipe: TypedPipe[(K, V)]) extends j // the pipe in using an implicit `Monoid[V]` and sinks all results // into the `sinkVersion` of data (or a new version) specified by // `src`. - def writeIncremental(src: VersionedKeyValSource[K, V], reducers: Int = 1)(implicit flowDef: FlowDef, mode: Mode): TypedPipe[(K, V)] = { + def writeIncremental(src: VersionedKeyValSource[K, V], reducers: Int = 1)(implicit flowDef: FlowDef, mode: Mode, q: Quoted): TypedPipe[(K, V)] = { val outPipe = if (!src.resourceExists(mode)) pipe diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/examples/KMeans.scala b/scalding-commons/src/main/scala/com/twitter/scalding/examples/KMeans.scala index d7db06d87e..71599ab5aa 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/examples/KMeans.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/examples/KMeans.scala @@ -2,6 +2,7 @@ package com.twitter.scalding.examples import com.twitter.scalding._ import com.twitter.scalding.typed.ComputedValue +import com.twitter.scalding.quotation.Quoted object KMeans { @@ -88,12 +89,12 @@ object KMeans { } } - def initializeClusters(k: Int, points: TypedPipe[Vector[Double]]): (ValuePipe[List[LabeledVector]], TypedPipe[LabeledVector]) = { + def initializeClusters(k: Int, points: TypedPipe[Vector[Double]])(implicit q: Quoted): (ValuePipe[List[LabeledVector]], TypedPipe[LabeledVector]) = { val rng = new java.util.Random(123) // take a random k vectors: val clusters = points.map { v => (rng.nextDouble, v) } .groupAll - .sortedTake(k)(Ordering.by(_._1)) + .sortedTake(k)(Ordering.by(_._1), q) .mapValues { randk => randk.iterator .zipWithIndex diff --git a/scalding-core/src/main/scala/com/twitter/scalding/CumulativeSum.scala b/scalding-core/src/main/scala/com/twitter/scalding/CumulativeSum.scala index b26882c732..f00c8e3b76 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/CumulativeSum.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/CumulativeSum.scala @@ -1,6 +1,7 @@ package com.twitter.scalding.typed import com.twitter.algebird._ +import com.twitter.scalding.quotation.Quoted /** * Extension for TypedPipe to add a cumulativeSum method. @@ -39,7 +40,8 @@ object CumulativeSum { def cumulativeSum( implicit sg: Semigroup[V], ordU: Ordering[U], - ordK: Ordering[K]): SortedGrouped[K, (U, V)] = { + ordK: Ordering[K], + quoted: Quoted): SortedGrouped[K, (U, V)] = { pipe.group .sortBy { case (u, _) => u } .scanLeft(Nil: List[(U, V)]) { @@ -62,7 +64,8 @@ object CumulativeSum { implicit ordS: Ordering[S], sg: Semigroup[V], ordU: Ordering[U], - ordK: Ordering[K]): TypedPipe[(K, (U, V))] = { + ordK: Ordering[K], + q: Quoted): TypedPipe[(K, (U, V))] = { val sumPerS = pipe .map { case (k, (u, v)) => (k, partition(u)) -> v } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/bdd/TBddDsl.scala b/scalding-core/src/main/scala/com/twitter/scalding/bdd/TBddDsl.scala index 7c1ace3474..b3d02db586 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/bdd/TBddDsl.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/bdd/TBddDsl.scala @@ -5,9 +5,12 @@ import com.twitter.scalding._ import com.twitter.scalding.source.TypedText import scala.collection.mutable.Buffer import TDsl._ +import com.twitter.scalding.quotation.Quoted trait TBddDsl extends FieldConversions with TypedPipeOperationsConversions { + private implicit val q: Quoted = Quoted.internal + def Given[TypeIn](source: TypedTestSource[TypeIn]): TestCaseGiven1[TypeIn] = new TestCaseGiven1[TypeIn](source) def GivenSources(sources: List[TypedTestSource[_]]): TestCaseGivenList = new TestCaseGivenList(sources) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix.scala b/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix.scala index be2e0f3d8c..ad05701b48 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix.scala @@ -30,6 +30,7 @@ import cascading.tap._ import com.twitter.scalding.Dsl._ import scala.math.max import scala.annotation.tailrec +import com.twitter.scalding.quotation.Quoted /** * Matrix class - represents an infinite (hopefully sparse) matrix. @@ -138,10 +139,11 @@ class MatrixMappableExtensions[T](mappable: Mappable[T])(implicit fd: FlowDef, m } def toBlockMatrix[Group, Row, Col, Val](implicit ev: <:<[T, (Group, Row, Col, Val)], ord: Ordering[(Group, Row)], - setter: TupleSetter[(Group, Row, Col, Val)]): BlockMatrix[Group, Row, Col, Val] = + setter: TupleSetter[(Group, Row, Col, Val)], + q: Quoted): BlockMatrix[Group, Row, Col, Val] = mapToBlockMatrix { _.asInstanceOf[(Group, Row, Col, Val)] } - def mapToBlockMatrix[Group, Row, Col, Val](fn: (T) => (Group, Row, Col, Val))(implicit ord: Ordering[(Group, Row)]): BlockMatrix[Group, Row, Col, Val] = { + def mapToBlockMatrix[Group, Row, Col, Val](fn: (T) => (Group, Row, Col, Val))(implicit ord: Ordering[(Group, Row)], q: Quoted): BlockMatrix[Group, Row, Col, Val] = { val matPipe = TypedPipe .from(mappable) .map(fn) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix2.scala b/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix2.scala index 8195ba22ac..2717ae21a2 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix2.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix2.scala @@ -26,6 +26,7 @@ import scala.collection.mutable.Map import scala.collection.mutable.HashMap import java.io.Serializable +import com.twitter.scalding.quotation.Quoted /** * This is the future Matrix API. The old one will be removed in scalding 0.10.0 (or 1.0.0). @@ -57,9 +58,9 @@ sealed trait Matrix2[R, C, V] extends Serializable { def *[C2](that: Matrix2[C, C2, V])(implicit ring: Ring[V], mj: MatrixJoiner2): Matrix2[R, C2, V] = Product(this, that, ring) - def *(that: Scalar2[V])(implicit ring: Ring[V], mj: MatrixJoiner2): Matrix2[R, C, V] = that * this + def *(that: Scalar2[V])(implicit ring: Ring[V], mj: MatrixJoiner2, q: Quoted): Matrix2[R, C, V] = that * this - def /(that: Scalar2[V])(implicit field: Field[V]): Matrix2[R, C, V] = + def /(that: Scalar2[V])(implicit field: Field[V], q: Quoted): Matrix2[R, C, V] = that divMatrix this /** * Convert the current Matrix to a TypedPipe @@ -100,7 +101,8 @@ sealed trait Matrix2[R, C, V] extends Serializable { */ def propagate[C2, VecV](vec: Matrix2[C, C2, VecV])(implicit ev: =:=[V, Boolean], mon: Monoid[VecV], - mj: MatrixJoiner2): Matrix2[R, C2, VecV] = { + mj: MatrixJoiner2, + q: Quoted): Matrix2[R, C2, VecV] = { //This cast will always succeed: lazy val joinedBool = mj.join(this.asInstanceOf[Matrix2[R, C, Boolean]], vec) @@ -116,11 +118,11 @@ sealed trait Matrix2[R, C, V] extends Serializable { MatrixLiteral(resultPipe, this.sizeHint) } - def propagateRow[C2](mat: Matrix2[C, C2, Boolean])(implicit ev: =:=[R, Unit], mon: Monoid[V], mj: MatrixJoiner2): Matrix2[Unit, C2, V] = + def propagateRow[C2](mat: Matrix2[C, C2, Boolean])(implicit ev: =:=[R, Unit], mon: Monoid[V], mj: MatrixJoiner2, q: Quoted): Matrix2[Unit, C2, V] = mat.transpose.propagate(this.transpose.asInstanceOf[Matrix2[C, Unit, V]]).transpose // Binarize values, all x != 0 become 1 - def binarizeAs[NewValT](implicit mon: Monoid[V], ring: Ring[NewValT]): Matrix2[R, C, NewValT] = { + def binarizeAs[NewValT](implicit mon: Monoid[V], ring: Ring[NewValT], q: Quoted): Matrix2[R, C, NewValT] = { lazy val newPipe = toTypedPipe.map { case (r, c, x) => (r, c, if (mon.isNonZero(x)) { ring.one } else { ring.zero }) @@ -133,7 +135,7 @@ sealed trait Matrix2[R, C, V] extends Serializable { * Row L2 normalization * After this operation, the sum(|x|^2) along each row will be 1. */ - def rowL2Normalize(implicit num: Numeric[V], mj: MatrixJoiner2): Matrix2[R, C, Double] = { + def rowL2Normalize(implicit num: Numeric[V], mj: MatrixJoiner2, q: Quoted): Matrix2[R, C, Double] = { val matD = MatrixLiteral(this.toTypedPipe.map{ case (r, c, x) => (r, c, num.toDouble(x)) }, this.sizeHint) lazy val result = MatrixLiteral(this.toTypedPipe.map { case (r, c, x) => (r, c, num.toDouble(x) * num.toDouble(x)) }, this.sizeHint) .sumColVectors @@ -146,7 +148,7 @@ sealed trait Matrix2[R, C, V] extends Serializable { * Row L1 normalization * After this operation, the sum(|x|) alone each row will be 1. */ - def rowL1Normalize(implicit num: Numeric[V], mj: MatrixJoiner2): Matrix2[R, C, Double] = { + def rowL1Normalize(implicit num: Numeric[V], mj: MatrixJoiner2, q: Quoted): Matrix2[R, C, Double] = { val matD = MatrixLiteral(this.toTypedPipe.map{ case (r, c, x) => (r, c, num.toDouble(x).abs) }, this.sizeHint) lazy val result = matD .sumColVectors @@ -155,13 +157,13 @@ sealed trait Matrix2[R, C, V] extends Serializable { MatrixLiteral(result, SizeHint.asDiagonal(this.sizeHint.setRowsToCols)) * matD } - def getRow(index: R): Matrix2[Unit, C, V] = + def getRow(index: R)(implicit q: Quoted): Matrix2[Unit, C, V] = MatrixLiteral( toTypedPipe .filter { case (r, c, v) => Ordering[R].equiv(r, index) } .map { case (r, c, v) => ((), c, v) }, this.sizeHint.setRows(1L)) - def getColumn(index: C): Matrix2[R, Unit, V] = + def getColumn(index: C)(implicit q: Quoted): Matrix2[R, Unit, V] = MatrixLiteral( toTypedPipe .filter { case (r, c, v) => Ordering[C].equiv(c, index) } @@ -171,21 +173,21 @@ sealed trait Matrix2[R, C, V] extends Serializable { * Consider this Matrix as the r2 row of a matrix. The current matrix must be a row, * which is to say, its row type must be Unit. */ - def asRow[R2](r2: R2)(implicit ev: R =:= Unit, rowOrd: Ordering[R2]): Matrix2[R2, C, V] = + def asRow[R2](r2: R2)(implicit ev: R =:= Unit, rowOrd: Ordering[R2], q: Quoted): Matrix2[R2, C, V] = MatrixLiteral(toTypedPipe.map { case (r, c, v) => (r2, c, v) }, this.sizeHint) - def asCol[C2](c2: C2)(implicit ev: C =:= Unit, colOrd: Ordering[C2]): Matrix2[R, C2, V] = + def asCol[C2](c2: C2)(implicit ev: C =:= Unit, colOrd: Ordering[C2], q: Quoted): Matrix2[R, C2, V] = MatrixLiteral(toTypedPipe.map { case (r, c, v) => (r, c2, v) }, this.sizeHint) // Compute the sum of the main diagonal. Only makes sense cases where the row and col type are // equal - def trace(implicit mon: Monoid[V], ev: =:=[R, C]): Scalar2[V] = + def trace(implicit mon: Monoid[V], ev: =:=[R, C], m: Quoted): Scalar2[V] = Scalar2(toTypedPipe.asInstanceOf[TypedPipe[(R, R, V)]] .filter{ case (r1, r2, _) => Ordering[R].equiv(r1, r2) } .map{ case (_, _, x) => x } - .sum(mon)) + .sum(mon, m)) - def write(sink: TypedSink[(R, C, V)])(implicit fd: FlowDef, m: Mode): Matrix2[R, C, V] = + def write(sink: TypedSink[(R, C, V)])(implicit fd: FlowDef, m: Mode, q: Quoted): Matrix2[R, C, V] = MatrixLiteral(toTypedPipe.write(sink), sizeHint) } @@ -196,7 +198,7 @@ sealed trait Matrix2[R, C, V] extends Serializable { * special handling */ trait MatrixJoiner2 extends java.io.Serializable { - def join[R, C, V, C2, V2](left: Matrix2[R, C, V], right: Matrix2[C, C2, V2]): TypedPipe[(C, ((R, V), (C2, V2)))] + def join[R, C, V, C2, V2](left: Matrix2[R, C, V], right: Matrix2[C, C2, V2])(implicit q: Quoted): TypedPipe[(C, ((R, V), (C2, V2)))] } object MatrixJoiner2 { @@ -205,7 +207,7 @@ object MatrixJoiner2 { implicit def default: MatrixJoiner2 = new DefaultMatrixJoiner(10000L) def join[R, C, V, C2, V2](left: Matrix2[R, C, V], - right: Matrix2[C, C2, V2])(implicit mj: MatrixJoiner2): TypedPipe[(C, ((R, V), (C2, V2)))] = + right: Matrix2[C, C2, V2])(implicit mj: MatrixJoiner2, q: Quoted): TypedPipe[(C, ((R, V), (C2, V2)))] = mj.join(left, right) } @@ -215,7 +217,7 @@ object MatrixJoiner2 { */ class DefaultMatrixJoiner(sizeRatioThreshold: Long) extends MatrixJoiner2 { def join[R, C, V, C2, V2](left: Matrix2[R, C, V], - right: Matrix2[C, C2, V2]): TypedPipe[(C, ((R, V), (C2, V2)))] = { + right: Matrix2[C, C2, V2])(implicit q: Quoted): TypedPipe[(C, ((R, V), (C2, V2)))] = { implicit val cOrd: Ordering[C] = left.colOrd val one = left.toTypedPipe.map { case (r, c, v) => (c, (r, v)) }.group val two = right.toTypedPipe.map { case (c, c2, v2) => (c, (c2, v2)) }.group @@ -293,14 +295,16 @@ final case class Product[R, C, C2, V](left: Matrix2[R, C, V], val leftMatrix = right.isInstanceOf[OneC[_, _]] val localRing = ring + implicit val q: Quoted = Quoted.internal + val joined = (if (leftMatrix) { val ord: Ordering[R] = left.rowOrd - left.toTypedPipe.groupBy(x => x._1)(ord) + left.toTypedPipe.groupBy(x => x._1)(ord, q) } else { val ord: Ordering[C] = right.rowOrd - right.toTypedPipe.groupBy(x => x._1)(ord) + right.toTypedPipe.groupBy(x => x._1)(ord, q) }).mapValues { _._3 } - .sum(localRing) + .sum(localRing, q) .filter { kv => localRing.isNonZero(kv._2) } if (leftMatrix) { @@ -317,6 +321,7 @@ final case class Product[R, C, C2, V](left: Matrix2[R, C, V], specialCase } else { implicit val ord: Ordering[C] = right.rowOrd + implicit val q: Quoted = Quoted.internal val localRing = ring joiner.join(left, right) .map { case (key, ((l1, lv), (r2, rv))) => (l1, r2, localRing.times(lv, rv)) } @@ -328,13 +333,13 @@ final case class Product[R, C, C2, V](left: Matrix2[R, C, V], } } - private def computePipe(joined: TypedPipe[(R, C2, V)] = toOuterSum): TypedPipe[(R, C2, V)] = { + private def computePipe(joined: TypedPipe[(R, C2, V)] = toOuterSum)(implicit q: Quoted): TypedPipe[(R, C2, V)] = { if (isSpecialCase) { joined } else { val localRing = ring joined.groupBy(w => (w._1, w._2)).mapValues { _._3 } - .sum(localRing) + .sum(localRing, q) .filter { kv => localRing.isNonZero(kv._2) } .map { case ((r, c), v) => (r, c, v) } } @@ -343,6 +348,7 @@ final case class Product[R, C, C2, V](left: Matrix2[R, C, V], override lazy val toTypedPipe: TypedPipe[(R, C2, V)] = { expressions match { case Some(m) => m.get(this).getOrElse { + implicit val q: Quoted = Quoted.internal val result = computePipe() m.put(this, result) result @@ -369,7 +375,7 @@ final case class Product[R, C, C2, V](left: Matrix2[R, C, V], /** * Trace(A B) = Trace(B A) so we optimize to choose the lowest cost item */ - override def trace(implicit mon: Monoid[V], ev1: =:=[R, C2]): Scalar2[V] = { + override def trace(implicit mon: Monoid[V], ev1: =:=[R, C2], m: Quoted): Scalar2[V] = { val (cost1, plan1) = Matrix2.optimize(this.asInstanceOf[Matrix2[Any, Any, V]]) // linter:ignore val (cost2, plan2) = Matrix2.optimize( // linter:ignore Product(right.asInstanceOf[Matrix2[C, R, V]], left.asInstanceOf[Matrix2[R, C, V]], ring, None) @@ -379,12 +385,12 @@ final case class Product[R, C, C2, V](left: Matrix2[R, C, V], val product2 = plan2.asInstanceOf[Product[C, R, C, V]] val ord = left.colOrd val filtered = product2.toOuterSum.filter{ case (c1, c2, _) => ord.equiv(c1, c2) } - Scalar2(product2.computePipe(filtered).map{ case (_, _, x) => x }.sum(mon)) + Scalar2(product2.computePipe(filtered).map{ case (_, _, x) => x }.sum(mon, m)) } else { val product1 = plan1.asInstanceOf[Product[R, C, R, V]] val ord = left.rowOrd val filtered = product1.toOuterSum.filter{ case (r1, r2, _) => ord.equiv(r1, r2) } - Scalar2(product1.computePipe(filtered).map{ case (_, _, x) => x }.sum(mon)) + Scalar2(product1.computePipe(filtered).map{ case (_, _, x) => x }.sum(mon, m)) } } @@ -418,13 +424,14 @@ final case class Sum[R, C, V](left: Matrix2[R, C, V], right: Matrix2[R, C, V], m } override lazy val toTypedPipe: TypedPipe[(R, C, V)] = { + implicit val q: Quoted = Quoted.internal if (left.equals(right)) { left.optimizedSelf.toTypedPipe.map(v => (v._1, v._2, mon.plus(v._3, v._3))) } else { collectAddends(this) .reduce((x, y) => x ++ y) .groupBy(x => (x._1, x._2)).mapValues { _._3 } - .sum(mon) + .sum(mon, q) .filter { kv => mon.isNonZero(kv._2) } .map { case ((r, c), v) => (r, c, v) } } @@ -441,7 +448,7 @@ final case class Sum[R, C, V](left: Matrix2[R, C, V], right: Matrix2[R, C, V], m override def sumColVectors(implicit ring: Ring[V], mj: MatrixJoiner2): Matrix2[R, Unit, V] = Sum(left.sumColVectors, right.sumColVectors, mon) - override def trace(implicit mon: Monoid[V], ev: =:=[R, C]): Scalar2[V] = + override def trace(implicit mon: Monoid[V], ev: =:=[R, C], m: Quoted): Scalar2[V] = Scalar2(collectAddends(this).map { pipe => pipe.asInstanceOf[TypedPipe[(R, R, V)]] .filter { case (r, c, v) => Ordering[R].equiv(r, c) } @@ -455,6 +462,7 @@ final case class HadamardProduct[R, C, V](left: Matrix2[R, C, V], // TODO: optimize / combine with Sums: https://github.com/tomtau/scalding/issues/14#issuecomment-22971582 override lazy val toTypedPipe: TypedPipe[(R, C, V)] = { + implicit val q: Quoted = Quoted.internal if (left.equals(right)) { left.optimizedSelf.toTypedPipe.map(v => (v._1, v._2, ring.times(v._3, v._3))) } else { @@ -469,7 +477,10 @@ final case class HadamardProduct[R, C, V](left: Matrix2[R, C, V], } } - override lazy val transpose: MatrixLiteral[C, R, V] = MatrixLiteral(toTypedPipe.map(x => (x._2, x._1, x._3)), sizeHint.transpose)(colOrd, rowOrd) + override lazy val transpose: MatrixLiteral[C, R, V] = { + implicit val q: Quoted = Quoted.internal + MatrixLiteral(toTypedPipe.map(x => (x._2, x._1, x._3)), sizeHint.transpose)(colOrd, rowOrd) + } override val sizeHint = left.sizeHint #*# right.sizeHint override def negate(implicit g: Group[V]): HadamardProduct[R, C, V] = if (left.sizeHint.total.getOrElse(BigInt(0L)) > right.sizeHint.total.getOrElse(BigInt(0L))) @@ -486,11 +497,15 @@ final case class MatrixLiteral[R, C, V](override val toTypedPipe: TypedPipe[(R, override val sizeHint: SizeHint)(implicit override val rowOrd: Ordering[R], override val colOrd: Ordering[C]) extends Matrix2[R, C, V] { - override lazy val transpose: MatrixLiteral[C, R, V] = + override lazy val transpose: MatrixLiteral[C, R, V] = { + implicit val q: Quoted = Quoted.internal MatrixLiteral(toTypedPipe.map(x => (x._2, x._1, x._3)), sizeHint.transpose)(colOrd, rowOrd) + } - override def negate(implicit g: Group[V]): MatrixLiteral[R, C, V] = + override def negate(implicit g: Group[V]): MatrixLiteral[R, C, V] = { + implicit val q: Quoted = Quoted.internal MatrixLiteral(toTypedPipe.map(x => (x._1, x._2, g.negate(x._3))), sizeHint) + } } /** @@ -499,7 +514,7 @@ final case class MatrixLiteral[R, C, V](override val toTypedPipe: TypedPipe[(R, trait Scalar2[V] extends Serializable { def value: ValuePipe[V] - def +(that: Scalar2[V])(implicit sg: Semigroup[V]): Scalar2[V] = { + def +(that: Scalar2[V])(implicit sg: Semigroup[V], m: Quoted): Scalar2[V] = { (value, that.value) match { case (EmptyValue, _) => that case (LiteralValue(v1), _) => that.map(sg.plus(v1, _)) @@ -507,17 +522,17 @@ trait Scalar2[V] extends Serializable { case (_, LiteralValue(v2)) => map(sg.plus(_, v2)) // TODO: optimize sums of scalars like sums of matrices: // only one M/R pass for the whole Sum. - case (_, ComputedValue(v2)) => Scalar2((value ++ v2).sum(sg)) + case (_, ComputedValue(v2)) => Scalar2((value ++ v2).sum(sg, m)) } } - def -(that: Scalar2[V])(implicit g: Group[V]): Scalar2[V] = this + that.map(x => g.negate(x)) - def *(that: Scalar2[V])(implicit ring: Ring[V]): Scalar2[V] = + def -(that: Scalar2[V])(implicit g: Group[V], q: Quoted): Scalar2[V] = this + that.map(x => g.negate(x)) + def *(that: Scalar2[V])(implicit ring: Ring[V], q: Quoted): Scalar2[V] = Scalar2(ValuePipe.fold(value, that.value)(ring.times _)) - def /(that: Scalar2[V])(implicit f: Field[V]): Scalar2[V] = + def /(that: Scalar2[V])(implicit f: Field[V], q: Quoted): Scalar2[V] = Scalar2(ValuePipe.fold(value, that.value)(f.div _)) - def unary_-(implicit g: Group[V]): Scalar2[V] = map(x => g.negate(x)) + def unary_-(implicit g: Group[V], q: Quoted): Scalar2[V] = map(x => g.negate(x)) - def *[R, C](that: Matrix2[R, C, V])(implicit ring: Ring[V], mj: MatrixJoiner2): Matrix2[R, C, V] = + def *[R, C](that: Matrix2[R, C, V])(implicit ring: Ring[V], mj: MatrixJoiner2, q: Quoted): Matrix2[R, C, V] = that match { case p @ Product(left, right, _, expressions) => if (left.sizeHint.total.getOrElse(BigInt(0L)) > right.sizeHint.total.getOrElse(BigInt(0L))) @@ -539,7 +554,7 @@ trait Scalar2[V] extends Serializable { .asInstanceOf[Matrix2[R, C, V]] } - def divMatrix[R, C](that: Matrix2[R, C, V])(implicit f: Field[V]): MatrixLiteral[R, C, V] = + def divMatrix[R, C](that: Matrix2[R, C, V])(implicit f: Field[V], q: Quoted): MatrixLiteral[R, C, V] = MatrixLiteral( that.toTypedPipe .mapWithValue(value) { @@ -548,7 +563,7 @@ trait Scalar2[V] extends Serializable { }, that.sizeHint)(that.rowOrd, that.colOrd) - def timesLiteral[R, C](that: Matrix2[R, C, V])(implicit ring: Ring[V]): MatrixLiteral[R, C, V] = + def timesLiteral[R, C](that: Matrix2[R, C, V])(implicit ring: Ring[V], q: Quoted): MatrixLiteral[R, C, V] = MatrixLiteral( that.toTypedPipe .mapWithValue(value) { @@ -557,8 +572,8 @@ trait Scalar2[V] extends Serializable { }, that.sizeHint)(that.rowOrd, that.colOrd) - def map[U](fn: V => U): Scalar2[U] = Scalar2(value.map(fn)) - def toMatrix: Matrix2[Unit, Unit, V] = + def map[U](fn: V => U)(implicit q: Quoted): Scalar2[U] = Scalar2(value.map(fn)) + def toMatrix(implicit q: Quoted): Matrix2[Unit, Unit, V] = MatrixLiteral(value.toTypedPipe.map(v => ((), (), v)), FiniteHint(1, 1)) // TODO: FunctionMatrix[R,C,V](fn: (R,C) => V) and a Literal scalar is just: FuctionMatrix[Unit, Unit, V]({ (_, _) => v }) } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/mathematics/TypedSimilarity.scala b/scalding-core/src/main/scala/com/twitter/scalding/mathematics/TypedSimilarity.scala index 593ae42aaf..f894161e2f 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/mathematics/TypedSimilarity.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/mathematics/TypedSimilarity.scala @@ -18,6 +18,7 @@ package com.twitter.scalding.mathematics import com.twitter.scalding.typed.{ Grouped, TypedPipe, WithReducers } import java.io.Serializable +import com.twitter.scalding.quotation.Quoted /** * Implementation of DISCO and DIMSUM approximation similarity algorithm @@ -43,7 +44,7 @@ object GraphOperations extends Serializable { /** * For each N, aggregate all the edges, and attach Edge state */ - def joinAggregate[N, E, T](grouped: Grouped[N, Edge[N, E]])(agfn: Iterable[Edge[N, E]] => T): TypedPipe[Edge[N, (E, T)]] = + def joinAggregate[N, E, T](grouped: Grouped[N, Edge[N, E]])(agfn: Iterable[Edge[N, E]] => T)(implicit q: Quoted): TypedPipe[Edge[N, (E, T)]] = grouped.cogroup(grouped) { (to: N, left: Iterator[Edge[N, E]], right: Iterable[Edge[N, E]]) => val newState = agfn(right) @@ -52,17 +53,17 @@ object GraphOperations extends Serializable { .values // Returns all Vertices with non-zero in-degree - def withInDegree[N, E](g: TypedPipe[Edge[N, E]])(implicit ord: Ordering[N]): TypedPipe[Edge[N, (E, InDegree)]] = joinAggregate(g.groupBy { _.to }) { it => + def withInDegree[N, E](g: TypedPipe[Edge[N, E]])(implicit ord: Ordering[N], q: Quoted): TypedPipe[Edge[N, (E, InDegree)]] = joinAggregate(g.groupBy { _.to }) { it => InDegree(it.size) } // Returns all Vertices with non-zero out-degree - def withOutDegree[N, E](g: TypedPipe[Edge[N, E]])(implicit ord: Ordering[N]): TypedPipe[Edge[N, (E, OutDegree)]] = joinAggregate(g.groupBy { _.from }) { it => + def withOutDegree[N, E](g: TypedPipe[Edge[N, E]])(implicit ord: Ordering[N], q: Quoted): TypedPipe[Edge[N, (E, OutDegree)]] = joinAggregate(g.groupBy { _.from }) { it => OutDegree(it.size) } // Returns all Vertices with weights and non-zero norms - def withInNorm[N, E](g: TypedPipe[Edge[N, Weight]])(implicit ord: Ordering[N]): TypedPipe[Edge[N, (Weight, L2Norm)]] = joinAggregate(g.groupBy { _.to }) { it => + def withInNorm[N, E](g: TypedPipe[Edge[N, Weight]])(implicit ord: Ordering[N], q: Quoted): TypedPipe[Edge[N, (Weight, L2Norm)]] = joinAggregate(g.groupBy { _.to }) { it => val norm = scala.math.sqrt( it.iterator.map { a => val x = a.data.weight @@ -97,9 +98,9 @@ trait TypedSimilarity[N, E, S] extends Serializable { */ def apply(g: TypedPipe[Edge[N, E]], smallpred: N => Boolean, - bigpred: N => Boolean): TypedPipe[Edge[N, S]] + bigpred: N => Boolean)(implicit q: Quoted): TypedPipe[Edge[N, S]] // Do similarity on all the nodes - def apply(g: TypedPipe[Edge[N, E]]): TypedPipe[Edge[N, S]] = { + def apply(g: TypedPipe[Edge[N, E]])(implicit q: Quoted): TypedPipe[Edge[N, S]] = { val always = { n: N => true } apply(g, always, always) } @@ -116,7 +117,7 @@ object TypedSimilarity extends Serializable { // value: (word, documentsWithWord) // return: Edge of similarity between words measured by documents def exactSetSimilarity[N: Ordering](g: Grouped[N, (N, Int)], - smallpred: N => Boolean, bigpred: N => Boolean): TypedPipe[Edge[N, SetSimilarity]] = + smallpred: N => Boolean, bigpred: N => Boolean)(implicit q: Quoted): TypedPipe[Edge[N, SetSimilarity]] = /* E_{ij} = 1 if document -> word exists * (E^T E)_ij = # of shared documents of i,j * = \sum_k E_ki E_kj @@ -147,7 +148,7 @@ object TypedSimilarity extends Serializable { * See: http://arxiv.org/pdf/1206.2082v2.pdf */ def discoCosineSimilarity[N: Ordering](smallG: Grouped[N, (N, Int)], - bigG: Grouped[N, (N, Int)], oversample: Double): TypedPipe[Edge[N, Double]] = { + bigG: Grouped[N, (N, Int)], oversample: Double)(implicit q: Quoted): TypedPipe[Edge[N, Double]] = { // 1) make rnd lazy due to serialization, // 2) fix seed so that map-reduce speculative execution does not give inconsistent results. lazy val rnd = new scala.util.Random(1024) @@ -184,7 +185,7 @@ object TypedSimilarity extends Serializable { * See: http://stanford.edu/~rezab/papers/dimsum.pdf */ def dimsumCosineSimilarity[N: Ordering](smallG: Grouped[N, (N, Double, Double)], - bigG: Grouped[N, (N, Double, Double)], oversample: Double): TypedPipe[Edge[N, Double]] = { + bigG: Grouped[N, (N, Double, Double)], oversample: Double)(implicit q: Quoted): TypedPipe[Edge[N, Double]] = { lazy val rnd = new scala.util.Random(1024) maybeWithReducers(smallG.cogroup(bigG) { (n: N, leftit: Iterator[(N, Double, Double)], rightit: Iterable[(N, Double, Double)]) => // Use a co-group to ensure this happens in the reducer: @@ -220,7 +221,7 @@ object TypedSimilarity extends Serializable { class ExactInCosine[N](reducers: Int = -1)(implicit override val nodeOrdering: Ordering[N]) extends TypedSimilarity[N, InDegree, Double] { def apply(graph: TypedPipe[Edge[N, InDegree]], - smallpred: N => Boolean, bigpred: N => Boolean): TypedPipe[Edge[N, Double]] = { + smallpred: N => Boolean, bigpred: N => Boolean)(implicit q: Quoted): TypedPipe[Edge[N, Double]] = { val groupedOnSrc = graph .filter { e => smallpred(e.to) || bigpred(e.to) } .map { e => (e.from, (e.to, e.data.degree)) } @@ -245,7 +246,7 @@ class DiscoInCosine[N](minCos: Double, delta: Double, boundedProb: Double, reduc private val oversample = (-2.0 * scala.math.log(boundedProb) / (delta * delta)) / minCos def apply(graph: TypedPipe[Edge[N, InDegree]], - smallpred: N => Boolean, bigpred: N => Boolean): TypedPipe[Edge[N, Double]] = { + smallpred: N => Boolean, bigpred: N => Boolean)(implicit q: Quoted): TypedPipe[Edge[N, Double]] = { val bigGroupedOnSrc = graph .filter { e => bigpred(e.to) } .map { e => (e.from, (e.to, e.data.degree)) } @@ -269,7 +270,7 @@ class DimsumInCosine[N](minCos: Double, delta: Double, boundedProb: Double, redu private val oversample = (-2.0 * scala.math.log(boundedProb) / (delta * delta)) / minCos def apply(graph: TypedPipe[Edge[N, (Weight, L2Norm)]], - smallpred: N => Boolean, bigpred: N => Boolean): TypedPipe[Edge[N, Double]] = { + smallpred: N => Boolean, bigpred: N => Boolean)(implicit q: Quoted): TypedPipe[Edge[N, Double]] = { val bigGroupedOnSrc = graph .filter { e => bigpred(e.to) } .map { e => (e.from, (e.to, e.data._1.weight, e.data._2.norm)) } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/GeneratedFlattenGroup.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/GeneratedFlattenGroup.scala index 458c100dce..ed50d8ac7a 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/GeneratedFlattenGroup.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/GeneratedFlattenGroup.scala @@ -1,6 +1,8 @@ // following were autogenerated by ./codegen/flatten_group_generator.rb at Mon Dec 01 19:29:52 -0800 2014 do not edit package com.twitter.scalding.typed +import com.twitter.scalding.quotation.Quoted + /** * Autogenerated methods for flattening the nested value tuples that result after * joining many pipes together. These methods can be used directly, or via the @@ -17,7 +19,7 @@ object FlattenGroup { } class FlattenLeftJoin3[KEY, KLL[KLL_K, +KLL_V] <: KeyedListLike[KLL_K, KLL_V, KLL], A, B, C](nested: KLL[KEY, ((A, B), C)]) { - def flattenValueTuple: KLL[KEY, (A, B, C)] = nested.mapValues { tup => FlattenGroup.flattenNestedTuple(tup) } + def flattenValueTuple(implicit q: Quoted): KLL[KEY, (A, B, C)] = nested.mapValues { tup => FlattenGroup.flattenNestedTuple(tup) } } implicit def toFlattenLeftJoin3[KEY, KLL[KLL_K, +KLL_V] <: KeyedListLike[KLL_K, KLL_V, KLL], A, B, C](nested: KLL[KEY, ((A, B), C)]): FlattenGroup.FlattenLeftJoin3[KEY, KLL, A, B, C] = new FlattenLeftJoin3(nested) @@ -28,7 +30,7 @@ object FlattenGroup { } class FlattenLeftJoin4[KEY, KLL[KLL_K, +KLL_V] <: KeyedListLike[KLL_K, KLL_V, KLL], A, B, C, D](nested: KLL[KEY, (((A, B), C), D)]) { - def flattenValueTuple: KLL[KEY, (A, B, C, D)] = nested.mapValues { tup => FlattenGroup.flattenNestedTuple(tup) } + def flattenValueTuple(implicit q: Quoted): KLL[KEY, (A, B, C, D)] = nested.mapValues { tup => FlattenGroup.flattenNestedTuple(tup) } } implicit def toFlattenLeftJoin4[KEY, KLL[KLL_K, +KLL_V] <: KeyedListLike[KLL_K, KLL_V, KLL], A, B, C, D](nested: KLL[KEY, (((A, B), C), D)]): FlattenGroup.FlattenLeftJoin4[KEY, KLL, A, B, C, D] = new FlattenLeftJoin4(nested) @@ -39,7 +41,7 @@ object FlattenGroup { } class FlattenLeftJoin5[KEY, KLL[KLL_K, +KLL_V] <: KeyedListLike[KLL_K, KLL_V, KLL], A, B, C, D, E](nested: KLL[KEY, ((((A, B), C), D), E)]) { - def flattenValueTuple: KLL[KEY, (A, B, C, D, E)] = nested.mapValues { tup => FlattenGroup.flattenNestedTuple(tup) } + def flattenValueTuple(implicit q: Quoted): KLL[KEY, (A, B, C, D, E)] = nested.mapValues { tup => FlattenGroup.flattenNestedTuple(tup) } } implicit def toFlattenLeftJoin5[KEY, KLL[KLL_K, +KLL_V] <: KeyedListLike[KLL_K, KLL_V, KLL], A, B, C, D, E](nested: KLL[KEY, ((((A, B), C), D), E)]): FlattenGroup.FlattenLeftJoin5[KEY, KLL, A, B, C, D, E] = new FlattenLeftJoin5(nested) @@ -50,7 +52,7 @@ object FlattenGroup { } class FlattenLeftJoin6[KEY, KLL[KLL_K, +KLL_V] <: KeyedListLike[KLL_K, KLL_V, KLL], A, B, C, D, E, F](nested: KLL[KEY, (((((A, B), C), D), E), F)]) { - def flattenValueTuple: KLL[KEY, (A, B, C, D, E, F)] = nested.mapValues { tup => FlattenGroup.flattenNestedTuple(tup) } + def flattenValueTuple(implicit q: Quoted): KLL[KEY, (A, B, C, D, E, F)] = nested.mapValues { tup => FlattenGroup.flattenNestedTuple(tup) } } implicit def toFlattenLeftJoin6[KEY, KLL[KLL_K, +KLL_V] <: KeyedListLike[KLL_K, KLL_V, KLL], A, B, C, D, E, F](nested: KLL[KEY, (((((A, B), C), D), E), F)]): FlattenGroup.FlattenLeftJoin6[KEY, KLL, A, B, C, D, E, F] = new FlattenLeftJoin6(nested) @@ -144,7 +146,7 @@ object FlattenGroup { } class FlattenOuterJoin3[KEY, KLL[KLL_K, +KLL_V] <: KeyedListLike[KLL_K, KLL_V, KLL], A, B, C](nested: KLL[KEY, (Option[(Option[A], Option[B])], Option[C])]) { - def flattenValueTuple: KLL[KEY, (Option[A], Option[B], Option[C])] = nested.mapValues { tup => FlattenGroup.flattenNestedOptionTuple(tup) } + def flattenValueTuple(implicit q: Quoted): KLL[KEY, (Option[A], Option[B], Option[C])] = nested.mapValues { tup => FlattenGroup.flattenNestedOptionTuple(tup) } } implicit def toFlattenOuterJoin3[KEY, KLL[KLL_K, +KLL_V] <: KeyedListLike[KLL_K, KLL_V, KLL], A, B, C](nested: KLL[KEY, (Option[(Option[A], Option[B])], Option[C])]): FlattenGroup.FlattenOuterJoin3[KEY, KLL, A, B, C] = new FlattenOuterJoin3(nested) @@ -157,7 +159,7 @@ object FlattenGroup { } class FlattenOuterJoin4[KEY, KLL[KLL_K, +KLL_V] <: KeyedListLike[KLL_K, KLL_V, KLL], A, B, C, D](nested: KLL[KEY, (Option[(Option[(Option[A], Option[B])], Option[C])], Option[D])]) { - def flattenValueTuple: KLL[KEY, (Option[A], Option[B], Option[C], Option[D])] = nested.mapValues { tup => FlattenGroup.flattenNestedOptionTuple(tup) } + def flattenValueTuple(implicit q: Quoted): KLL[KEY, (Option[A], Option[B], Option[C], Option[D])] = nested.mapValues { tup => FlattenGroup.flattenNestedOptionTuple(tup) } } implicit def toFlattenOuterJoin4[KEY, KLL[KLL_K, +KLL_V] <: KeyedListLike[KLL_K, KLL_V, KLL], A, B, C, D](nested: KLL[KEY, (Option[(Option[(Option[A], Option[B])], Option[C])], Option[D])]): FlattenGroup.FlattenOuterJoin4[KEY, KLL, A, B, C, D] = new FlattenOuterJoin4(nested) @@ -171,7 +173,7 @@ object FlattenGroup { } class FlattenOuterJoin5[KEY, KLL[KLL_K, +KLL_V] <: KeyedListLike[KLL_K, KLL_V, KLL], A, B, C, D, E](nested: KLL[KEY, (Option[(Option[(Option[(Option[A], Option[B])], Option[C])], Option[D])], Option[E])]) { - def flattenValueTuple: KLL[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E])] = nested.mapValues { tup => FlattenGroup.flattenNestedOptionTuple(tup) } + def flattenValueTuple(implicit q: Quoted): KLL[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E])] = nested.mapValues { tup => FlattenGroup.flattenNestedOptionTuple(tup) } } implicit def toFlattenOuterJoin5[KEY, KLL[KLL_K, +KLL_V] <: KeyedListLike[KLL_K, KLL_V, KLL], A, B, C, D, E](nested: KLL[KEY, (Option[(Option[(Option[(Option[A], Option[B])], Option[C])], Option[D])], Option[E])]): FlattenGroup.FlattenOuterJoin5[KEY, KLL, A, B, C, D, E] = new FlattenOuterJoin5(nested) @@ -186,7 +188,7 @@ object FlattenGroup { } class FlattenOuterJoin6[KEY, KLL[KLL_K, +KLL_V] <: KeyedListLike[KLL_K, KLL_V, KLL], A, B, C, D, E, F](nested: KLL[KEY, (Option[(Option[(Option[(Option[(Option[A], Option[B])], Option[C])], Option[D])], Option[E])], Option[F])]) { - def flattenValueTuple: KLL[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F])] = nested.mapValues { tup => FlattenGroup.flattenNestedOptionTuple(tup) } + def flattenValueTuple(implicit q: Quoted): KLL[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F])] = nested.mapValues { tup => FlattenGroup.flattenNestedOptionTuple(tup) } } implicit def toFlattenOuterJoin6[KEY, KLL[KLL_K, +KLL_V] <: KeyedListLike[KLL_K, KLL_V, KLL], A, B, C, D, E, F](nested: KLL[KEY, (Option[(Option[(Option[(Option[(Option[A], Option[B])], Option[C])], Option[D])], Option[E])], Option[F])]): FlattenGroup.FlattenOuterJoin6[KEY, KLL, A, B, C, D, E, F] = new FlattenOuterJoin6(nested) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/Grouped.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/Grouped.scala index 612ddf79f9..a19b840cf9 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/Grouped.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/Grouped.scala @@ -22,6 +22,7 @@ import com.twitter.scalding.typed.functions.ComposedFunctions.ComposedMapGroup import scala.collection.JavaConverters._ import scala.util.hashing.MurmurHash3 import java.io.Serializable +import com.twitter.scalding.quotation.Quoted object CoGroupable extends Serializable { /** @@ -234,7 +235,10 @@ object CoGrouped extends Serializable { } final case class FilterKeys[K, V](on: CoGrouped[K, V], fn: K => Boolean) extends CoGrouped[K, V] { - val inputs = on.inputs.map(TypedPipe.FilterKeys(_, fn)) + val inputs = { + implicit val q: Quoted = Quoted.internal + on.inputs.map(TypedPipe.FilterKeys(_, fn)) + } def reducers = on.reducers def keyOrdering = on.keyOrdering def joinFunction = on.joinFunction @@ -271,21 +275,15 @@ sealed trait CoGrouped[K, +R] extends KeyedListLike[K, R, CoGrouped] * but it is not clear how to generalize that for general cogrouping functions. * For now, just do a normal take. */ - override def bufferedTake(n: Int): CoGrouped[K, R] = + override def bufferedTake(n: Int)(implicit q: Quoted): CoGrouped[K, R] = take(n) // Filter the keys before doing the join - override def filterKeys(fn: K => Boolean): CoGrouped[K, R] = + override def filterKeys(fn: K => Boolean)(implicit q: Quoted): CoGrouped[K, R] = CoGrouped.FilterKeys(this, fn) - override def mapGroup[R1](fn: (K, Iterator[R]) => Iterator[R1]): CoGrouped[K, R1] = - /* - * After the join, if the key has no values, don't present it to the mapGroup - * function. Doing so would break the invariant: - * - * a.join(b).toTypedPipe.group.mapGroup(fn) == a.join(b).mapGroup(fn) - */ - CoGrouped.MapGroup(this, Grouped.addEmptyGuard(fn)) + override def mapGroup[R1](fn: (K, Iterator[R]) => Iterator[R1])(implicit q: Quoted): CoGrouped[K, R1] = + CoGrouped.MapGroup(this, fn) override def toTypedPipe: TypedPipe[(K, R)] = TypedPipe.CoGroupedPipe(this) @@ -377,16 +375,16 @@ object Grouped extends Serializable { * of each key in memory on the reducer. */ sealed trait Sortable[+T, +Sorted[+_]] { - def withSortOrdering[U >: T](so: Ordering[U]): Sorted[T] + def withSortOrdering[U >: T](so: Ordering[U])(implicit q: Quoted): Sorted[T] - def sortBy[B: Ordering](fn: (T) => B): Sorted[T] = + def sortBy[B: Ordering](fn: (T) => B)(implicit q: Quoted): Sorted[T] = withSortOrdering(Ordering.by(fn)) // Sorts the values for each key - def sorted[B >: T](implicit ord: Ordering[B]): Sorted[T] = + def sorted[B >: T](implicit ord: Ordering[B], q: Quoted): Sorted[T] = withSortOrdering(ord) - def sortWith(lt: (T, T) => Boolean): Sorted[T] = + def sortWith(lt: (T, T) => Boolean)(implicit q: Quoted): Sorted[T] = withSortOrdering(Ordering.fromLessThan(lt)) } @@ -594,11 +592,11 @@ final case class IdentityReduce[K, V1, V2]( * before sending to the mappers. This is a big help if there are relatively * few keys and n is relatively small. */ - override def bufferedTake(n: Int) = + override def bufferedTake(n: Int)(implicit q: Quoted) = toUIR.bufferedTake(n) - override def withSortOrdering[U >: V2](so: Ordering[U]): IdentityValueSortedReduce[K, V2, V2] = - IdentityValueSortedReduce[K, V2, V2](keyOrdering, mappedV2, TypedPipe.narrowOrdering(so), reducers, descriptions, implicitly) + override def withSortOrdering[U >: V2](so: Ordering[U])(implicit q: Quoted): IdentityValueSortedReduce[K, V2, V2] = + IdentityValueSortedReduce[K, V1](keyOrdering, mapped, so, reducers, descriptions) override def withReducers(red: Int): IdentityReduce[K, V1, V2] = copy(reducers = Some(red)) @@ -606,23 +604,23 @@ final case class IdentityReduce[K, V1, V2]( override def withDescription(description: String): IdentityReduce[K, V1, V2] = copy(descriptions = descriptions :+ description) - override def filterKeys(fn: K => Boolean) = + override def filterKeys(fn: K => Boolean)(implicit q: Quoted) = toUIR.filterKeys(fn) - override def mapGroup[V3](fn: (K, Iterator[V2]) => Iterator[V3]) = { + override def mapGroup[V3](fn: (K, Iterator[V2]) => Iterator[V3])(implicit q: Quoted) = { // Only pass non-Empty iterators to subsequent functions IteratorMappedReduce(keyOrdering, mappedV2, Grouped.addEmptyGuard(fn), reducers, descriptions) } // It would be nice to return IdentityReduce here, but // the type constraints prevent it currently - override def mapValues[V3](fn: V2 => V3) = + override def mapValues[V3](fn: V2 => V3)(implicit q: Quoted) = toUIR.mapValues(fn) // This is not correct in the type-system, but would be nice to encode //override def mapValues[V3](fn: V1 => V3) = IdentityReduce(keyOrdering, mapped.mapValues(fn), reducers) - override def sum[U >: V2](implicit sg: Semigroup[U]) = { + override def sum[U >: V2](implicit sg: Semigroup[U], q: Quoted) = { // there is no sort, mapValueStream or force to reducers: val upipe: TypedPipe[(K, U)] = mappedV2 // use covariance to set the type UnsortedIdentityReduce[K, U, U](keyOrdering, upipe.sumByLocalKeys, reducers, descriptions, implicitly).sumLeft @@ -646,7 +644,7 @@ final case class UnsortedIdentityReduce[K, V1, V2]( * before sending to the reducers. This is a big help if there are relatively * few keys and n is relatively small. */ - override def bufferedTake(n: Int) = + override def bufferedTake(n: Int)(implicit q: Quoted) = if (n < 1) { // This means don't take anything, which is legal, but strange filterKeys(Constant(false)) @@ -677,7 +675,7 @@ final case class UnsortedIdentityReduce[K, V1, V2]( override def withDescription(description: String): UnsortedIdentityReduce[K, V1, V2] = copy(descriptions = descriptions :+ description) - override def filterKeys(fn: K => Boolean) = + override def filterKeys(fn: K => Boolean)(implicit q: Quoted) = UnsortedIdentityReduce[K, V1, V2](keyOrdering, mapped.filterKeys(fn), reducers, descriptions, evidence) private[this] def mappedV2 = { @@ -685,16 +683,16 @@ final case class UnsortedIdentityReduce[K, V1, V2]( evidence.subst[TK](mapped) } - override def mapGroup[V3](fn: (K, Iterator[V2]) => Iterator[V3]) = + override def mapGroup[V3](fn: (K, Iterator[V2]) => Iterator[V3])(implicit q: Quoted) = // Only pass non-Empty iterators to subsequent functions IteratorMappedReduce[K, V2, V3](keyOrdering, mappedV2, Grouped.addEmptyGuard(fn), reducers, descriptions) // It would be nice to return IdentityReduce here, but // the type constraints prevent it currently - override def mapValues[V3](fn: V2 => V3) = + override def mapValues[V3](fn: V2 => V3)(implicit q: Quoted) = UnsortedIdentityReduce[K, V3, V3](keyOrdering, mappedV2.mapValues(fn), reducers, descriptions, implicitly) - override def sum[U >: V2](implicit sg: Semigroup[U]) = { + override def sum[U >: V2](implicit sg: Semigroup[U])(implicit q: Quoted) = { // there is no sort, mapValueStream or force to reducers: val upipe: TypedPipe[(K, U)] = mappedV2 // use covariance to set the type UnsortedIdentityReduce[K, U, U](keyOrdering, upipe.sumByLocalKeys, reducers, descriptions, implicitly).sumLeft @@ -724,11 +722,11 @@ final case class IdentityValueSortedReduce[K, V1, V2]( override def withDescription(description: String): IdentityValueSortedReduce[K, V1, V2] = IdentityValueSortedReduce[K, V1, V2](keyOrdering, mapped, valueSort, reducers, descriptions = descriptions :+ description, evidence) - override def filterKeys(fn: K => Boolean) = + override def filterKeys(fn: K => Boolean)(implicit q: Quoted) = // copy fails to get the types right, :/ IdentityValueSortedReduce[K, V1, V2](keyOrdering, mapped.filterKeys(fn), valueSort, reducers, descriptions, evidence) - override def mapGroup[V3](fn: (K, Iterator[V2]) => Iterator[V3]) = { + override def mapGroup[V3](fn: (K, Iterator[V2]) => Iterator[V3])(implicit q: Quoted) = { // Only pass non-Empty iterators to subsequent functions val gfn = Grouped.addEmptyGuard(fn) type TK[V] = TypedPipe[(K, V)] @@ -740,7 +738,7 @@ final case class IdentityValueSortedReduce[K, V1, V2]( * before sending to the reducers. This is a big help if there are relatively * few keys and n is relatively small. */ - override def bufferedTake(n: Int): SortedGrouped[K, V2] = + override def bufferedTake(n: Int)(implicit q: Quoted): SortedGrouped[K, V2] = if (n <= 0) { // This means don't take anything, which is legal, but strange filterKeys(Constant(false)) @@ -762,7 +760,7 @@ final case class IdentityValueSortedReduce[K, V1, V2]( * To force a memory-based take, use bufferedTake * Otherwise, we send all the values to the reducers */ - override def take(n: Int) = + override def take(n: Int)(implicit q: Quoted) = if (n <= 1) bufferedTake(n) else mapValueStream(_.take(n)) } @@ -780,7 +778,7 @@ final case class ValueSortedReduce[K, V1, V2]( * After sorting, then reducing, there is no chance * to operate in the mappers. Just call take. */ - override def bufferedTake(n: Int) = take(n) + override def bufferedTake(n: Int)(implicit q: Quoted) = take(n) override def withReducers(red: Int) = // copy infers loose types. :( @@ -791,11 +789,11 @@ final case class ValueSortedReduce[K, V1, V2]( ValueSortedReduce[K, V1, V2]( keyOrdering, mapped, valueSort, reduceFn, reducers, descriptions :+ description) - override def filterKeys(fn: K => Boolean) = + override def filterKeys(fn: K => Boolean)(implicit q: Quoted) = // copy fails to get the types right, :/ ValueSortedReduce[K, V1, V2](keyOrdering, mapped.filterKeys(fn), valueSort, reduceFn, reducers, descriptions) - override def mapGroup[V3](fn: (K, Iterator[V2]) => Iterator[V3]) = { + override def mapGroup[V3](fn: (K, Iterator[V2]) => Iterator[V3])(implicit q: Quoted) = { // we don't need the empty guard here because ComposedMapGroup already does it val newReduce = ComposedMapGroup(reduceFn, fn) ValueSortedReduce[K, V1, V3]( @@ -815,7 +813,7 @@ final case class IteratorMappedReduce[K, V1, V2]( * After reducing, we are always * operating in memory. Just call take. */ - override def bufferedTake(n: Int) = take(n) + override def bufferedTake(n: Int)(implicit q: Quoted) = take(n) override def withReducers(red: Int): IteratorMappedReduce[K, V1, V2] = copy(reducers = Some(red)) @@ -823,10 +821,10 @@ final case class IteratorMappedReduce[K, V1, V2]( override def withDescription(description: String): IteratorMappedReduce[K, V1, V2] = copy(descriptions = descriptions :+ description) - override def filterKeys(fn: K => Boolean) = + override def filterKeys(fn: K => Boolean)(implicit q: Quoted) = copy(mapped = mapped.filterKeys(fn)) - override def mapGroup[V3](fn: (K, Iterator[V2]) => Iterator[V3]) = { + override def mapGroup[V3](fn: (K, Iterator[V2]) => Iterator[V3])(implicit q: Quoted) = { // we don't need the empty guard here because ComposedMapGroup already does it val newReduce = ComposedMapGroup(reduceFn, fn) copy(reduceFn = newReduce) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/KeyedList.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/KeyedList.scala index 2321633341..4987f28ef7 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/KeyedList.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/KeyedList.scala @@ -24,6 +24,7 @@ import com.twitter.algebird.mutable.PriorityQueueMonoid import com.twitter.scalding._ import com.twitter.scalding.typed.functions._ +import com.twitter.scalding.quotation.Quoted object KeyedListLike { /** KeyedListLike items are implicitly convertable to TypedPipe */ @@ -61,7 +62,7 @@ trait KeyedListLike[K, +T, +This[K, +T] <: KeyedListLike[K, T, This]] extends Se * (as you may aggregate n items in a memory heap for each key) * If you get OOM issues, try to resolve using the method `take` instead. */ - def bufferedTake(n: Int): This[K, T] + def bufferedTake(n: Int)(implicit q: Quoted): This[K, T] /* Here is an example implementation, but since each subclass of KeyedListLike has its own constaints, this is always to be @@ -96,7 +97,7 @@ trait KeyedListLike[K, +T, +This[K, +T] <: KeyedListLike[K, T, This]] extends Se * filter keys on a predicate. More efficient than filter if you are * only looking at keys */ - def filterKeys(fn: K => Boolean): This[K, T] + def filterKeys(fn: K => Boolean)(implicit q: Quoted): This[K, T] /* an inefficient implementation is below, but * since this can always be pushed mapside, we should avoid * using this implementation, lest we accidentally forget to @@ -116,7 +117,7 @@ trait KeyedListLike[K, +T, +This[K, +T] <: KeyedListLike[K, T, This]] extends Se * Note, any key that has all values removed will not appear in subsequent * .mapGroup/mapValueStream */ - def mapGroup[V](smfn: (K, Iterator[T]) => Iterator[V]): This[K, V] + def mapGroup[V](smfn: (K, Iterator[T]) => Iterator[V])(implicit q: Quoted): This[K, V] /////////// /// The below are all implemented in terms of the above: @@ -125,9 +126,9 @@ trait KeyedListLike[K, +T, +This[K, +T] <: KeyedListLike[K, T, This]] extends Se /** * Use Algebird Aggregator to do the reduction */ - def aggregate[B, C](agg: Aggregator[T, B, C]): This[K, C] = + def aggregate[B, C](agg: Aggregator[T, B, C])(implicit q: Quoted): This[K, C] = mapValues[B](AggPrepare(agg)) - .sum[B](agg.semigroup) + .sum[B](agg.semigroup, q) .mapValues[C](AggPresent(agg)) /** @@ -136,28 +137,28 @@ trait KeyedListLike[K, +T, +This[K, +T] <: KeyedListLike[K, T, This]] extends Se * as long as possible: this minimizes the times we go in * and out of cascading/hadoop types. */ - def filter(fn: ((K, T)) => Boolean): This[K, T] = + def filter(fn: ((K, T)) => Boolean)(implicit q: Quoted): This[K, T] = mapGroup(FilterGroup(fn)) /** * flatten the values * Useful after sortedTake, for instance */ - def flattenValues[U](implicit ev: T <:< TraversableOnce[U]): This[K, U] = + def flattenValues[U](implicit ev: T <:< TraversableOnce[U], q: Quoted): This[K, U] = flatMapValues(Widen(SubTypes.fromEv(ev))) /** * This is just short hand for mapValueStream(identity), it makes sure the * planner sees that you want to force a shuffle. For expert tuning */ - def forceToReducers: This[K, T] = + def forceToReducers(implicit q: Quoted): This[K, T] = mapValueStream(Identity()) /** * Use this to get the first value encountered. * prefer this to take(1). */ - def head: This[K, T] = sum(HeadSemigroup[T]()) + def head(implicit q: Quoted): This[K, T] = sum(HeadSemigroup[T](), q) /** * This is a special case of mapValueStream, but can be optimized because it doesn't need @@ -165,21 +166,21 @@ trait KeyedListLike[K, +T, +This[K, +T] <: KeyedListLike[K, T, This]] extends Se * mapValueStream { _.map { fn } } * but for Grouped we can avoid resorting to mapValueStream */ - def mapValues[V](fn: T => V): This[K, V] = + def mapValues[V](fn: T => V)(implicit q: Quoted): This[K, V] = mapGroup(MapGroupMapValues(fn)) /** * Similar to mapValues, but works like flatMap, returning a collection of outputs * for each value input. */ - def flatMapValues[V](fn: T => TraversableOnce[V]): This[K, V] = + def flatMapValues[V](fn: T => TraversableOnce[V])(implicit q: Quoted): This[K, V] = mapGroup(MapGroupFlatMapValues(fn)) /** * Use this when you don't care about the key for the group, * otherwise use mapGroup */ - def mapValueStream[V](smfn: Iterator[T] => Iterator[V]): This[K, V] = + def mapValueStream[V](smfn: Iterator[T] => Iterator[V])(implicit q: Quoted): This[K, V] = mapGroup(MapValueStream(smfn)) /** @@ -191,22 +192,22 @@ trait KeyedListLike[K, +T, +This[K, +T] <: KeyedListLike[K, T, This]] extends Se * Semigroups MAY have a faster implementation of sum for iterators, * so prefer using sum/sumLeft to reduce */ - def sum[U >: T](implicit sg: Semigroup[U]): This[K, U] = sumLeft[U] + def sum[U >: T](implicit sg: Semigroup[U], q: Quoted): This[K, U] = sumLeft[U] /** * reduce with fn which must be associative and commutative. * Like the above this can be optimized in some Grouped cases. * If you don't have a commutative operator, use reduceLeft */ - def reduce[U >: T](fn: (U, U) => U): This[K, U] = - sum(SemigroupFromFn(fn)) + def reduce[U >: T](fn: (U, U) => U)(implicit q: Quoted): This[K, U] = + sum(SemigroupFromFn(fn), q) /** * Take the largest k things according to the implicit ordering. * Useful for top-k without having to call ord.reverse */ - def sortedReverseTake(k: Int)(implicit ord: Ordering[_ >: T]): This[K, Seq[T]] = - sortedTake(k)(ord.reverse) + def sortedReverseTake(k: Int)(implicit ord: Ordering[_ >: T], q: Quoted): This[K, Seq[T]] = + sortedTake(k)(ord.reverse, q) /** * This implements bottom-k (smallest k items) on each mapper for each key, then @@ -214,47 +215,47 @@ trait KeyedListLike[K, +T, +This[K, +T] <: KeyedListLike[K, T, This]] extends Se * than using .take if k * (number of Keys) is small enough * to fit in memory. */ - def sortedTake(k: Int)(implicit ord: Ordering[_ >: T]): This[K, Seq[T]] = { + def sortedTake(k: Int)(implicit ord: Ordering[_ >: T], q: Quoted): This[K, Seq[T]] = { val ordT: Ordering[T] = TypedPipe.narrowOrdering(ord) val mon = new PriorityQueueMonoid[T](k)(ordT) mapValues(mon.build(_)) - .sum(mon) // results in a PriorityQueue + .sum(mon, q) // results in a PriorityQueue // scala can't infer the type, possibly due to the view bound on TypedPipe .mapValues(_.iterator.asScala.toList.sorted(ordT)) } /** Like the above, but with a less than operation for the ordering */ - def sortWithTake[U >: T](k: Int)(lessThan: (U, U) => Boolean): This[K, Seq[T]] = - sortedTake(k)(Ordering.fromLessThan(lessThan)) + def sortWithTake[U >: T](k: Int)(lessThan: (U, U) => Boolean)(implicit q: Quoted): This[K, Seq[T]] = + sortedTake(k)(Ordering.fromLessThan(lessThan), q) /** For each key, Return the product of all the values */ - def product[U >: T](implicit ring: Ring[U]): This[K, U] = - sum(SemigroupFromProduct(ring)) + def product[U >: T](implicit ring: Ring[U], q: Quoted): This[K, U] = + sum(SemigroupFromProduct(ring), q) /** For each key, count the number of values that satisfy a predicate */ - def count(fn: T => Boolean): This[K, Long] = + def count(fn: T => Boolean)(implicit q: Quoted): This[K, Long] = mapValues(Count(fn)).sum /** For each key, check to see if a predicate is true for all Values*/ - def forall(fn: T => Boolean): This[K, Boolean] = + def forall(fn: T => Boolean)(implicit q: Quoted): This[K, Boolean] = mapValues(fn).product /** * For each key, selects all elements except first n ones. */ - def drop(n: Int): This[K, T] = + def drop(n: Int)(implicit q: Quoted): This[K, T] = mapValueStream(Drop(n)) /** * For each key, Drops longest prefix of elements that satisfy the given predicate. */ - def dropWhile(p: T => Boolean): This[K, T] = + def dropWhile(p: T => Boolean)(implicit q: Quoted): This[K, T] = mapValueStream(DropWhile(p)) /** * For each key, Selects first n elements. Don't use this if n == 1, head is faster in that case. */ - def take(n: Int): This[K, T] = + def take(n: Int)(implicit q: Quoted): This[K, T] = if (n < 1) filterKeys(Constant(false)) // just don't keep anything else if (n == 1) head else mapValueStream(Take(n)) @@ -262,7 +263,7 @@ trait KeyedListLike[K, +T, +This[K, +T] <: KeyedListLike[K, T, This]] extends Se /** * For each key, Takes longest prefix of elements that satisfy the given predicate. */ - def takeWhile(p: T => Boolean): This[K, T] = + def takeWhile(p: T => Boolean)(implicit q: Quoted): This[K, T] = mapValueStream(TakeWhile(p)) /** @@ -270,22 +271,22 @@ trait KeyedListLike[K, +T, +This[K, +T] <: KeyedListLike[K, T, This]] extends Se * If you need to do several custom folds over the same data, use Fold.join * and this method */ - def fold[V](f: Fold[T, V]): This[K, V] = + def fold[V](f: Fold[T, V])(implicit q: Quoted): This[K, V] = mapValueStream(FoldIterator(f)) /** * If the fold depends on the key, use this method to construct * the fold for each key */ - def foldWithKey[V](fn: K => Fold[T, V]): This[K, V] = + def foldWithKey[V](fn: K => Fold[T, V])(implicit q: Quoted): This[K, V] = mapGroup(FoldWithKeyIterator(fn)) /** For each key, fold the values. see scala.collection.Iterable.foldLeft */ - def foldLeft[B](z: B)(fn: (B, T) => B): This[K, B] = + def foldLeft[B](z: B)(fn: (B, T) => B)(implicit q: Quoted): This[K, B] = mapValueStream(FoldLeftIterator(z, fn)) /** For each key, scanLeft the values. see scala.collection.Iterable.scanLeft */ - def scanLeft[B](z: B)(fn: (B, T) => B): This[K, B] = + def scanLeft[B](z: B)(fn: (B, T) => B)(implicit q: Quoted): This[K, B] = mapValueStream(ScanLeftIterator(z, fn)) /** @@ -294,31 +295,31 @@ trait KeyedListLike[K, +T, +This[K, +T] <: KeyedListLike[K, T, This]] extends Se * Makes sense when you want to reduce, but in a particular sorted order. * the old value comes in on the left. */ - def reduceLeft[U >: T](fn: (U, U) => U): This[K, U] = - sumLeft[U](SemigroupFromFn(fn)) + def reduceLeft[U >: T](fn: (U, U) => U)(implicit q: Quoted): This[K, U] = + sumLeft[U](SemigroupFromFn(fn), q) /** * Semigroups MAY have a faster implementation of sum for iterators, * so prefer using sum/sumLeft to reduce/reduceLeft */ - def sumLeft[U >: T](implicit sg: Semigroup[U]): This[K, U] = + def sumLeft[U >: T](implicit sg: Semigroup[U], q: Quoted): This[K, U] = mapValueStream[U](SumAll(sg)) /** For each key, give the number of values */ - def size: This[K, Long] = mapValues(Constant(1L)).sum + def size(implicit q: Quoted): This[K, Long] = mapValues(Constant(1L)).sum /** * For each key, give the number of unique values. WARNING: May OOM. * This assumes the values for each key can fit in memory. */ - def distinctSize: This[K, Long] = + def distinctSize(implicit q: Quoted): This[K, Long] = toSet[T].mapValues(SizeOfSet()) /** * For each key, remove duplicate values. WARNING: May OOM. * This assumes the values for each key can fit in memory. */ - def distinctValues: This[K, T] = toSet[T].flattenValues + def distinctValues(implicit q: Quoted): This[K, T] = toSet[T].flattenValues /** * AVOID THIS IF POSSIBLE @@ -331,7 +332,7 @@ trait KeyedListLike[K, +T, +This[K, +T] <: KeyedListLike[K, T, This]] extends Se * toList does not decrease the size of the data at all, so in practice * it only wastes effort to try to cache. */ - def toList: This[K, List[T]] = mapValueStream(ToList[T]()) + def toList(implicit q: Quoted): This[K, List[T]] = mapValueStream(ToList[T]()) /** * AVOID THIS IF POSSIBLE * Same risks apply here as to toList: you may OOM. See toList. @@ -340,22 +341,22 @@ trait KeyedListLike[K, +T, +This[K, +T] <: KeyedListLike[K, T, This]] extends Se * but Set is invariant. See: * http://stackoverflow.com/questions/676615/why-is-scalas-immutable-set-not-covariant-in-its-type */ - def toSet[U >: T]: This[K, Set[U]] = mapValues(ToSet[U]()).sum + def toSet[U >: T](implicit q: Quoted): This[K, Set[U]] = mapValues(ToSet[U]()).sum /** For each key, give the maximum value*/ - def max[B >: T](implicit cmp: Ordering[B]): This[K, T] = + def max[B >: T](implicit cmp: Ordering[B], q: Quoted): This[K, T] = reduce(MaxOrd[T, B](cmp)) /** For each key, give the maximum value by some function*/ - def maxBy[B](fn: T => B)(implicit cmp: Ordering[B]): This[K, T] = + def maxBy[B](fn: T => B)(implicit cmp: Ordering[B], q: Quoted): This[K, T] = reduce(MaxOrdBy(fn, cmp)) /** For each key, give the minimum value*/ - def min[B >: T](implicit cmp: Ordering[B]): This[K, T] = + def min[B >: T](implicit cmp: Ordering[B], q: Quoted): This[K, T] = reduce(MinOrd[T, B](cmp)) /** For each key, give the minimum value by some function*/ - def minBy[B](fn: T => B)(implicit cmp: Ordering[B]): This[K, T] = + def minBy[B](fn: T => B)(implicit cmp: Ordering[B], q: Quoted): This[K, T] = reduce(MinOrdBy(fn, cmp)) @@ -372,7 +373,7 @@ trait KeyedListLike[K, +T, +This[K, +T] <: KeyedListLike[K, T, This]] extends Se mapValueStream(SumAll(RequireSingleSemigroup())) /** Convert to a TypedPipe and only keep the keys */ - def keys: TypedPipe[K] = toTypedPipe.keys + def keys(implicit q: Quoted): TypedPipe[K] = toTypedPipe.keys /** Convert to a TypedPipe and only keep the values */ - def values: TypedPipe[T] = toTypedPipe.values + def values(implicit q: Quoted): TypedPipe[T] = toTypedPipe.values } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/LookupJoin.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/LookupJoin.scala index 4df34601a0..2721e78be1 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/LookupJoin.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/LookupJoin.scala @@ -19,6 +19,7 @@ package com.twitter.scalding.typed import java.io.Serializable import com.twitter.algebird.Semigroup +import com.twitter.scalding.quotation.Quoted /* Copyright 2013 Twitter, Inc. @@ -82,7 +83,7 @@ object LookupJoin extends Serializable { def apply[T: Ordering, K: Ordering, V, JoinedV]( left: TypedPipe[(T, (K, V))], right: TypedPipe[(T, (K, JoinedV))], - reducers: Option[Int] = None): TypedPipe[(T, (K, (V, Option[JoinedV])))] = + reducers: Option[Int] = None)(implicit q: Quoted): TypedPipe[(T, (K, (V, Option[JoinedV])))] = withWindow(left, right, reducers)((_, _) => true) @@ -92,7 +93,7 @@ object LookupJoin extends Serializable { */ def rightSumming[T: Ordering, K: Ordering, V, JoinedV: Semigroup](left: TypedPipe[(T, (K, V))], right: TypedPipe[(T, (K, JoinedV))], - reducers: Option[Int] = None): TypedPipe[(T, (K, (V, Option[JoinedV])))] = + reducers: Option[Int] = None)(implicit q: Quoted): TypedPipe[(T, (K, (V, Option[JoinedV])))] = withWindowRightSumming(left, right, reducers)((_, _) => true) /** @@ -102,7 +103,7 @@ object LookupJoin extends Serializable { */ def withWindow[T: Ordering, K: Ordering, V, JoinedV](left: TypedPipe[(T, (K, V))], right: TypedPipe[(T, (K, JoinedV))], - reducers: Option[Int] = None)(gate: (T, T) => Boolean): TypedPipe[(T, (K, (V, Option[JoinedV])))] = { + reducers: Option[Int] = None)(gate: (T, T) => Boolean)(implicit q: Quoted): TypedPipe[(T, (K, (V, Option[JoinedV])))] = { implicit val keepNew: Semigroup[JoinedV] = Semigroup.from { (older, newer) => newer } withWindowRightSumming(left, right, reducers)(gate) @@ -115,7 +116,7 @@ object LookupJoin extends Serializable { */ def withWindowRightSumming[T: Ordering, K: Ordering, V, JoinedV: Semigroup](left: TypedPipe[(T, (K, V))], right: TypedPipe[(T, (K, JoinedV))], - reducers: Option[Int] = None)(gate: (T, T) => Boolean): TypedPipe[(T, (K, (V, Option[JoinedV])))] = { + reducers: Option[Int] = None)(gate: (T, T) => Boolean)(implicit q: Quoted): TypedPipe[(T, (K, (V, Option[JoinedV])))] = { /** * Implicit ordering on an either that doesn't care about the * actual container values, puts the lookups before the service writes diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/MultiJoin.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/MultiJoin.scala index 6ac4a59e49..df08d011c8 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/MultiJoin.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/MultiJoin.scala @@ -1,6 +1,8 @@ // following were autogenerated by ./codegen/multi_join_generator.rb at Mon Dec 01 19:28:47 -0800 2014 do not edit package com.twitter.scalding.typed +import com.twitter.scalding.quotation.Quoted + /** * This is an autogenerated object which gives you easy access to * doing N-way joins so the types are cleaner. However, it just calls @@ -9,28 +11,28 @@ package com.twitter.scalding.typed object MultiJoin extends java.io.Serializable { import com.twitter.scalding.typed.FlattenGroup._ - def apply[KEY, A, B](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B]): CoGrouped[KEY, (A, B)] = + def apply[KEY, A, B](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B])(implicit qt: Quoted): CoGrouped[KEY, (A, B)] = a.join(b) - def apply[KEY, A, B, C](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C]): CoGrouped[KEY, (A, B, C)] = + def apply[KEY, A, B, C](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C])(implicit qt: Quoted): CoGrouped[KEY, (A, B, C)] = a.join(b) .join(c) .mapValues { tup => flattenNestedTuple(tup) } - def apply[KEY, A, B, C, D](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D]): CoGrouped[KEY, (A, B, C, D)] = + def apply[KEY, A, B, C, D](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D])(implicit qt: Quoted): CoGrouped[KEY, (A, B, C, D)] = a.join(b) .join(c) .join(d) .mapValues { tup => flattenNestedTuple(tup) } - def apply[KEY, A, B, C, D, E](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E]): CoGrouped[KEY, (A, B, C, D, E)] = + def apply[KEY, A, B, C, D, E](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E])(implicit qt: Quoted): CoGrouped[KEY, (A, B, C, D, E)] = a.join(b) .join(c) .join(d) .join(e) .mapValues { tup => flattenNestedTuple(tup) } - def apply[KEY, A, B, C, D, E, F](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F]): CoGrouped[KEY, (A, B, C, D, E, F)] = + def apply[KEY, A, B, C, D, E, F](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F])(implicit qt: Quoted): CoGrouped[KEY, (A, B, C, D, E, F)] = a.join(b) .join(c) .join(d) @@ -38,7 +40,7 @@ object MultiJoin extends java.io.Serializable { .join(f) .mapValues { tup => flattenNestedTuple(tup) } - def apply[KEY, A, B, C, D, E, F, G](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G]): CoGrouped[KEY, (A, B, C, D, E, F, G)] = + def apply[KEY, A, B, C, D, E, F, G](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G])(implicit qt: Quoted): CoGrouped[KEY, (A, B, C, D, E, F, G)] = a.join(b) .join(c) .join(d) @@ -47,7 +49,7 @@ object MultiJoin extends java.io.Serializable { .join(g) .mapValues { tup => flattenNestedTuple(tup) } - def apply[KEY, A, B, C, D, E, F, G, H](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H]): CoGrouped[KEY, (A, B, C, D, E, F, G, H)] = + def apply[KEY, A, B, C, D, E, F, G, H](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H])(implicit qt: Quoted): CoGrouped[KEY, (A, B, C, D, E, F, G, H)] = a.join(b) .join(c) .join(d) @@ -57,7 +59,7 @@ object MultiJoin extends java.io.Serializable { .join(h) .mapValues { tup => flattenNestedTuple(tup) } - def apply[KEY, A, B, C, D, E, F, G, H, I](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I]): CoGrouped[KEY, (A, B, C, D, E, F, G, H, I)] = + def apply[KEY, A, B, C, D, E, F, G, H, I](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I])(implicit qt: Quoted): CoGrouped[KEY, (A, B, C, D, E, F, G, H, I)] = a.join(b) .join(c) .join(d) @@ -68,7 +70,7 @@ object MultiJoin extends java.io.Serializable { .join(i) .mapValues { tup => flattenNestedTuple(tup) } - def apply[KEY, A, B, C, D, E, F, G, H, I, J](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J]): CoGrouped[KEY, (A, B, C, D, E, F, G, H, I, J)] = + def apply[KEY, A, B, C, D, E, F, G, H, I, J](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J])(implicit qt: Quoted): CoGrouped[KEY, (A, B, C, D, E, F, G, H, I, J)] = a.join(b) .join(c) .join(d) @@ -80,7 +82,7 @@ object MultiJoin extends java.io.Serializable { .join(j) .mapValues { tup => flattenNestedTuple(tup) } - def apply[KEY, A, B, C, D, E, F, G, H, I, J, K](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K]): CoGrouped[KEY, (A, B, C, D, E, F, G, H, I, J, K)] = + def apply[KEY, A, B, C, D, E, F, G, H, I, J, K](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K])(implicit qt: Quoted): CoGrouped[KEY, (A, B, C, D, E, F, G, H, I, J, K)] = a.join(b) .join(c) .join(d) @@ -93,7 +95,7 @@ object MultiJoin extends java.io.Serializable { .join(k) .mapValues { tup => flattenNestedTuple(tup) } - def apply[KEY, A, B, C, D, E, F, G, H, I, J, K, L](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L]): CoGrouped[KEY, (A, B, C, D, E, F, G, H, I, J, K, L)] = + def apply[KEY, A, B, C, D, E, F, G, H, I, J, K, L](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L])(implicit qt: Quoted): CoGrouped[KEY, (A, B, C, D, E, F, G, H, I, J, K, L)] = a.join(b) .join(c) .join(d) @@ -107,7 +109,7 @@ object MultiJoin extends java.io.Serializable { .join(l) .mapValues { tup => flattenNestedTuple(tup) } - def apply[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M]): CoGrouped[KEY, (A, B, C, D, E, F, G, H, I, J, K, L, M)] = + def apply[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M])(implicit qt: Quoted): CoGrouped[KEY, (A, B, C, D, E, F, G, H, I, J, K, L, M)] = a.join(b) .join(c) .join(d) @@ -122,7 +124,7 @@ object MultiJoin extends java.io.Serializable { .join(m) .mapValues { tup => flattenNestedTuple(tup) } - def apply[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N]): CoGrouped[KEY, (A, B, C, D, E, F, G, H, I, J, K, L, M, N)] = + def apply[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N])(implicit qt: Quoted): CoGrouped[KEY, (A, B, C, D, E, F, G, H, I, J, K, L, M, N)] = a.join(b) .join(c) .join(d) @@ -138,7 +140,7 @@ object MultiJoin extends java.io.Serializable { .join(n) .mapValues { tup => flattenNestedTuple(tup) } - def apply[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O]): CoGrouped[KEY, (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O)] = + def apply[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O])(implicit qt: Quoted): CoGrouped[KEY, (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O)] = a.join(b) .join(c) .join(d) @@ -155,7 +157,7 @@ object MultiJoin extends java.io.Serializable { .join(o) .mapValues { tup => flattenNestedTuple(tup) } - def apply[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P]): CoGrouped[KEY, (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P)] = + def apply[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P])(implicit qt: Quoted): CoGrouped[KEY, (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P)] = a.join(b) .join(c) .join(d) @@ -173,7 +175,7 @@ object MultiJoin extends java.io.Serializable { .join(p) .mapValues { tup => flattenNestedTuple(tup) } - def apply[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q]): CoGrouped[KEY, (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q)] = + def apply[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q])(implicit qt: Quoted): CoGrouped[KEY, (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q)] = a.join(b) .join(c) .join(d) @@ -192,7 +194,7 @@ object MultiJoin extends java.io.Serializable { .join(q) .mapValues { tup => flattenNestedTuple(tup) } - def apply[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q], r: CoGroupable[KEY, R]): CoGrouped[KEY, (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R)] = + def apply[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q], r: CoGroupable[KEY, R])(implicit qt: Quoted): CoGrouped[KEY, (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R)] = a.join(b) .join(c) .join(d) @@ -212,7 +214,7 @@ object MultiJoin extends java.io.Serializable { .join(r) .mapValues { tup => flattenNestedTuple(tup) } - def apply[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q], r: CoGroupable[KEY, R], s: CoGroupable[KEY, S]): CoGrouped[KEY, (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S)] = + def apply[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q], r: CoGroupable[KEY, R], s: CoGroupable[KEY, S])(implicit qt: Quoted): CoGrouped[KEY, (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S)] = a.join(b) .join(c) .join(d) @@ -233,7 +235,7 @@ object MultiJoin extends java.io.Serializable { .join(s) .mapValues { tup => flattenNestedTuple(tup) } - def apply[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q], r: CoGroupable[KEY, R], s: CoGroupable[KEY, S], t: CoGroupable[KEY, T]): CoGrouped[KEY, (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T)] = + def apply[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q], r: CoGroupable[KEY, R], s: CoGroupable[KEY, S], t: CoGroupable[KEY, T])(implicit qt: Quoted): CoGrouped[KEY, (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T)] = a.join(b) .join(c) .join(d) @@ -255,7 +257,7 @@ object MultiJoin extends java.io.Serializable { .join(t) .mapValues { tup => flattenNestedTuple(tup) } - def apply[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q], r: CoGroupable[KEY, R], s: CoGroupable[KEY, S], t: CoGroupable[KEY, T], u: CoGroupable[KEY, U]): CoGrouped[KEY, (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U)] = + def apply[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q], r: CoGroupable[KEY, R], s: CoGroupable[KEY, S], t: CoGroupable[KEY, T], u: CoGroupable[KEY, U])(implicit qt: Quoted): CoGrouped[KEY, (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U)] = a.join(b) .join(c) .join(d) @@ -278,7 +280,7 @@ object MultiJoin extends java.io.Serializable { .join(u) .mapValues { tup => flattenNestedTuple(tup) } - def apply[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q], r: CoGroupable[KEY, R], s: CoGroupable[KEY, S], t: CoGroupable[KEY, T], u: CoGroupable[KEY, U], v: CoGroupable[KEY, V]): CoGrouped[KEY, (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V)] = + def apply[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q], r: CoGroupable[KEY, R], s: CoGroupable[KEY, S], t: CoGroupable[KEY, T], u: CoGroupable[KEY, U], v: CoGroupable[KEY, V])(implicit qt: Quoted): CoGrouped[KEY, (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V)] = a.join(b) .join(c) .join(d) @@ -302,28 +304,28 @@ object MultiJoin extends java.io.Serializable { .join(v) .mapValues { tup => flattenNestedTuple(tup) } - def left[KEY, A, B](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B]): CoGrouped[KEY, (A, Option[B])] = + def left[KEY, A, B](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B])(implicit qt: Quoted): CoGrouped[KEY, (A, Option[B])] = a.leftJoin(b) - def left[KEY, A, B, C](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C]): CoGrouped[KEY, (A, Option[B], Option[C])] = + def left[KEY, A, B, C](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C])(implicit qt: Quoted): CoGrouped[KEY, (A, Option[B], Option[C])] = a.leftJoin(b) .leftJoin(c) .mapValues { tup => flattenNestedTuple(tup) } - def left[KEY, A, B, C, D](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D]): CoGrouped[KEY, (A, Option[B], Option[C], Option[D])] = + def left[KEY, A, B, C, D](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D])(implicit qt: Quoted): CoGrouped[KEY, (A, Option[B], Option[C], Option[D])] = a.leftJoin(b) .leftJoin(c) .leftJoin(d) .mapValues { tup => flattenNestedTuple(tup) } - def left[KEY, A, B, C, D, E](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E]): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E])] = + def left[KEY, A, B, C, D, E](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E])(implicit qt: Quoted): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E])] = a.leftJoin(b) .leftJoin(c) .leftJoin(d) .leftJoin(e) .mapValues { tup => flattenNestedTuple(tup) } - def left[KEY, A, B, C, D, E, F](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F]): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E], Option[F])] = + def left[KEY, A, B, C, D, E, F](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F])(implicit qt: Quoted): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E], Option[F])] = a.leftJoin(b) .leftJoin(c) .leftJoin(d) @@ -331,7 +333,7 @@ object MultiJoin extends java.io.Serializable { .leftJoin(f) .mapValues { tup => flattenNestedTuple(tup) } - def left[KEY, A, B, C, D, E, F, G](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G]): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E], Option[F], Option[G])] = + def left[KEY, A, B, C, D, E, F, G](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G])(implicit qt: Quoted): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E], Option[F], Option[G])] = a.leftJoin(b) .leftJoin(c) .leftJoin(d) @@ -340,7 +342,7 @@ object MultiJoin extends java.io.Serializable { .leftJoin(g) .mapValues { tup => flattenNestedTuple(tup) } - def left[KEY, A, B, C, D, E, F, G, H](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H]): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H])] = + def left[KEY, A, B, C, D, E, F, G, H](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H])(implicit qt: Quoted): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H])] = a.leftJoin(b) .leftJoin(c) .leftJoin(d) @@ -350,7 +352,7 @@ object MultiJoin extends java.io.Serializable { .leftJoin(h) .mapValues { tup => flattenNestedTuple(tup) } - def left[KEY, A, B, C, D, E, F, G, H, I](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I]): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I])] = + def left[KEY, A, B, C, D, E, F, G, H, I](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I])(implicit qt: Quoted): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I])] = a.leftJoin(b) .leftJoin(c) .leftJoin(d) @@ -361,7 +363,7 @@ object MultiJoin extends java.io.Serializable { .leftJoin(i) .mapValues { tup => flattenNestedTuple(tup) } - def left[KEY, A, B, C, D, E, F, G, H, I, J](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J]): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J])] = + def left[KEY, A, B, C, D, E, F, G, H, I, J](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J])(implicit qt: Quoted): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J])] = a.leftJoin(b) .leftJoin(c) .leftJoin(d) @@ -373,7 +375,7 @@ object MultiJoin extends java.io.Serializable { .leftJoin(j) .mapValues { tup => flattenNestedTuple(tup) } - def left[KEY, A, B, C, D, E, F, G, H, I, J, K](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K]): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K])] = + def left[KEY, A, B, C, D, E, F, G, H, I, J, K](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K])(implicit qt: Quoted): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K])] = a.leftJoin(b) .leftJoin(c) .leftJoin(d) @@ -386,7 +388,7 @@ object MultiJoin extends java.io.Serializable { .leftJoin(k) .mapValues { tup => flattenNestedTuple(tup) } - def left[KEY, A, B, C, D, E, F, G, H, I, J, K, L](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L]): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L])] = + def left[KEY, A, B, C, D, E, F, G, H, I, J, K, L](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L])(implicit qt: Quoted): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L])] = a.leftJoin(b) .leftJoin(c) .leftJoin(d) @@ -400,7 +402,7 @@ object MultiJoin extends java.io.Serializable { .leftJoin(l) .mapValues { tup => flattenNestedTuple(tup) } - def left[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M]): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M])] = + def left[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M])(implicit qt: Quoted): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M])] = a.leftJoin(b) .leftJoin(c) .leftJoin(d) @@ -415,7 +417,7 @@ object MultiJoin extends java.io.Serializable { .leftJoin(m) .mapValues { tup => flattenNestedTuple(tup) } - def left[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N]): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N])] = + def left[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N])(implicit qt: Quoted): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N])] = a.leftJoin(b) .leftJoin(c) .leftJoin(d) @@ -431,7 +433,7 @@ object MultiJoin extends java.io.Serializable { .leftJoin(n) .mapValues { tup => flattenNestedTuple(tup) } - def left[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O]): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N], Option[O])] = + def left[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O])(implicit qt: Quoted): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N], Option[O])] = a.leftJoin(b) .leftJoin(c) .leftJoin(d) @@ -448,7 +450,7 @@ object MultiJoin extends java.io.Serializable { .leftJoin(o) .mapValues { tup => flattenNestedTuple(tup) } - def left[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P]): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N], Option[O], Option[P])] = + def left[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P])(implicit qt: Quoted): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N], Option[O], Option[P])] = a.leftJoin(b) .leftJoin(c) .leftJoin(d) @@ -466,7 +468,7 @@ object MultiJoin extends java.io.Serializable { .leftJoin(p) .mapValues { tup => flattenNestedTuple(tup) } - def left[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q]): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N], Option[O], Option[P], Option[Q])] = + def left[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q])(implicit qt: Quoted): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N], Option[O], Option[P], Option[Q])] = a.leftJoin(b) .leftJoin(c) .leftJoin(d) @@ -485,7 +487,7 @@ object MultiJoin extends java.io.Serializable { .leftJoin(q) .mapValues { tup => flattenNestedTuple(tup) } - def left[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q], r: CoGroupable[KEY, R]): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N], Option[O], Option[P], Option[Q], Option[R])] = + def left[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q], r: CoGroupable[KEY, R])(implicit qt: Quoted): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N], Option[O], Option[P], Option[Q], Option[R])] = a.leftJoin(b) .leftJoin(c) .leftJoin(d) @@ -505,7 +507,7 @@ object MultiJoin extends java.io.Serializable { .leftJoin(r) .mapValues { tup => flattenNestedTuple(tup) } - def left[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q], r: CoGroupable[KEY, R], s: CoGroupable[KEY, S]): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N], Option[O], Option[P], Option[Q], Option[R], Option[S])] = + def left[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q], r: CoGroupable[KEY, R], s: CoGroupable[KEY, S])(implicit qt: Quoted): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N], Option[O], Option[P], Option[Q], Option[R], Option[S])] = a.leftJoin(b) .leftJoin(c) .leftJoin(d) @@ -526,7 +528,7 @@ object MultiJoin extends java.io.Serializable { .leftJoin(s) .mapValues { tup => flattenNestedTuple(tup) } - def left[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q], r: CoGroupable[KEY, R], s: CoGroupable[KEY, S], t: CoGroupable[KEY, T]): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N], Option[O], Option[P], Option[Q], Option[R], Option[S], Option[T])] = + def left[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q], r: CoGroupable[KEY, R], s: CoGroupable[KEY, S], t: CoGroupable[KEY, T])(implicit qt: Quoted): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N], Option[O], Option[P], Option[Q], Option[R], Option[S], Option[T])] = a.leftJoin(b) .leftJoin(c) .leftJoin(d) @@ -548,7 +550,7 @@ object MultiJoin extends java.io.Serializable { .leftJoin(t) .mapValues { tup => flattenNestedTuple(tup) } - def left[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q], r: CoGroupable[KEY, R], s: CoGroupable[KEY, S], t: CoGroupable[KEY, T], u: CoGroupable[KEY, U]): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N], Option[O], Option[P], Option[Q], Option[R], Option[S], Option[T], Option[U])] = + def left[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q], r: CoGroupable[KEY, R], s: CoGroupable[KEY, S], t: CoGroupable[KEY, T], u: CoGroupable[KEY, U])(implicit qt: Quoted): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N], Option[O], Option[P], Option[Q], Option[R], Option[S], Option[T], Option[U])] = a.leftJoin(b) .leftJoin(c) .leftJoin(d) @@ -571,7 +573,7 @@ object MultiJoin extends java.io.Serializable { .leftJoin(u) .mapValues { tup => flattenNestedTuple(tup) } - def left[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q], r: CoGroupable[KEY, R], s: CoGroupable[KEY, S], t: CoGroupable[KEY, T], u: CoGroupable[KEY, U], v: CoGroupable[KEY, V]): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N], Option[O], Option[P], Option[Q], Option[R], Option[S], Option[T], Option[U], Option[V])] = + def left[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q], r: CoGroupable[KEY, R], s: CoGroupable[KEY, S], t: CoGroupable[KEY, T], u: CoGroupable[KEY, U], v: CoGroupable[KEY, V])(implicit qt: Quoted): CoGrouped[KEY, (A, Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N], Option[O], Option[P], Option[Q], Option[R], Option[S], Option[T], Option[U], Option[V])] = a.leftJoin(b) .leftJoin(c) .leftJoin(d) @@ -595,28 +597,28 @@ object MultiJoin extends java.io.Serializable { .leftJoin(v) .mapValues { tup => flattenNestedTuple(tup) } - def outer[KEY, A, B](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B]): CoGrouped[KEY, (Option[A], Option[B])] = + def outer[KEY, A, B](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B])(implicit qt: Quoted): CoGrouped[KEY, (Option[A], Option[B])] = a.outerJoin(b) - def outer[KEY, A, B, C](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C]): CoGrouped[KEY, (Option[A], Option[B], Option[C])] = + def outer[KEY, A, B, C](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C])(implicit qt: Quoted): CoGrouped[KEY, (Option[A], Option[B], Option[C])] = a.outerJoin(b) .outerJoin(c) .mapValues { tup => flattenNestedOptionTuple(tup) } - def outer[KEY, A, B, C, D](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D]): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D])] = + def outer[KEY, A, B, C, D](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D])(implicit qt: Quoted): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D])] = a.outerJoin(b) .outerJoin(c) .outerJoin(d) .mapValues { tup => flattenNestedOptionTuple(tup) } - def outer[KEY, A, B, C, D, E](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E]): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E])] = + def outer[KEY, A, B, C, D, E](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E])(implicit qt: Quoted): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E])] = a.outerJoin(b) .outerJoin(c) .outerJoin(d) .outerJoin(e) .mapValues { tup => flattenNestedOptionTuple(tup) } - def outer[KEY, A, B, C, D, E, F](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F]): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F])] = + def outer[KEY, A, B, C, D, E, F](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F])(implicit qt: Quoted): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F])] = a.outerJoin(b) .outerJoin(c) .outerJoin(d) @@ -624,7 +626,7 @@ object MultiJoin extends java.io.Serializable { .outerJoin(f) .mapValues { tup => flattenNestedOptionTuple(tup) } - def outer[KEY, A, B, C, D, E, F, G](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G]): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F], Option[G])] = + def outer[KEY, A, B, C, D, E, F, G](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G])(implicit qt: Quoted): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F], Option[G])] = a.outerJoin(b) .outerJoin(c) .outerJoin(d) @@ -633,7 +635,7 @@ object MultiJoin extends java.io.Serializable { .outerJoin(g) .mapValues { tup => flattenNestedOptionTuple(tup) } - def outer[KEY, A, B, C, D, E, F, G, H](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H]): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H])] = + def outer[KEY, A, B, C, D, E, F, G, H](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H])(implicit qt: Quoted): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H])] = a.outerJoin(b) .outerJoin(c) .outerJoin(d) @@ -643,7 +645,7 @@ object MultiJoin extends java.io.Serializable { .outerJoin(h) .mapValues { tup => flattenNestedOptionTuple(tup) } - def outer[KEY, A, B, C, D, E, F, G, H, I](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I]): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I])] = + def outer[KEY, A, B, C, D, E, F, G, H, I](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I])(implicit qt: Quoted): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I])] = a.outerJoin(b) .outerJoin(c) .outerJoin(d) @@ -654,7 +656,7 @@ object MultiJoin extends java.io.Serializable { .outerJoin(i) .mapValues { tup => flattenNestedOptionTuple(tup) } - def outer[KEY, A, B, C, D, E, F, G, H, I, J](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J]): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J])] = + def outer[KEY, A, B, C, D, E, F, G, H, I, J](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J])(implicit qt: Quoted): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J])] = a.outerJoin(b) .outerJoin(c) .outerJoin(d) @@ -666,7 +668,7 @@ object MultiJoin extends java.io.Serializable { .outerJoin(j) .mapValues { tup => flattenNestedOptionTuple(tup) } - def outer[KEY, A, B, C, D, E, F, G, H, I, J, K](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K]): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K])] = + def outer[KEY, A, B, C, D, E, F, G, H, I, J, K](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K])(implicit qt: Quoted): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K])] = a.outerJoin(b) .outerJoin(c) .outerJoin(d) @@ -679,7 +681,7 @@ object MultiJoin extends java.io.Serializable { .outerJoin(k) .mapValues { tup => flattenNestedOptionTuple(tup) } - def outer[KEY, A, B, C, D, E, F, G, H, I, J, K, L](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L]): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L])] = + def outer[KEY, A, B, C, D, E, F, G, H, I, J, K, L](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L])(implicit qt: Quoted): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L])] = a.outerJoin(b) .outerJoin(c) .outerJoin(d) @@ -693,7 +695,7 @@ object MultiJoin extends java.io.Serializable { .outerJoin(l) .mapValues { tup => flattenNestedOptionTuple(tup) } - def outer[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M]): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M])] = + def outer[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M])(implicit qt: Quoted): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M])] = a.outerJoin(b) .outerJoin(c) .outerJoin(d) @@ -708,7 +710,7 @@ object MultiJoin extends java.io.Serializable { .outerJoin(m) .mapValues { tup => flattenNestedOptionTuple(tup) } - def outer[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N]): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N])] = + def outer[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N])(implicit qt: Quoted): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N])] = a.outerJoin(b) .outerJoin(c) .outerJoin(d) @@ -724,7 +726,7 @@ object MultiJoin extends java.io.Serializable { .outerJoin(n) .mapValues { tup => flattenNestedOptionTuple(tup) } - def outer[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O]): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N], Option[O])] = + def outer[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O])(implicit qt: Quoted): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N], Option[O])] = a.outerJoin(b) .outerJoin(c) .outerJoin(d) @@ -741,7 +743,7 @@ object MultiJoin extends java.io.Serializable { .outerJoin(o) .mapValues { tup => flattenNestedOptionTuple(tup) } - def outer[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P]): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N], Option[O], Option[P])] = + def outer[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P])(implicit qt: Quoted): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N], Option[O], Option[P])] = a.outerJoin(b) .outerJoin(c) .outerJoin(d) @@ -759,7 +761,7 @@ object MultiJoin extends java.io.Serializable { .outerJoin(p) .mapValues { tup => flattenNestedOptionTuple(tup) } - def outer[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q]): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N], Option[O], Option[P], Option[Q])] = + def outer[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q])(implicit qt: Quoted): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N], Option[O], Option[P], Option[Q])] = a.outerJoin(b) .outerJoin(c) .outerJoin(d) @@ -778,7 +780,7 @@ object MultiJoin extends java.io.Serializable { .outerJoin(q) .mapValues { tup => flattenNestedOptionTuple(tup) } - def outer[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q], r: CoGroupable[KEY, R]): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N], Option[O], Option[P], Option[Q], Option[R])] = + def outer[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q], r: CoGroupable[KEY, R])(implicit qt: Quoted): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N], Option[O], Option[P], Option[Q], Option[R])] = a.outerJoin(b) .outerJoin(c) .outerJoin(d) @@ -798,7 +800,7 @@ object MultiJoin extends java.io.Serializable { .outerJoin(r) .mapValues { tup => flattenNestedOptionTuple(tup) } - def outer[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q], r: CoGroupable[KEY, R], s: CoGroupable[KEY, S]): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N], Option[O], Option[P], Option[Q], Option[R], Option[S])] = + def outer[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q], r: CoGroupable[KEY, R], s: CoGroupable[KEY, S])(implicit qt: Quoted): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N], Option[O], Option[P], Option[Q], Option[R], Option[S])] = a.outerJoin(b) .outerJoin(c) .outerJoin(d) @@ -819,7 +821,7 @@ object MultiJoin extends java.io.Serializable { .outerJoin(s) .mapValues { tup => flattenNestedOptionTuple(tup) } - def outer[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q], r: CoGroupable[KEY, R], s: CoGroupable[KEY, S], t: CoGroupable[KEY, T]): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N], Option[O], Option[P], Option[Q], Option[R], Option[S], Option[T])] = + def outer[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q], r: CoGroupable[KEY, R], s: CoGroupable[KEY, S], t: CoGroupable[KEY, T])(implicit qt: Quoted): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N], Option[O], Option[P], Option[Q], Option[R], Option[S], Option[T])] = a.outerJoin(b) .outerJoin(c) .outerJoin(d) @@ -841,7 +843,7 @@ object MultiJoin extends java.io.Serializable { .outerJoin(t) .mapValues { tup => flattenNestedOptionTuple(tup) } - def outer[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q], r: CoGroupable[KEY, R], s: CoGroupable[KEY, S], t: CoGroupable[KEY, T], u: CoGroupable[KEY, U]): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N], Option[O], Option[P], Option[Q], Option[R], Option[S], Option[T], Option[U])] = + def outer[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q], r: CoGroupable[KEY, R], s: CoGroupable[KEY, S], t: CoGroupable[KEY, T], u: CoGroupable[KEY, U])(implicit qt: Quoted): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N], Option[O], Option[P], Option[Q], Option[R], Option[S], Option[T], Option[U])] = a.outerJoin(b) .outerJoin(c) .outerJoin(d) @@ -864,7 +866,7 @@ object MultiJoin extends java.io.Serializable { .outerJoin(u) .mapValues { tup => flattenNestedOptionTuple(tup) } - def outer[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q], r: CoGroupable[KEY, R], s: CoGroupable[KEY, S], t: CoGroupable[KEY, T], u: CoGroupable[KEY, U], v: CoGroupable[KEY, V]): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N], Option[O], Option[P], Option[Q], Option[R], Option[S], Option[T], Option[U], Option[V])] = + def outer[KEY, A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V](a: CoGroupable[KEY, A], b: CoGroupable[KEY, B], c: CoGroupable[KEY, C], d: CoGroupable[KEY, D], e: CoGroupable[KEY, E], f: CoGroupable[KEY, F], g: CoGroupable[KEY, G], h: CoGroupable[KEY, H], i: CoGroupable[KEY, I], j: CoGroupable[KEY, J], k: CoGroupable[KEY, K], l: CoGroupable[KEY, L], m: CoGroupable[KEY, M], n: CoGroupable[KEY, N], o: CoGroupable[KEY, O], p: CoGroupable[KEY, P], q: CoGroupable[KEY, Q], r: CoGroupable[KEY, R], s: CoGroupable[KEY, S], t: CoGroupable[KEY, T], u: CoGroupable[KEY, U], v: CoGroupable[KEY, V])(implicit qt: Quoted): CoGrouped[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F], Option[G], Option[H], Option[I], Option[J], Option[K], Option[L], Option[M], Option[N], Option[O], Option[P], Option[Q], Option[R], Option[S], Option[T], Option[U], Option[V])] = a.outerJoin(b) .outerJoin(c) .outerJoin(d) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/Sketched.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/Sketched.scala index 0c128ad300..83c4d443bc 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/Sketched.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/Sketched.scala @@ -24,6 +24,7 @@ import scala.language.experimental.macros // This was a bad design choice, we should have just put these in the CMSHasher object import CMSHasherImplicits._ +import com.twitter.scalding.quotation.Quoted /** * This class is generally only created by users @@ -34,7 +35,7 @@ case class Sketched[K, V](pipe: TypedPipe[(K, V)], delta: Double, eps: Double, seed: Int)(implicit val serialization: K => Array[Byte], - ordering: Ordering[K]) + ordering: Ordering[K], q: Quoted) extends MustHaveReducers { def reducers = Some(numReducers) @@ -47,7 +48,6 @@ case class Sketched[K, V](pipe: TypedPipe[(K, V)], // every 10k items, compact into a CMS to prevent very slow mappers lazy implicit val batchedSG: com.twitter.algebird.Semigroup[Batched[CMS[Bytes]]] = Batched.compactingSemigroup[CMS[Bytes]](10000) - pipe .map { case (k, _) => ((), Batched(cms.create(Bytes(localSer(k))))) } .sumByLocalKeys @@ -83,7 +83,7 @@ case class Sketched[K, V](pipe: TypedPipe[(K, V)], case class SketchJoined[K: Ordering, V, V2, R](left: Sketched[K, V], right: TypedPipe[(K, V2)], - numReducers: Int)(joiner: (K, V, Iterable[V2]) => Iterator[R]) + numReducers: Int)(joiner: (K, V, Iterable[V2]) => Iterator[R])(implicit q: Quoted) extends MustHaveReducers { def reducers = Some(numReducers) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/TDsl.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/TDsl.scala index 445f3690cb..e3faf69fae 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/TDsl.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/TDsl.scala @@ -22,6 +22,7 @@ import cascading.pipe.Pipe import cascading.tuple.Fields import com.twitter.scalding._ +import com.twitter.scalding.quotation.Quoted /** * implicits for the type-safe DSL @@ -60,8 +61,8 @@ class PipeTExtensions(pipe: Pipe, flowDef: FlowDef, mode: Mode) extends Serializ * } * The above sums all the tuples and returns a TypedPipe[Int] which has the total sum. */ - def typed[T, U](fielddef: (Fields, Fields))(fn: TypedPipe[T] => TypedPipe[U])(implicit conv: TupleConverter[T], setter: TupleSetter[U]): Pipe = - fn(TypedPipe.from(pipe, fielddef._1)(flowDef, mode, conv)).toPipe(fielddef._2)(flowDef, mode, setter) + def typed[T, U](fielddef: (Fields, Fields))(fn: TypedPipe[T] => TypedPipe[U])(implicit conv: TupleConverter[T], setter: TupleSetter[U], m: Quoted): Pipe = + fn(TypedPipe.from(pipe, fielddef._1)(flowDef, mode, conv)).toPipe(fielddef._2)(flowDef, mode, setter, m) def toTypedPipe[T](fields: Fields)(implicit conv: TupleConverter[T]): TypedPipe[T] = TypedPipe.from[T](pipe, fields)(flowDef, mode, conv) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipe.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipe.scala index 60d16e8f08..207c3961d7 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipe.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipe.scala @@ -31,6 +31,9 @@ import com.stripe.dagon.{Memoize, RefPair} import scala.util.Try import scala.util.hashing.MurmurHash3 +import com.twitter.scalding.quotation.Quoted +import com.twitter.scalding.quotation.Projection +import com.twitter.scalding.quotation.Projections /** * factory methods for TypedPipe, which is the typed representation of distributed lists in scalding. @@ -110,7 +113,7 @@ object TypedPipe extends Serializable { * * This method is the Vitaly-was-right method. */ - implicit def toHashJoinable[K, V](pipe: TypedPipe[(K, V)])(implicit ord: Ordering[K]): HashJoinable[K, V] = + implicit def toHashJoinable[K, V](pipe: TypedPipe[(K, V)])(implicit ord: Ordering[K], q: Quoted): HashJoinable[K, V] = /* * Note, it would not be safe to make the return type of this Grouped[K, V] since that has some * different semantics than TypedPipe, however, it is not unclear when we only go to @@ -121,7 +124,7 @@ object TypedPipe extends Serializable { /** * TypedPipe instances are monoids. They are isomorphic to multisets. */ - implicit def typedPipeMonoid[T]: Monoid[TypedPipe[T]] = new Monoid[TypedPipe[T]] { + implicit def typedPipeMonoid[T](implicit m: Quoted): Monoid[TypedPipe[T]] = new Monoid[TypedPipe[T]] { def zero = TypedPipe.empty def plus(left: TypedPipe[T], right: TypedPipe[T]): TypedPipe[T] = left ++ right @@ -143,11 +146,11 @@ object TypedPipe extends Serializable { final case class CoGroupedPipe[K, V](@transient cogrouped: CoGrouped[K, V]) extends TypedPipe[(K, V)] final case class CounterPipe[A](pipe: TypedPipe[(A, Iterable[((String, String), Long)])]) extends TypedPipe[A] final case class CrossPipe[T, U](left: TypedPipe[T], right: TypedPipe[U]) extends TypedPipe[(T, U)] { - def viaHashJoin: TypedPipe[(T, U)] = + def viaHashJoin(implicit q: Quoted): TypedPipe[(T, U)] = left.withKey(()).hashJoin(right.withKey(())).values } final case class CrossValue[T, U](left: TypedPipe[T], right: ValuePipe[U]) extends TypedPipe[(T, U)] { - def viaHashJoin: TypedPipe[(T, U)] = + def viaHashJoin(implicit m: Quoted): TypedPipe[(T, U)] = right match { case EmptyValue => EmptyTypedPipe @@ -193,8 +196,8 @@ object TypedPipe extends Serializable { /** * If any errors happen below this line, but before a groupBy, write to a TypedSink */ - def addTrap(trapSink: Source with TypedSink[T])(implicit conv: TupleConverter[T]): TypedPipe[T] = - TypedPipe.TrappedPipe[T](pipe, trapSink, conv).withLine + def addTrap(trapSink: Source with TypedSink[T])(implicit conv: TupleConverter[T], m: Quoted): TypedPipe[T] = + TypedPipe.TrappedPipe[T](pipe, trapSink, conv).withQuoted } /** @@ -494,13 +497,8 @@ sealed abstract class TypedPipe[+T] extends Serializable with Product { case _ => false } - protected def withLine: TypedPipe[T] = - LineNumber.tryNonScaldingCaller.map(_.toString) match { - case None => - this - case Some(desc) => - TypedPipe.WithDescriptionTypedPipe(this, (desc, true) :: Nil) // deduplicate line numbers - } + protected def withQuoted(implicit m: Quoted): TypedPipe[T] = + TypedPipe.WithDescriptionTypedPipe(this, (m.toString, true) :: Nil, Some(m)) /** * Increment diagnostic counters by 1 for each item in the pipe. @@ -532,8 +530,8 @@ sealed abstract class TypedPipe[+T] extends Serializable with Product { * This gives the same results as * {code for { l <- list1; l2 <- list2 } yield (l, l2) } */ - def cross[U](tiny: TypedPipe[U]): TypedPipe[(T, U)] = - TypedPipe.CrossPipe(this, tiny).withLine + def cross[U](tiny: TypedPipe[U])(implicit m: Quoted): TypedPipe[(T, U)] = + TypedPipe.CrossPipe(this, tiny).withQuoted /** * This is the fundamental mapper operation. @@ -548,26 +546,26 @@ sealed abstract class TypedPipe[+T] extends Serializable with Product { * (by returning 1 item per input), it can be used to explode 1 input into many outputs, * or even a combination of all of the above at once. */ - def flatMap[U](f: T => TraversableOnce[U]): TypedPipe[U] = - TypedPipe.FlatMapped(this, f).withLine + def flatMap[U](f: T => TraversableOnce[U])(implicit m: Quoted): TypedPipe[U] = + TypedPipe.FlatMapped(this, f).withQuoted /** * Export back to a raw cascading Pipe. useful for interop with the scalding * Fields API or with Cascading code. * Avoid this if possible. Prefer to write to TypedSink. */ - final def toPipe[U >: T](fieldNames: Fields)(implicit flowDef: FlowDef, mode: Mode, setter: TupleSetter[U]): Pipe = + final def toPipe[U >: T](fieldNames: Fields)(implicit flowDef: FlowDef, mode: Mode, setter: TupleSetter[U], m: Quoted): Pipe = // we have to be cafeful to pass the setter we want since a low priority implicit can always be // found :( - cascading_backend.CascadingBackend.toPipe[U](withLine, fieldNames)(flowDef, mode, setter) + cascading_backend.CascadingBackend.toPipe[U](withQuoted, fieldNames)(flowDef, mode, setter) /** * Merge two TypedPipes (no order is guaranteed) * This is only realized when a group (or join) is * performed. */ - def ++[U >: T](other: TypedPipe[U]): TypedPipe[U] = - TypedPipe.MergedTypedPipe(this, other).withLine + def ++[U >: T](other: TypedPipe[U])(implicit m: Quoted): TypedPipe[U] = + TypedPipe.MergedTypedPipe(this, other).withQuoted /** * Aggregate all items in this pipe into a single ValuePipe @@ -577,7 +575,7 @@ sealed abstract class TypedPipe[+T] extends Serializable with Product { * * Same as groupAll.aggregate.values */ - def aggregate[B, C](agg: Aggregator[T, B, C]): ValuePipe[C] = + def aggregate[B, C](agg: Aggregator[T, B, C])(implicit m: Quoted): ValuePipe[C] = ComputedValue(groupAll.aggregate(agg).values) /** @@ -585,7 +583,7 @@ sealed abstract class TypedPipe[+T] extends Serializable with Product { * in some sense, this is the dual of groupAll */ @annotation.implicitNotFound(msg = "For asKeys method to work, the type in TypedPipe must have an Ordering.") - def asKeys[U >: T](implicit ord: Ordering[U]): Grouped[U, Unit] = + def asKeys[U >: T](implicit ord: Ordering[U], q: Quoted): Grouped[U, Unit] = widen[U] .withValue(()) .group @@ -593,13 +591,13 @@ sealed abstract class TypedPipe[+T] extends Serializable with Product { /** * Set a key to to the given value. */ - def withKey[K](key: K): TypedPipe[(K, T)] = + def withKey[K](key: K)(implicit q: Quoted): TypedPipe[(K, T)] = map(ConstantKey(key)) /** * Set a key to to the given value. */ - def withValue[V](value: V): TypedPipe[(T, V)] = + def withValue[V](value: V)(implicit q: Quoted): TypedPipe[(T, V)] = map(WithConstant(value)) /** @@ -614,22 +612,25 @@ sealed abstract class TypedPipe[+T] extends Serializable with Product { * collect { case Some(x) => fn(x) } * } */ - def collect[U](fn: PartialFunction[T, U]): TypedPipe[U] = + def collect[U](fn: PartialFunction[T, U])(implicit m: Quoted): TypedPipe[U] = filter(PartialFunctionToFilter(fn)).map(fn) /** * Attach a ValuePipe to each element this TypedPipe */ - def cross[V](p: ValuePipe[V]): TypedPipe[(T, V)] = - TypedPipe.CrossValue(this, p).withLine + def cross[V](p: ValuePipe[V])(implicit m: Quoted): TypedPipe[(T, V)] = + TypedPipe.CrossValue(this, p).withQuoted /** prints the current pipe to stdout */ - def debug: TypedPipe[T] = - TypedPipe.DebugPipe(this).withLine + def debug(implicit m: Quoted): TypedPipe[T] = + TypedPipe.DebugPipe(this).withQuoted - /** adds a description to the pipe */ + /** + * Adds a description to the pipe. Note: it doesn't take a `Quoted`, + * giving the user complete control over the description. + */ def withDescription(description: String): TypedPipe[T] = - TypedPipe.WithDescriptionTypedPipe[T](this, (description, false) :: Nil) + TypedPipe.WithDescriptionTypedPipe[T](this, (description, false) :: Nil, None) /** * Returns the set of distinct elements in the TypedPipe @@ -646,14 +647,14 @@ sealed abstract class TypedPipe[+T] extends Serializable with Product { * The latter creates 1 map/reduce phase rather than 2 */ @annotation.implicitNotFound(msg = "For distinct method to work, the type in TypedPipe must have an Ordering.") - def distinct(implicit ord: Ordering[_ >: T]): TypedPipe[T] = + def distinct(implicit ord: Ordering[_ >: T], q: Quoted): TypedPipe[T] = asKeys[T](TypedPipe.narrowOrdering(ord)).sum.keys /** * Returns the set of distinct elements identified by a given lambda extractor in the TypedPipe */ @annotation.implicitNotFound(msg = "For distinctBy method to work, the type to distinct on in the TypedPipe must have an Ordering.") - def distinctBy[U](fn: T => U, numReducers: Option[Int] = None)(implicit ord: Ordering[_ >: U]): TypedPipe[T] = { + def distinctBy[U](fn: T => U, numReducers: Option[Int] = None)(implicit ord: Ordering[_ >: U], q: Quoted): TypedPipe[T] = { implicit val ordT: Ordering[U] = TypedPipe.narrowOrdering(ord) val op = groupBy(fn).head @@ -665,7 +666,7 @@ sealed abstract class TypedPipe[+T] extends Serializable with Product { } /** Merge two TypedPipes of different types by using Either */ - def either[R](that: TypedPipe[R]): TypedPipe[Either[T, R]] = + def either[R](that: TypedPipe[R])(implicit m: Quoted): TypedPipe[Either[T, R]] = map(AsLeft()) ++ (that.map(AsRight())) /** @@ -678,54 +679,58 @@ sealed abstract class TypedPipe[+T] extends Serializable with Product { * * Ideally the planner would see this */ - def fork: TypedPipe[T] = TypedPipe.Fork(this).withLine + def fork(implicit m: Quoted): TypedPipe[T] = TypedPipe.Fork(this).withQuoted /** * limit the output to at most count items, if at least count items exist. */ - def limit(count: Int): TypedPipe[T] = + def limit(count: Int)(implicit m: Quoted): TypedPipe[T] = groupAll.bufferedTake(count).values /** Transform each element via the function f */ - def map[U](f: T => U): TypedPipe[U] = - TypedPipe.Mapped(this, f).withLine + def map[U](f: T => U)(implicit m: Quoted): TypedPipe[U] = + TypedPipe.Mapped(this, f).withQuoted /** * Keep only items that satisfy this predicate */ - def filter(f: T => Boolean): TypedPipe[T] = - TypedPipe.Filter(this, f).withLine + def filter(f: T => Boolean)(implicit m: Quoted): TypedPipe[T] = + TypedPipe.Filter(this, f).withQuoted // This is just to appease for comprehension - def withFilter(f: T => Boolean): TypedPipe[T] = filter(f) + def withFilter(f: T => Boolean)(implicit m: Quoted): TypedPipe[T] = filter(f) /** * Keep only items that don't satisfy the predicate. * `filterNot` is the same as `filter` with a negated predicate. */ - def filterNot(f: T => Boolean): TypedPipe[T] = + def filterNot(f: T => Boolean)(implicit m: Quoted): TypedPipe[T] = filter(!f(_)) /** flatten an Iterable */ - def flatten[U](implicit ev: T <:< TraversableOnce[U]): TypedPipe[U] = + def flatten[U](implicit ev: T <:< TraversableOnce[U], q: Quoted): TypedPipe[U] = widen[TraversableOnce[U]].flatMap(Identity[TraversableOnce[U]]()) /** - * Force a materialization of this pipe prior to the next operation. - * This is useful if you filter almost everything before a hashJoin, for instance. - * This is useful for experts who see some heuristic of the planner causing - * slower performance. + * flatten just the values + * This is more useful on KeyedListLike, but added here to reduce assymmetry in the APIs */ - def forceToDisk: TypedPipe[T] = - TypedPipe.ForceToDisk(this).withLine - + def flattenValues[K, U](implicit ev: T <:< (K, TraversableOnce[U]), m: Quoted): TypedPipe[(K, U)] = + flatMapValues[K, TraversableOnce[U], U](Identity[TraversableOnce[U]]()) - /** Send all items to a single reducer */ - def groupAll: Grouped[Unit, T] = - groupBy(Constant(()))(UnitOrderedSerialization).withReducers(1) + /** + * This is the default means of grouping all pairs with the same key. Generally this triggers 1 Map/Reduce transition + */ + def group[K, V](implicit ev: <:<[T, (K, V)], ord: Ordering[K], m: Quoted): Grouped[K, V] = + //If the type of T is not (K,V), then at compile time, this will fail. It uses implicits to do + //a compile time check that one type is equivalent to another. If T is not (K,V), we can't + //automatically group. We cast because it is safe to do so, and we need to convert to K,V, but + //the ev is not needed for the cast. In fact, you can do the cast with ev(t) and it will return + //it as (K,V), but the problem is, ev is not serializable. So we do the cast, which due to ev + //being present, will always pass. + Grouped(raiseTo[(K, V)].withQuoted) - /** Given a key function, add the key, then call .group */ - def groupBy[K](g: T => K)(implicit ord: Ordering[K]): Grouped[K, T] = + def groupBy[K](g: T => K)(implicit ord: Ordering[K], q: Quoted): Grouped[K, T] = map(MakeKey(g)).group /** @@ -737,8 +742,8 @@ sealed abstract class TypedPipe[+T] extends Serializable with Product { * * You probably want shard if you are just forcing a shuffle. */ - def groupRandomly(partitions: Int): Grouped[Int, T] = - groupBy(RandomNextInt(123, partitions))(TypedPipe.identityOrdering) + def groupRandomly(partitions: Int)(implicit m: Quoted): Grouped[Int, T] = + groupBy(RandomNextInt(123, partitions))(TypedPipe.identityOrdering, m) .withReducers(partitions) /** @@ -746,7 +751,7 @@ sealed abstract class TypedPipe[+T] extends Serializable with Product { * * Sometimes what you really want is a groupBy in these cases. */ - def partition(p: T => Boolean): (TypedPipe[T], TypedPipe[T]) = { + def partition(p: T => Boolean)(implicit m: Quoted): (TypedPipe[T], TypedPipe[T]) = { val forked = fork (forked.filter(p), forked.filterNot(p)) } @@ -757,14 +762,14 @@ sealed abstract class TypedPipe[+T] extends Serializable with Product { * does not require a reduce step. * This method makes sure to fix the seed, otherwise restarts cause subtle errors. */ - def sample(fraction: Double): TypedPipe[T] = sample(fraction, defaultSeed) + def sample(fraction: Double)(implicit m: Quoted): TypedPipe[T] = sample(fraction, defaultSeed) /** * Sample a fraction (between 0 and 1) uniformly independently at random each element of the pipe with * a given seed. * Does not require a reduce step. */ - def sample(fraction: Double, seed: Long): TypedPipe[T] = { + def sample(fraction: Double, seed: Long)(implicit m: Quoted): TypedPipe[T] = { require(0.0 <= fraction && fraction <= 1.0, s"got $fraction which is an invalid fraction") filter(RandomFilter(seed, fraction)) } @@ -774,13 +779,13 @@ sealed abstract class TypedPipe[+T] extends Serializable with Product { * Only use this if your mappers are taking far longer than * the time to shuffle. */ - def shard(partitions: Int): TypedPipe[T] = groupRandomly(partitions).forceToReducers.values + def shard(partitions: Int)(implicit m: Quoted): TypedPipe[T] = groupRandomly(partitions).forceToReducers.values /** * Reasonably common shortcut for cases of total associative/commutative reduction * returns a ValuePipe with only one element if there is any input, otherwise EmptyValue. */ - def sum[U >: T](implicit plus: Semigroup[U]): ValuePipe[U] = { + def sum[U >: T](implicit plus: Semigroup[U], m: Quoted): ValuePipe[U] = { // every 1000 items, compact. lazy implicit val batchedSG: Semigroup[Batched[U]] = Batched.compactingSemigroup[U](1000) // TODO: literals like this defeat caching in the planner @@ -819,9 +824,9 @@ sealed abstract class TypedPipe[+T] extends Serializable with Product { Execution.toIterable(this) /** use a TupleUnpacker to flatten U out into a cascading Tuple */ - def unpackToPipe[U >: T](fieldNames: Fields)(implicit fd: FlowDef, mode: Mode, up: TupleUnpacker[U]): Pipe = { + def unpackToPipe[U >: T](fieldNames: Fields)(implicit fd: FlowDef, mode: Mode, up: TupleUnpacker[U], m: Quoted): Pipe = { val setter = up.newSetter(fieldNames) - toPipe[U](fieldNames)(fd, mode, setter) + toPipe[U](fieldNames)(fd, mode, setter, m) } /** @@ -831,15 +836,15 @@ sealed abstract class TypedPipe[+T] extends Serializable with Product { * to avoid it. Execution also has onComplete that can run when an Execution * has completed. */ - def onComplete(fn: () => Unit): TypedPipe[T] = - TypedPipe.WithOnComplete[T](this, fn).withLine + def onComplete(fn: () => Unit)(implicit m: Quoted): TypedPipe[T] = + TypedPipe.WithOnComplete[T](this, fn).withQuoted /** * Safely write to a TypedSink[T]. If you want to write to a Source (not a Sink) * you need to do something like: toPipe(fieldNames).write(dest) * @return a pipe equivalent to the current pipe. */ - def write(dest: TypedSink[T])(implicit flowDef: FlowDef, mode: Mode): TypedPipe[T] = { + def write(dest: TypedSink[T])(implicit flowDef: FlowDef, mode: Mode, q: Quoted): TypedPipe[T] = { // We do want to record the line number that this occured at val next = withLine FlowStateMap.merge(flowDef, FlowState.withTypedWrite(next, dest, mode)) @@ -875,12 +880,11 @@ sealed abstract class TypedPipe[+T] extends Serializable with Product { } } - /** * ValuePipe may be empty, so, this attaches it as an Option * cross is the same as leftCross(p).collect { case (t, Some(v)) => (t, v) } */ - def leftCross[V](p: ValuePipe[V]): TypedPipe[(T, Option[V])] = + def leftCross[V](p: ValuePipe[V])(implicit m: Quoted): TypedPipe[(T, Option[V])] = p match { case EmptyValue => map(WithConstant(None)) case LiteralValue(v) => map(WithConstant(Some(v))) @@ -888,7 +892,7 @@ sealed abstract class TypedPipe[+T] extends Serializable with Product { } /** uses hashJoin but attaches None if thatPipe is empty */ - def leftCross[V](thatPipe: TypedPipe[V]): TypedPipe[(T, Option[V])] = + def leftCross[V](thatPipe: TypedPipe[V])(implicit q: Quoted): TypedPipe[(T, Option[V])] = withKey(()).hashLeftJoin(thatPipe.withKey(())).values /** @@ -902,7 +906,7 @@ sealed abstract class TypedPipe[+T] extends Serializable with Product { * } * } */ - def mapWithValue[U, V](value: ValuePipe[U])(f: (T, Option[U]) => V): TypedPipe[V] = + def mapWithValue[U, V](value: ValuePipe[U])(f: (T, Option[U]) => V)(implicit m: Quoted): TypedPipe[V] = leftCross(value).map(TuplizeFunction(f)) /** @@ -916,7 +920,7 @@ sealed abstract class TypedPipe[+T] extends Serializable with Product { * } * } */ - def flatMapWithValue[U, V](value: ValuePipe[U])(f: (T, Option[U]) => TraversableOnce[V]): TypedPipe[V] = + def flatMapWithValue[U, V](value: ValuePipe[U])(f: (T, Option[U]) => TraversableOnce[V])(implicit m: Quoted): TypedPipe[V] = leftCross(value).flatMap(TuplizeFunction(f)) /** @@ -930,14 +934,14 @@ sealed abstract class TypedPipe[+T] extends Serializable with Product { * } * } */ - def filterWithValue[U](value: ValuePipe[U])(f: (T, Option[U]) => Boolean): TypedPipe[T] = + def filterWithValue[U](value: ValuePipe[U])(f: (T, Option[U]) => Boolean)(implicit m: Quoted): TypedPipe[T] = leftCross(value).filter(TuplizeFunction(f)).map(GetKey()) /** * For each element, do a map-side (hash) left join to look up a value */ - def hashLookup[K >: T, V](grouped: HashJoinable[K, V]): TypedPipe[(K, Option[V])] = + def hashLookup[K >: T, V](grouped: HashJoinable[K, V])(implicit m: Quoted): TypedPipe[(K, Option[V])] = map(WithConstant(())) .widen[(K, Unit)] .hashLeftJoin(grouped) @@ -951,10 +955,10 @@ sealed abstract class TypedPipe[+T] extends Serializable with Product { * import Syntax.joinOnMappablePipe */ class MappablePipeJoinEnrichment[T](pipe: TypedPipe[T]) { - def joinBy[K, U](smaller: TypedPipe[U])(g: (T => K), h: (U => K), reducers: Int = -1)(implicit ord: Ordering[K]): CoGrouped[K, (T, U)] = pipe.groupBy(g).withReducers(reducers).join(smaller.groupBy(h)) - def leftJoinBy[K, U](smaller: TypedPipe[U])(g: (T => K), h: (U => K), reducers: Int = -1)(implicit ord: Ordering[K]): CoGrouped[K, (T, Option[U])] = pipe.groupBy(g).withReducers(reducers).leftJoin(smaller.groupBy(h)) - def rightJoinBy[K, U](smaller: TypedPipe[U])(g: (T => K), h: (U => K), reducers: Int = -1)(implicit ord: Ordering[K]): CoGrouped[K, (Option[T], U)] = pipe.groupBy(g).withReducers(reducers).rightJoin(smaller.groupBy(h)) - def outerJoinBy[K, U](smaller: TypedPipe[U])(g: (T => K), h: (U => K), reducers: Int = -1)(implicit ord: Ordering[K]): CoGrouped[K, (Option[T], Option[U])] = pipe.groupBy(g).withReducers(reducers).outerJoin(smaller.groupBy(h)) + def joinBy[K, U](smaller: TypedPipe[U])(g: (T => K), h: (U => K), reducers: Int = -1)(implicit ord: Ordering[K], m: Quoted): CoGrouped[K, (T, U)] = pipe.groupBy(g).withReducers(reducers).join(smaller.groupBy(h)) + def leftJoinBy[K, U](smaller: TypedPipe[U])(g: (T => K), h: (U => K), reducers: Int = -1)(implicit ord: Ordering[K], m: Quoted): CoGrouped[K, (T, Option[U])] = pipe.groupBy(g).withReducers(reducers).leftJoin(smaller.groupBy(h)) + def rightJoinBy[K, U](smaller: TypedPipe[U])(g: (T => K), h: (U => K), reducers: Int = -1)(implicit ord: Ordering[K], m: Quoted): CoGrouped[K, (Option[T], U)] = pipe.groupBy(g).withReducers(reducers).rightJoin(smaller.groupBy(h)) + def outerJoinBy[K, U](smaller: TypedPipe[U])(g: (T => K), h: (U => K), reducers: Int = -1)(implicit ord: Ordering[K], m: Quoted): CoGrouped[K, (Option[T], Option[U])] = pipe.groupBy(g).withReducers(reducers).outerJoin(smaller.groupBy(h)) } /** diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipeDiff.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipeDiff.scala index 39e9f7f2d1..cac0c8f08a 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipeDiff.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipeDiff.scala @@ -5,6 +5,7 @@ import java.io.{ BufferedWriter, File, FileWriter } import com.twitter.scalding.Execution import scala.reflect.ClassTag +import com.twitter.scalding.quotation.Quoted /** * Some methods for comparing two typed pipes and finding out the difference between them. @@ -27,7 +28,7 @@ object TypedPipeDiff { * Requires that T have an ordering and a hashCode and equals that is stable across JVMs (not reference based). * See diffArrayPipes for diffing pipes of arrays, since arrays do not meet these requirements by default. */ - def diff[T: Ordering](left: TypedPipe[T], right: TypedPipe[T], reducers: Option[Int] = None): UnsortedGrouped[T, (Long, Long)] = { + def diff[T: Ordering](left: TypedPipe[T], right: TypedPipe[T], reducers: Option[Int] = None)(implicit q: Quoted): UnsortedGrouped[T, (Long, Long)] = { val lefts = left.map { x => (x, (1L, 0L)) } val rights = right.map { x => (x, (0L, 1L)) } val counts = (lefts ++ rights).sumByKey @@ -42,7 +43,7 @@ object TypedPipeDiff { */ def diffArrayPipes[T: ClassTag](left: TypedPipe[Array[T]], right: TypedPipe[Array[T]], - reducers: Option[Int] = None): TypedPipe[(Array[T], (Long, Long))] = { + reducers: Option[Int] = None)(implicit q: Quoted): TypedPipe[(Array[T], (Long, Long))] = { // cache this instead of reflecting on every single array val wrapFn = HashEqualsArrayWrapper.wrapByClassTagFn[T] @@ -77,7 +78,7 @@ object TypedPipeDiff { def diffByGroup[T, K: Ordering]( left: TypedPipe[T], right: TypedPipe[T], - reducers: Option[Int] = None)(groupByFn: T => K): TypedPipe[(T, (Long, Long))] = { + reducers: Option[Int] = None)(groupByFn: T => K)(implicit q: Quoted): TypedPipe[(T, (Long, Long))] = { val lefts = left.map { t => (groupByFn(t), Map(t -> (1L, 0L))) } val rights = right.map { t => (groupByFn(t), Map(t -> (0L, 1L))) } @@ -100,24 +101,25 @@ object TypedPipeDiff { def diffByHashCode[T]( left: TypedPipe[T], right: TypedPipe[T], - reducers: Option[Int] = None): TypedPipe[(T, (Long, Long))] = diffByGroup(left, right, reducers)(_.hashCode) + reducers: Option[Int] = None)(implicit q: Quoted): TypedPipe[(T, (Long, Long))] = diffByGroup(left, right, reducers)(_.hashCode) object Enrichments { implicit class Diff[T](val left: TypedPipe[T]) extends AnyVal { - def diff(right: TypedPipe[T], reducers: Option[Int] = None)(implicit ev: Ordering[T]): UnsortedGrouped[T, (Long, Long)] = + def diff(right: TypedPipe[T], reducers: Option[Int] = None)(implicit ev: Ordering[T], q: Quoted): UnsortedGrouped[T, (Long, Long)] = TypedPipeDiff.diff(left, right, reducers) - def diffByGroup[K: Ordering](right: TypedPipe[T], reducers: Option[Int] = None)(groupByFn: T => K): TypedPipe[(T, (Long, Long))] = + def diffByGroup[K: Ordering](right: TypedPipe[T], reducers: Option[Int] = None)(groupByFn: T => K)(implicit q: Quoted): TypedPipe[(T, (Long, Long))] = TypedPipeDiff.diffByGroup(left, right, reducers)(groupByFn) - def diffByHashCode(right: TypedPipe[T], reducers: Option[Int] = None): TypedPipe[(T, (Long, Long))] = TypedPipeDiff.diffByHashCode(left, right, reducers) + def diffByHashCode(right: TypedPipe[T], reducers: Option[Int] = None)(implicit q: Quoted): TypedPipe[(T, (Long, Long))] = + TypedPipeDiff.diffByHashCode(left, right, reducers) } implicit class DiffArray[T](val left: TypedPipe[Array[T]]) extends AnyVal { - def diffArrayPipes(right: TypedPipe[Array[T]], reducers: Option[Int] = None)(implicit ev: ClassTag[T]): TypedPipe[(Array[T], (Long, Long))] = + def diffArrayPipes(right: TypedPipe[Array[T]], reducers: Option[Int] = None)(implicit ev: ClassTag[T], q: Quoted): TypedPipe[(Array[T], (Long, Long))] = TypedPipeDiff.diffArrayPipes(left, right, reducers) } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/ValuePipe.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/ValuePipe.scala index 5a0ec8f860..c4db6aa4ec 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/ValuePipe.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/ValuePipe.scala @@ -19,11 +19,12 @@ import com.twitter.algebird._ import com.twitter.scalding.{ Mode, IterableSource } import com.twitter.scalding.Execution +import com.twitter.scalding.quotation.Quoted object ValuePipe extends java.io.Serializable { implicit def toTypedPipe[V](v: ValuePipe[V]): TypedPipe[V] = v.toTypedPipe - def fold[T, U, V](l: ValuePipe[T], r: ValuePipe[U])(f: (T, U) => V): ValuePipe[V] = + def fold[T, U, V](l: ValuePipe[T], r: ValuePipe[U])(f: (T, U) => V)(implicit q: Quoted): ValuePipe[V] = l.leftCross(r).collect { case (t, Some(u)) => f(t, u) } def apply[T](t: T): ValuePipe[T] = LiteralValue(t) @@ -36,17 +37,17 @@ object ValuePipe extends java.io.Serializable { * It allows to perform scalar based operations on pipes like normalization. */ sealed trait ValuePipe[+T] extends java.io.Serializable { - def leftCross[U](that: ValuePipe[U]): ValuePipe[(T, Option[U])] = that match { + def leftCross[U](that: ValuePipe[U])(implicit q: Quoted): ValuePipe[(T, Option[U])] = that match { case EmptyValue => map((_, None)) case LiteralValue(v2) => map((_, Some(v2))) // We don't know if a computed value is empty or not. We need to run the MR job: case _ => ComputedValue(toTypedPipe.leftCross(that)) } - def collect[U](fn: PartialFunction[T, U]): ValuePipe[U] = + def collect[U](fn: PartialFunction[T, U])(implicit q: Quoted): ValuePipe[U] = filter(fn.isDefinedAt(_)).map(fn(_)) - def map[U](fn: T => U): ValuePipe[U] - def filter(fn: T => Boolean): ValuePipe[T] + def map[U](fn: T => U)(implicit q: Quoted): ValuePipe[U] + def filter(fn: T => Boolean)(implicit q: Quoted): ValuePipe[T] /** * Identical to toOptionExecution.map(_.get) * The result will be an exception if there is no value. @@ -84,36 +85,36 @@ sealed trait ValuePipe[+T] extends java.io.Serializable { } } - def debug: ValuePipe[T] + def debug(implicit q: Quoted): ValuePipe[T] } case object EmptyValue extends ValuePipe[Nothing] { - override def leftCross[U](that: ValuePipe[U]) = this - override def map[U](fn: Nothing => U): ValuePipe[U] = this - override def filter(fn: Nothing => Boolean) = this + override def leftCross[U](that: ValuePipe[U])(implicit q: Quoted) = this + override def map[U](fn: Nothing => U)(implicit q: Quoted): ValuePipe[U] = this + override def filter(fn: Nothing => Boolean)(implicit q: Quoted) = this override def toTypedPipe: TypedPipe[Nothing] = TypedPipe.empty override def toOptionExecution = Execution.from(None) - def debug: ValuePipe[Nothing] = { + def debug(implicit q: Quoted): ValuePipe[Nothing] = { println("EmptyValue") this } } final case class LiteralValue[T](value: T) extends ValuePipe[T] { - override def map[U](fn: T => U) = LiteralValue(fn(value)) - override def filter(fn: T => Boolean) = if (fn(value)) this else EmptyValue + override def map[U](fn: T => U)(implicit q: Quoted) = LiteralValue(fn(value)) + override def filter(fn: T => Boolean)(implicit q: Quoted) = if (fn(value)) this else EmptyValue override def toTypedPipe = TypedPipe.from(Iterable(value)) override def toOptionExecution = Execution.from(Some(value)) - def debug: ValuePipe[T] = map { v => + def debug(implicit q: Quoted): ValuePipe[T] = map { (v: T) => println("LiteralValue(" + v.toString + ")") v } } final case class ComputedValue[T](override val toTypedPipe: TypedPipe[T]) extends ValuePipe[T] { - override def map[U](fn: T => U) = ComputedValue(toTypedPipe.map(fn)) - override def filter(fn: T => Boolean) = ComputedValue(toTypedPipe.filter(fn)) + override def map[U](fn: T => U)(implicit q: Quoted) = ComputedValue(toTypedPipe.map(fn)) + override def filter(fn: T => Boolean)(implicit q: Quoted) = ComputedValue(toTypedPipe.filter(fn)) - def debug: ValuePipe[T] = map { value => + def debug(implicit q: Quoted): ValuePipe[T] = map { (value: T) => println("ComputedValue(" + value.toString + ")") value } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/cascading_backend/CascadingBackend.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/cascading_backend/CascadingBackend.scala index 44e2898928..6c213ed1a4 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/cascading_backend/CascadingBackend.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/cascading_backend/CascadingBackend.scala @@ -24,6 +24,8 @@ import com.twitter.scalding.serialization.{ } import java.util.WeakHashMap import scala.collection.mutable.{ Map => MMap } +import org.apache.hadoop.mapred.JobConf +import com.twitter.scalding.quotation.Quoted object CascadingBackend { @@ -194,9 +196,9 @@ object CascadingBackend { } go(cp) case (cp@CrossPipe(_, _), rec) => - rec(cp.viaHashJoin) + rec(cp.viaHashJoin(Quoted.internal)) case (cv@CrossValue(_, _), rec) => - rec(cv.viaHashJoin) + rec(cv.viaHashJoin(Quoted.internal)) case (DebugPipe(p), rec) => val inner = rec(p) inner.copy(pipe = new Each(inner.pipe, new Debug)) @@ -306,12 +308,11 @@ object CascadingBackend { val pipe = RichPipe.assignName(pp) fd.addTrap(pipe, sink.createTap(Write)(mode)) CascadingPipe[u](pipe, sink.sinkFields, fd, conv) - case (WithDescriptionTypedPipe(input, descs), rec) => - + case (WithDescriptionTypedPipe(input, descs, quoted), rec) => @annotation.tailrec def loop[A](t: TypedPipe[A], acc: List[(String, Boolean)]): (TypedPipe[A], List[(String, Boolean)]) = t match { - case WithDescriptionTypedPipe(i, descs) => + case WithDescriptionTypedPipe(i, descs, quoted) => loop(i, descs ::: acc) case notDescr => (notDescr, acc) } diff --git a/scalding-core/src/test/scala/com/twitter/scalding/ReferencedClassFinderTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/ReferencedClassFinderTest.scala index 6d6af5bf8d..a8db445106 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/ReferencedClassFinderTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/ReferencedClassFinderTest.scala @@ -2,6 +2,7 @@ package com.twitter.scalding import org.apache.hadoop.io.BytesWritable import org.scalatest.{ Matchers, WordSpec } +import com.twitter.scalding.quotation.Quoted case class C1(a: Int) case class C2(b: Int) @@ -18,7 +19,7 @@ class ReferencedClassFinderExample(args: Args) extends Job(args) with TraitType val tp = TypedPipe.from(List(C1(1), C1(1), C1(2), C1(3), C1(5))) val grouped = tp.groupBy(c => C2(c.a))(new Ordering[C2] { override def compare(a: C2, b: C2) = b.b - a.b - }) + }, implicitly[Quoted]) // Verify that we can inspect private[this] fields private[this] val withTuple = grouped.toList.mapValues(list => C3(list.length)) // Verify that we don't assign a >= 128 token to a class that has a < 128 token diff --git a/scalding-core/src/test/scala/com/twitter/scalding/typed/InAnotherPackage.scala b/scalding-core/src/test/scala/com/twitter/scalding/typed/InAnotherPackage.scala deleted file mode 100644 index 108884acce..0000000000 --- a/scalding-core/src/test/scala/com/twitter/scalding/typed/InAnotherPackage.scala +++ /dev/null @@ -1,14 +0,0 @@ -package com.twitter.example.scalding.typed - -import com.twitter.scalding._ -import scala.concurrent.{ ExecutionContext => SExecutionContext, _ } -import SExecutionContext.Implicits.global - -object InAnotherPackage { - def buildF: Future[TypedPipe[(Int, Int)]] = { - Future { - TypedPipe.from(List(1, 2, 3, 4, 555, 3)) - .map { case x => (x, x) } - } - } -} diff --git a/scalding-core/src/test/scala/com/twitter/scalding/typed/NoStackLineNumberTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/typed/NoStackLineNumberTest.scala deleted file mode 100644 index 6475d28fc0..0000000000 --- a/scalding-core/src/test/scala/com/twitter/scalding/typed/NoStackLineNumberTest.scala +++ /dev/null @@ -1,51 +0,0 @@ -/* -Copyright 2015 Twitter, Inc. - -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. -*/ -package com.twitter.scalding.typed - -import org.scalatest.WordSpec - -import com.twitter.scalding._ -import scala.concurrent.{ ExecutionContext => SExecutionContext, _ } -import SExecutionContext.Implicits.global -import scala.concurrent.duration.{ Duration => SDuration } - -import cascading.flow.FlowDef -import org.apache.hadoop.conf.Configuration - -class NoStackLineNumberTest extends WordSpec { - - "No Stack Shouldn't block getting line number info" should { - "actually get the no stack info" in { - import Dsl._ - implicit val fd: FlowDef = new FlowDef - implicit val m: Hdfs = new Hdfs(false, new Configuration) - - val pipeFut = com.twitter.example.scalding.typed.InAnotherPackage.buildF.map { tp => - tp.toPipe('a, 'b) - } - val pipe = Await.result(pipeFut, SDuration.Inf) - // We pick up line number info via TypedPipe.withLine - // So this should have some non-scalding info in it. - val allDesc = RichPipe(pipe) - .upstreamPipes - .map(RichPipe.getPipeDescriptions(_).toSet) - .foldLeft(Set.empty[String])(_ | _) - - assert(allDesc.size > 0) - assert(allDesc.exists(_.contains("com.twitter.example.scalding.typed.InAnotherPackage"))) - } - } -} diff --git a/scalding-core/src/test/scala/com/twitter/scalding/typed/OptimizationRulesTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/typed/OptimizationRulesTest.scala index 4d10e3137d..18a6e2e689 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/typed/OptimizationRulesTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/typed/OptimizationRulesTest.scala @@ -14,6 +14,7 @@ import org.scalatest.prop.PropertyChecks import org.scalatest.prop.GeneratorDrivenPropertyChecks.PropertyCheckConfiguration import org.scalacheck.{ Arbitrary, Gen } import scala.util.{ Failure, Success, Try } +import com.twitter.scalding.quotation.Quoted object TypedPipeGen { val srcGen: Gen[TypedPipe[Int]] = { @@ -468,7 +469,7 @@ class OptimizationRulesTest extends FunSuite with PropertyChecks { eqCheck(tp.hashLookup(keyed)) eqCheck(tp.groupRandomly(100)) val ordInt = implicitly[Ordering[Int]] - eqCheck(tp.distinctBy(fn0)(ordInt)) + eqCheck(tp.distinctBy(fn0)(ordInt, implicitly[Quoted])) } } diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/platform/PlatformTest.scala b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/platform/PlatformTest.scala index 62f7d081a9..1bebd1c1d2 100644 --- a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/platform/PlatformTest.scala +++ b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/platform/PlatformTest.scala @@ -31,6 +31,7 @@ import org.scalatest.{ Matchers, WordSpec } import scala.collection.JavaConverters._ import scala.language.experimental.macros +import com.twitter.scalding.quotation.Quoted class InAndOutJob(args: Args) extends Job(args) { Tsv("input").read.write(Tsv("output")) @@ -116,13 +117,13 @@ class MultipleGroupByJob(args: Args) extends Job(args) { TypedPipe.from(data) .map{ k => (k, 1L) } - .group(stringOrdSer) + .group(stringOrdSer, implicitly[Quoted]) .sum .map { case (k, _) => ((k, k), 1L) } - .sumByKey(stringTup2OrdSer, implicitly) + .sumByKey(stringTup2OrdSer, implicitly, implicitly[Quoted]) .map(_._1._1) .map { t => (t.toString, t) @@ -653,8 +654,8 @@ class PlatformTest extends WordSpec with Matchers with HadoopSharedPlatformTest "reduce stage - sum", "write", // should see the .group and the .write show up as line numbers - "com.twitter.scalding.platform.TypedPipeWithDescriptionJob.(TestJobsWithDescriptions.scala:30)", - "com.twitter.scalding.platform.TypedPipeWithDescriptionJob.(TestJobsWithDescriptions.scala:34)") + "TestJobsWithDescriptions.scala:31", + "TestJobsWithDescriptions.scala:34") val foundDescs = steps.map(_.getConfig.get(Config.StepDescriptions)) descs.foreach { d =>