From 727522110a308b45da40d8c5b1d1e4e305fb56c7 Mon Sep 17 00:00:00 2001 From: Oscar Boykin Date: Thu, 29 Mar 2012 17:07:32 -0700 Subject: [PATCH 01/25] Adds a test for leftJoin --- .../scala/com/twitter/scalding/CoreTest.scala | 48 ++++++++++++++++++- 1 file changed, 46 insertions(+), 2 deletions(-) diff --git a/src/test/scala/com/twitter/scalding/CoreTest.scala b/src/test/scala/com/twitter/scalding/CoreTest.scala index 9611e4cc7d..8b50804036 100644 --- a/src/test/scala/com/twitter/scalding/CoreTest.scala +++ b/src/test/scala/com/twitter/scalding/CoreTest.scala @@ -4,6 +4,7 @@ import cascading.tuple.Fields import cascading.tuple.TupleEntry import org.specs._ +import java.lang.{Integer => JInt} class NumberJoinerJob(args : Args) extends Job(args) { val in0 = Tsv("input0").read.mapTo((0,1) -> ('x0, 'y0)) { input : (Int, Int) => input } @@ -131,6 +132,7 @@ class JoinJob(args: Args) extends Job(args) { } class JoinTest extends Specification with TupleConversions { + noDetailedDiffs() //Fixes an issue with scala 2.9 "A JoinJob" should { val input1 = List("a" -> 1, "b" -> 2, "c" -> 3) val input2 = List("b" -> -1, "c" -> 5, "d" -> 4) @@ -212,7 +214,7 @@ class TinyJoinJob(args: Args) extends Job(args) { class TinyJoinTest extends Specification with TupleConversions { noDetailedDiffs() //Fixes an issue with scala 2.9 - "A JoinJob" should { + "A TinyJoinJob" should { val input1 = List("a" -> 1, "b" -> 2, "c" -> 3) val input2 = List("b" -> -1, "c" -> 5, "d" -> 4) val correctOutput = Map("b" -> (2, -1), "c" -> (3, 5)) @@ -251,7 +253,7 @@ class TinyCollisionJoinJob(args: Args) extends Job(args) { class TinyCollisionJoinTest extends Specification with TupleConversions { noDetailedDiffs() //Fixes an issue with scala 2.9 - "A JoinJob" should { + "A TinyCollisionJoinJob" should { val input1 = List("a" -> 1, "b" -> 2, "c" -> 3) val input2 = List("b" -> -1, "c" -> 5, "d" -> 4) val correctOutput = Map("b" -> (2, -1), "c" -> (3, 5)) @@ -312,6 +314,48 @@ class TinyThenSmallJoinTest extends Specification with TupleConversions with Fie } } +class LeftJoinJob(args: Args) extends Job(args) { + val p1 = Tsv(args("input1")) + .mapTo((0, 1) -> ('k1, 'v1)) { v : (String, Int) => v } + val p2 = Tsv(args("input2")) + .mapTo((0, 1) -> ('k2, 'v2)) { v : (String, Int) => v } + p1.leftJoinWithSmaller('k1 -> 'k2, p2) + .project('k1, 'v1, 'v2) + // Null sent to TSV will not be read in properly + .map('v2 -> 'v2) { v : AnyRef => Option(v).map { _.toString }.getOrElse("NULL") } + .write( Tsv(args("output")) ) +} + +class LeftJoinTest extends Specification with TupleConversions { + noDetailedDiffs() //Fixes an issue with scala 2.9 + "A LeftJoinJob" should { + val input1 = List("a" -> 1, "b" -> 2, "c" -> 3) + val input2 = List("b" -> -1, "c" -> 5, "d" -> 4) + val correctOutput = Map[String,(Int,AnyRef)]("a" -> (1,"NULL"), "b" -> (2, "-1"), + "c" -> (3, "5")) + + JobTest("com.twitter.scalding.LeftJoinJob") + .arg("input1", "fakeInput1") + .arg("input2", "fakeInput2") + .arg("output", "fakeOutput") + .source(Tsv("fakeInput1"), input1) + .source(Tsv("fakeInput2"), input2) + .sink[(String,Int,JInt)](Tsv("fakeOutput")) { outBuf => + val actualOutput = outBuf.map { input : (String,Int,AnyRef) => + println(input) + val (k, v1, v2) = input + (k,(v1, v2)) + }.toMap + "join tuples with the same key" in { + correctOutput must be_==(actualOutput) + } + } + .run + .runHadoop + .finish + } +} + class MergeTestJob(args : Args) extends Job(args) { val in = TextLine(args("in")).read.mapTo(1->('x,'y)) { line : String => From a6b9fdc6fd2d5094acb261712b3c4e83a6aa3385 Mon Sep 17 00:00:00 2001 From: Oscar Boykin Date: Thu, 5 Apr 2012 11:28:39 -0700 Subject: [PATCH 02/25] Clean up head/last --- .../com/twitter/scalding/GroupBuilder.scala | 16 ++++++++----- .../twitter/scalding/TupleConversions.scala | 7 ++++++ .../scala/com/twitter/scalding/CoreTest.scala | 23 +++++++++++++++++++ 3 files changed, 40 insertions(+), 6 deletions(-) diff --git a/src/main/scala/com/twitter/scalding/GroupBuilder.scala b/src/main/scala/com/twitter/scalding/GroupBuilder.scala index fe1888a84d..7641d00cc1 100644 --- a/src/main/scala/com/twitter/scalding/GroupBuilder.scala +++ b/src/main/scala/com/twitter/scalding/GroupBuilder.scala @@ -23,6 +23,7 @@ import cascading.operation._ import cascading.operation.aggregator._ import cascading.operation.filter._ import cascading.tuple.Fields +import cascading.tuple.{Tuple => CTuple} import scala.collection.JavaConverters._ import scala.annotation.tailrec @@ -61,10 +62,6 @@ class GroupBuilder(val groupFields : Fields) extends FieldConversions //Put any pure reduce functions into the below object import CommonReduceFunctions._ - def aggregate(args : Fields)(a : Aggregator[_]) : GroupBuilder = { - every(pipe => new Every(pipe, args, a)) - } - private def tryAggregateBy(ab : AggregateBy, ev : Pipe => Every) : Boolean = { // Concat if there if not none reds = reds.map(rl => ab::rl) @@ -271,8 +268,15 @@ class GroupBuilder(val groupFields : Fields) extends FieldConversions } // Return the first, useful probably only for sorted case. - def head(f : Fields) = aggregate(f)(new First()) - def last(f : Fields) = aggregate(f)(new Last()) + def head(fd : (Fields,Fields)) : GroupBuilder = { + reduce[CTuple](fd) { (oldVal, newVal) => oldVal } + } + def head(f : Symbol*) : GroupBuilder = head(f -> f) + + def last(fd : (Fields,Fields)) = { + reduce[CTuple](fd) { (oldVal, newVal) => newVal } + } + def last(f : Symbol*) : GroupBuilder = last(f -> f) private def extremum(max : Boolean, fieldDef : (Fields,Fields)) : GroupBuilder = { val (fromFields, toFields) = fieldDef diff --git a/src/main/scala/com/twitter/scalding/TupleConversions.scala b/src/main/scala/com/twitter/scalding/TupleConversions.scala index 94e0a57d54..3d492c8a98 100644 --- a/src/main/scala/com/twitter/scalding/TupleConversions.scala +++ b/src/main/scala/com/twitter/scalding/TupleConversions.scala @@ -18,6 +18,7 @@ package com.twitter.scalding import cascading.tuple.TupleEntry import cascading.tuple.TupleEntryIterator import cascading.tuple.{Tuple => CTuple} +import cascading.tuple.Tuples trait TupleConversions extends GeneratedConversions { @@ -47,6 +48,12 @@ trait TupleConversions extends GeneratedConversions { override def arity = -1 } + implicit object CTupleConverter extends TupleConverter[CTuple] { + override def apply(tup : TupleEntry) = Tuples.asUnmodifiable(tup.getTuple) + override def arity = -1 + } + + implicit def iterableToIterable [A] (iterable : java.lang.Iterable[A]) : Iterable[A] = { if(iterable == null) { None diff --git a/src/test/scala/com/twitter/scalding/CoreTest.scala b/src/test/scala/com/twitter/scalding/CoreTest.scala index 8b50804036..ae4bcec219 100644 --- a/src/test/scala/com/twitter/scalding/CoreTest.scala +++ b/src/test/scala/com/twitter/scalding/CoreTest.scala @@ -809,3 +809,26 @@ class IterableSourceTest extends Specification with TupleConversions with FieldC .finish } } + +class HeadLastJob(args : Args) extends Job(args) { + Tsv("input",('x,'y)).groupBy('x) { + _.sortBy('y) + .head('y -> 'yh).last('y -> 'yl) + }.write(Tsv("output")) +} + +class HeadLastTest extends Specification with TupleConversions with FieldConversions { + noDetailedDiffs() + val input = List((1,10),(1,20),(1,30),(2,0)) + "A IterableSourceJob" should { + JobTest("com.twitter.scalding.HeadLastJob") + .source(Tsv("input",('x,'y)), input) + .sink[(Int,Int,Int)](Tsv("output")) { outBuf => + "Correctly do head/last" in { + outBuf.toList must be_==(List((1,10,30),(2,0,0))) + } + } + .run + .finish + } +} From cbf393b18639b16003f6555d8b4b411bef880499 Mon Sep 17 00:00:00 2001 From: Argyris Zymnis Date: Mon, 2 Apr 2012 23:05:57 -0700 Subject: [PATCH 03/25] Add coGroup as a method of GroupBuilder. --- .../com/twitter/scalding/GroupBuilder.scala | 82 +++++++++++++++---- .../com/twitter/scalding/JoinAlgorithms.scala | 13 ++- .../com/twitter/scalding/CoGroupTest.scala | 40 +++++++++ 3 files changed, 117 insertions(+), 18 deletions(-) create mode 100644 src/test/scala/com/twitter/scalding/CoGroupTest.scala diff --git a/src/main/scala/com/twitter/scalding/GroupBuilder.scala b/src/main/scala/com/twitter/scalding/GroupBuilder.scala index 7641d00cc1..5980b5640b 100644 --- a/src/main/scala/com/twitter/scalding/GroupBuilder.scala +++ b/src/main/scala/com/twitter/scalding/GroupBuilder.scala @@ -15,10 +15,9 @@ limitations under the License. */ package com.twitter.scalding -import cascading.pipe.Pipe -import cascading.pipe.Every -import cascading.pipe.GroupBy +import cascading.pipe._ import cascading.pipe.assembly._ +import cascading.pipe.joiner._ import cascading.operation._ import cascading.operation.aggregator._ import cascading.operation.filter._ @@ -29,6 +28,8 @@ import scala.collection.JavaConverters._ import scala.annotation.tailrec import scala.math.Ordering +import java.lang.IllegalArgumentException + // This controls the sequence of reductions that happen inside a // particular grouping operation. Not all elements can be combined, // for instance, a scanLeft/foldLeft generally requires a sorting @@ -48,6 +49,8 @@ class GroupBuilder(val groupFields : Fields) extends FieldConversions private var evs : List[Pipe => Every] = Nil private var isReversed : Boolean = false private var sortBy : Option[Fields] = None + private var coGroups : List[(Fields, Pipe)] = Nil + private var joiner : Option[Joiner] = None /* * maxMF is the maximum index of a "middle field" allocated for mapReduceMap operations */ @@ -62,6 +65,17 @@ class GroupBuilder(val groupFields : Fields) extends FieldConversions //Put any pure reduce functions into the below object import CommonReduceFunctions._ + // Joins (cogroups) with pipe p on fields f. + // Make sure that pipe p is smaller than the left side pipe, otherwise this + // might take a while. + def coGroup(f : Fields, p : Pipe) = { + coGroups ::= (f, RichPipe.assignName(p)) + // aggregateBy replaces the grouping operation + // but we actually want to do a coGroup + reds = None + this + } + private def tryAggregateBy(ab : AggregateBy, ev : Pipe => Every) : Boolean = { // Concat if there if not none reds = reds.map(rl => ab::rl) @@ -409,14 +423,25 @@ class GroupBuilder(val groupFields : Fields) extends FieldConversions every(pipe => new Every(pipe, inFields, b)) } - def schedule(name : String, allpipes : Pipe*) : Pipe = { - val mpipes : Array[Pipe] = allpipes.toArray - reds match { - case None => { - //We cannot aggregate, so group: + def groupMode : GroupMode = { + if(!coGroups.isEmpty) { + return CoGroupMode + } + return reds match { + case None => GroupByMode + case Some(Nil) => IdentityMode + case Some(redList) => AggregateByMode + } + } + + def schedule(name : String, pipe : Pipe) : Pipe = { + + groupMode match { + //In this case we cannot aggregate, so group: + case GroupByMode => { val startPipe : Pipe = sortBy match { - case None => new GroupBy(name, mpipes, groupFields) - case Some(sf) => new GroupBy(name, mpipes, groupFields, sf, isReversed) + case None => new GroupBy(name, pipe, groupFields) + case Some(sf) => new GroupBy(name, pipe, groupFields, sf, isReversed) } overrideReducers(startPipe) @@ -424,20 +449,43 @@ class GroupBuilder(val groupFields : Fields) extends FieldConversions evs.foldRight(startPipe)( (op : Pipe => Every, p) => op(p) ) } //This is the case where the group function is identity: { g => g } - case Some(Nil) => { - val gb = new GroupBy(name, mpipes, groupFields) + case IdentityMode => { + val gb = new GroupBy(name, pipe, groupFields) overrideReducers(gb) gb } //There is some non-empty AggregateBy to do: - case Some(redlist) => { + case AggregateByMode => { + val redlist = reds.get val THRESHOLD = 100000 //tune this, default is 10k - val ag = new AggregateBy(name, mpipes, groupFields, + val ag = new AggregateBy(name, pipe, groupFields, THRESHOLD, redlist.reverse.toArray : _*) overrideReducers(ag.getGroupBy()) ag } + case CoGroupMode => { + assert(!sortBy.isDefined, "cannot use a sortBy when doing a coGroup") + // overrideReducers(pipe) + val fields = (groupFields :: coGroups.map{ _._1 }).toArray + val pipes = (pipe :: coGroups.map{ _._2 }).toArray + val cg : Pipe = new CoGroup(pipes, fields, null, joiner.getOrElse(new InnerJoin)) + overrideReducers(cg) + evs.foldRight(cg)( (op : Pipe => Every, p) => op(p) ) + } + } + } + + def joiner(j : Joiner) : GroupBuilder = { + this.joiner = this.joiner match { + case None => Some(j) + case Some(otherJ) => if ( otherJ != j ) { + throw new IllegalArgumentException("trying to set joiner to: " + + j + " while already set to: " + otherJ) + } else { + Some(otherJ) + } } + this } //This invalidates aggregateBy! @@ -554,3 +602,9 @@ object CommonReduceFunctions extends java.io.Serializable { mergeSortR(Nil, v1, v2, k).reverse } } + +sealed abstract class GroupMode +case object AggregateByMode extends GroupMode +case object GroupByMode extends GroupMode +case object CoGroupMode extends GroupMode +case object IdentityMode extends GroupMode diff --git a/src/main/scala/com/twitter/scalding/JoinAlgorithms.scala b/src/main/scala/com/twitter/scalding/JoinAlgorithms.scala index df33e3d704..c9354a47b7 100644 --- a/src/main/scala/com/twitter/scalding/JoinAlgorithms.scala +++ b/src/main/scala/com/twitter/scalding/JoinAlgorithms.scala @@ -97,7 +97,11 @@ trait JoinAlgorithms { val intersection = asSet(fs._1).intersect(asSet(fs._2)) if (intersection.size == 0) { // Common case: no intersection in names: just CoGroup, which duplicates the grouping fields: - setReducers(new CoGroup(assignName(pipe), fs._1, assignName(that), fs._2, joiner), reducers) + assignName(pipe).groupBy(fs._1) { + _.coGroup(fs._2, that) + .joiner(joiner) + .reducers(reducers) + } } else if (joiner.isInstanceOf[InnerJoin]) { /* @@ -106,9 +110,10 @@ trait JoinAlgorithms { * So, we rename the right hand side to temporary names, then discard them after the operation */ val (renamedThat, newJoinFields, temp) = renameCollidingFields(that, fs._2, intersection) - setReducers(new CoGroup(assignName(pipe), fs._1, - assignName(renamedThat), newJoinFields, joiner), reducers) - .discard(temp) + assignName(pipe).groupBy(fs._1) { + _.coGroup(newJoinFields, renamedThat) + .reducers(reducers) + }.discard(temp) } else { throw new IllegalArgumentException("join keys must be disjoint unless you are doing an InnerJoin. Found: " + diff --git a/src/test/scala/com/twitter/scalding/CoGroupTest.scala b/src/test/scala/com/twitter/scalding/CoGroupTest.scala new file mode 100644 index 0000000000..6034b6fed3 --- /dev/null +++ b/src/test/scala/com/twitter/scalding/CoGroupTest.scala @@ -0,0 +1,40 @@ +package com.twitter.scalding + +import cascading.pipe.joiner._ +import org.specs._ + +class StarJoinJob(args : Args) extends Job(args) { + val in0 = Tsv("input0").read.mapTo((0,1) -> ('x0, 'a)) { input : (Int, Int) => input } + val in1 = Tsv("input1").read.mapTo((0,1) -> ('x1, 'b)) { input : (Int, Int) => input } + val in2 = Tsv("input2").read.mapTo((0,1) -> ('x2, 'c)) { input : (Int, Int) => input } + val in3 = Tsv("input3").read.mapTo((0,1) -> ('x3, 'd)) { input : (Int, Int) => input } + + in0.groupBy('x0) { + _.coGroup('x1, in1) + .coGroup('x2, in2) + .coGroup('x3, in3) + .joiner(new MixedJoin(Array(true, false, false, false))) + } + .project('x0, 'a, 'b, 'c, 'd) + .write(Tsv("output")) +} + +class CoGroupTest extends Specification with TupleConversions { + noDetailedDiffs() + "A StarJoinJob" should { + JobTest("com.twitter.scalding.StarJoinJob") + .source(Tsv("input0"), List((0, 1), (1, 1), (2, 1), (3, 2))) + .source(Tsv("input1"), List((0, 1), (2, 5), (3, 2))) + .source(Tsv("input2"), List((1, 1), (2, 8))) + .source(Tsv("input3"), List((0, 9), (2, 11))) + .sink[(Int, Int, Int, Int, Int)](Tsv("output")) { outputBuf => + "be able to work" in { + val out = outputBuf.toSet + val expected = Set((0,1,1,0,9), (1,1,0,1,0), (2,1,5,8,11), (3,2,2,0,0)) + out must_== expected + } + } + .run + .finish + } +} From 722adfae65b35708aa051bd976a3fe6c3a2d75c0 Mon Sep 17 00:00:00 2001 From: Argyris Zymnis Date: Thu, 5 Apr 2012 14:05:48 -0700 Subject: [PATCH 04/25] Move coGroup to its own builder. --- .../com/twitter/scalding/GroupBuilder.scala | 56 +++---------------- .../com/twitter/scalding/JoinAlgorithms.scala | 49 ++++++++-------- .../com/twitter/scalding/BlockJoinTest.scala | 10 ++-- .../com/twitter/scalding/CoGroupTest.scala | 9 ++- 4 files changed, 43 insertions(+), 81 deletions(-) diff --git a/src/main/scala/com/twitter/scalding/GroupBuilder.scala b/src/main/scala/com/twitter/scalding/GroupBuilder.scala index 5980b5640b..1ec2290d0b 100644 --- a/src/main/scala/com/twitter/scalding/GroupBuilder.scala +++ b/src/main/scala/com/twitter/scalding/GroupBuilder.scala @@ -17,7 +17,6 @@ package com.twitter.scalding import cascading.pipe._ import cascading.pipe.assembly._ -import cascading.pipe.joiner._ import cascading.operation._ import cascading.operation.aggregator._ import cascading.operation.filter._ @@ -46,11 +45,9 @@ class GroupBuilder(val groupFields : Fields) extends FieldConversions /** * This is the description of this Grouping in terms of a sequence of Every operations */ - private var evs : List[Pipe => Every] = Nil - private var isReversed : Boolean = false - private var sortBy : Option[Fields] = None - private var coGroups : List[(Fields, Pipe)] = Nil - private var joiner : Option[Joiner] = None + protected var evs : List[Pipe => Every] = Nil + protected var isReversed : Boolean = false + protected var sortBy : Option[Fields] = None /* * maxMF is the maximum index of a "middle field" allocated for mapReduceMap operations */ @@ -65,17 +62,6 @@ class GroupBuilder(val groupFields : Fields) extends FieldConversions //Put any pure reduce functions into the below object import CommonReduceFunctions._ - // Joins (cogroups) with pipe p on fields f. - // Make sure that pipe p is smaller than the left side pipe, otherwise this - // might take a while. - def coGroup(f : Fields, p : Pipe) = { - coGroups ::= (f, RichPipe.assignName(p)) - // aggregateBy replaces the grouping operation - // but we actually want to do a coGroup - reds = None - this - } - private def tryAggregateBy(ab : AggregateBy, ev : Pipe => Every) : Boolean = { // Concat if there if not none reds = reds.map(rl => ab::rl) @@ -98,7 +84,7 @@ class GroupBuilder(val groupFields : Fields) extends FieldConversions this } - private def overrideReducers(p : Pipe) : Pipe = { + protected def overrideReducers(p : Pipe) : Pipe = { numReducers.map { r => RichPipe.setReducers(p, r) }.getOrElse(p) } @@ -424,9 +410,6 @@ class GroupBuilder(val groupFields : Fields) extends FieldConversions } def groupMode : GroupMode = { - if(!coGroups.isEmpty) { - return CoGroupMode - } return reds match { case None => GroupByMode case Some(Nil) => IdentityMode @@ -463,31 +446,9 @@ class GroupBuilder(val groupFields : Fields) extends FieldConversions overrideReducers(ag.getGroupBy()) ag } - case CoGroupMode => { - assert(!sortBy.isDefined, "cannot use a sortBy when doing a coGroup") - // overrideReducers(pipe) - val fields = (groupFields :: coGroups.map{ _._1 }).toArray - val pipes = (pipe :: coGroups.map{ _._2 }).toArray - val cg : Pipe = new CoGroup(pipes, fields, null, joiner.getOrElse(new InnerJoin)) - overrideReducers(cg) - evs.foldRight(cg)( (op : Pipe => Every, p) => op(p) ) - } } } - def joiner(j : Joiner) : GroupBuilder = { - this.joiner = this.joiner match { - case None => Some(j) - case Some(otherJ) => if ( otherJ != j ) { - throw new IllegalArgumentException("trying to set joiner to: " + - j + " while already set to: " + otherJ) - } else { - Some(otherJ) - } - } - this - } - //This invalidates aggregateBy! def sortBy(f : Fields) : GroupBuilder = { reds = None @@ -603,8 +564,7 @@ object CommonReduceFunctions extends java.io.Serializable { } } -sealed abstract class GroupMode -case object AggregateByMode extends GroupMode -case object GroupByMode extends GroupMode -case object CoGroupMode extends GroupMode -case object IdentityMode extends GroupMode +sealed private[scalding] abstract class GroupMode +private[scalding] case object AggregateByMode extends GroupMode +private[scalding] case object GroupByMode extends GroupMode +private[scalding] case object IdentityMode extends GroupMode diff --git a/src/main/scala/com/twitter/scalding/JoinAlgorithms.scala b/src/main/scala/com/twitter/scalding/JoinAlgorithms.scala index c9354a47b7..198a66af67 100644 --- a/src/main/scala/com/twitter/scalding/JoinAlgorithms.scala +++ b/src/main/scala/com/twitter/scalding/JoinAlgorithms.scala @@ -19,7 +19,7 @@ import cascading.tap._ import cascading.scheme._ import cascading.pipe._ import cascading.pipe.assembly._ -import cascading.pipe.joiner._ +import cascading.pipe.joiner.{InnerJoin => CInnerJoin, LeftJoin => CLeftJoin} import cascading.flow._ import cascading.operation._ import cascading.operation.aggregator._ @@ -38,6 +38,10 @@ trait JoinAlgorithms { def pipe : Pipe + def coGroupBy(f : Fields, j : JoinMode = InnerJoinMode)(builder : CoGroupBuilder => GroupBuilder) : Pipe = { + builder(new CoGroupBuilder(f, j)).schedule(pipe.getName, pipe) + } + /* * WARNING! doing a cross product with even a moderate sized pipe can * create ENORMOUS output. The use-case here is attaching a constant (e.g. @@ -92,26 +96,25 @@ trait JoinAlgorithms { * joinWithSmaller(('other1, 'other2)->('this1, 'this2), pipe, new FancyJoin) * } */ - def joinWithSmaller(fs :(Fields,Fields), that : Pipe, joiner : Joiner = new InnerJoin, reducers : Int = -1) = { + def joinWithSmaller(fs :(Fields,Fields), that : Pipe, joiners : (JoinMode, JoinMode) = (InnerJoinMode, InnerJoinMode), reducers : Int = -1) = { // If we are not doing an inner join, the join fields must be disjoint: val intersection = asSet(fs._1).intersect(asSet(fs._2)) if (intersection.size == 0) { // Common case: no intersection in names: just CoGroup, which duplicates the grouping fields: - assignName(pipe).groupBy(fs._1) { - _.coGroup(fs._2, that) - .joiner(joiner) + assignName(pipe).coGroupBy(fs._1, joiners._1) { + _.coGroup(fs._2, that, joiners._2) .reducers(reducers) } } - else if (joiner.isInstanceOf[InnerJoin]) { + else if (joiners._1 == InnerJoinMode && joiners._2 == InnerJoinMode) { /* * Since it is an inner join, we only output if the key is present an equal in both sides. * For this (common) case, it doesn't matter if we drop one of the matching grouping fields. * So, we rename the right hand side to temporary names, then discard them after the operation */ val (renamedThat, newJoinFields, temp) = renameCollidingFields(that, fs._2, intersection) - assignName(pipe).groupBy(fs._1) { - _.coGroup(newJoinFields, renamedThat) + assignName(pipe).coGroupBy(fs._1, joiners._1) { + _.coGroup(newJoinFields, renamedThat, joiners._2) .reducers(reducers) }.discard(temp) } @@ -121,17 +124,17 @@ trait JoinAlgorithms { } } - def joinWithLarger(fs : (Fields, Fields), that : Pipe, joiner : Joiner = new InnerJoin, reducers : Int = -1) = { - that.joinWithSmaller((fs._2, fs._1), pipe, joiner, reducers) + def joinWithLarger(fs : (Fields, Fields), that : Pipe, joiners : (JoinMode, JoinMode) = (InnerJoinMode,InnerJoinMode), reducers : Int = -1) = { + that.joinWithSmaller((fs._2, fs._1), pipe, joiners, reducers) } def leftJoinWithSmaller(fs :(Fields,Fields), that : Pipe, reducers : Int = -1) = { - joinWithSmaller(fs, that, new LeftJoin, reducers) + joinWithSmaller(fs, that, (InnerJoinMode, OuterJoinMode), reducers) } def leftJoinWithLarger(fs :(Fields,Fields), that : Pipe, reducers : Int = -1) = { //We swap the order, and turn left into right: - that.joinWithSmaller((fs._2, fs._1), pipe, new RightJoin, reducers) + that.joinWithSmaller((fs._2, fs._1), pipe, (OuterJoinMode, InnerJoinMode), reducers) } /** @@ -148,18 +151,18 @@ trait JoinAlgorithms { def joinWithTiny(fs :(Fields,Fields), that : Pipe) = { val intersection = asSet(fs._1).intersect(asSet(fs._2)) if (intersection.size == 0) { - new Join(assignName(pipe), fs._1, assignName(that), fs._2, new InnerJoin) + new Join(assignName(pipe), fs._1, assignName(that), fs._2, new CInnerJoin) } else { val (renamedThat, newJoinFields, temp) = renameCollidingFields(that, fs._2, intersection) - (new Join(assignName(pipe), fs._1, assignName(renamedThat), newJoinFields, new InnerJoin)) + (new Join(assignName(pipe), fs._1, assignName(renamedThat), newJoinFields, new CInnerJoin)) .discard(temp) } } def leftJoinWithTiny(fs :(Fields,Fields), that : Pipe) = { //Rename these pipes to avoid cascading name conflicts - new Join(assignName(pipe), fs._1, assignName(that), fs._2, new LeftJoin) + new Join(assignName(pipe), fs._1, assignName(that), fs._2, new CLeftJoin) } /* @@ -190,11 +193,11 @@ trait JoinAlgorithms { */ def blockJoinWithSmaller(fs : (Fields, Fields), otherPipe : Pipe, rightReplication : Int = 1, leftReplication : Int = 1, - joiner : Joiner = new InnerJoin, reducers : Int = -1) : Pipe = { + joiners : (JoinMode, JoinMode) = (InnerJoinMode, InnerJoinMode), reducers : Int = -1) : Pipe = { assert(rightReplication > 0, "Must specify a positive number for the right replication in block join") assert(leftReplication > 0, "Must specify a positive number for the left replication in block join") - assertValidJoinMode(joiner, leftReplication, rightReplication) + assertValidJoinMode(joiners, leftReplication, rightReplication) // These are the new dummy fields used in the skew join val leftFields = new Fields("__LEFT_I__", "__LEFT_J__") @@ -208,7 +211,7 @@ trait JoinAlgorithms { val rightJoinFields = Fields.join(fs._2, rightFields) newLeft - .joinWithSmaller((leftJoinFields, rightJoinFields), newRight, joiner, reducers) + .joinWithSmaller((leftJoinFields, rightJoinFields), newRight, joiners, reducers) .discard(leftFields) .discard(rightFields) } @@ -225,11 +228,11 @@ trait JoinAlgorithms { } } - private def assertValidJoinMode(joiner : Joiner, left : Int, right : Int) { - (joiner, left, right) match { - case (i : InnerJoin, _, _) => true - case (k : LeftJoin, 1, _) => true - case (m : RightJoin, _, 1) => true + private def assertValidJoinMode(joiners : (JoinMode, JoinMode), left : Int, right : Int) { + (joiners, left, right) match { + case ((InnerJoinMode, InnerJoinMode), _, _) => true + case ((InnerJoinMode, OuterJoinMode), 1, _) => true + case ((OuterJoinMode, InnerJoinMode), _, 1) => true case (j, l, r) => throw new InvalidJoinModeException( "you cannot use joiner " + j + " with left replication " + l + " and right replication " + r diff --git a/src/test/scala/com/twitter/scalding/BlockJoinTest.scala b/src/test/scala/com/twitter/scalding/BlockJoinTest.scala index 32218cbd2c..ae54135a49 100644 --- a/src/test/scala/com/twitter/scalding/BlockJoinTest.scala +++ b/src/test/scala/com/twitter/scalding/BlockJoinTest.scala @@ -12,16 +12,16 @@ class InnerProductJob(args : Args) extends Job(args) { val l = args.getOrElse("left", "1").toInt val r = args.getOrElse("right", "1").toInt val j = args.getOrElse("joiner", "i") match { - case "i" => new InnerJoin - case "l" => new LeftJoin - case "r" => new RightJoin - case "o" => new OuterJoin + case "i" => (InnerJoinMode, InnerJoinMode) + case "l" => (InnerJoinMode, OuterJoinMode) + case "r" => (OuterJoinMode, InnerJoinMode) + case "o" => (OuterJoinMode, OuterJoinMode) } val in0 = Tsv("input0").read.mapTo((0,1,2) -> ('x1, 'y1, 's1)) { input : (Int, Int, Int) => input } val in1 = Tsv("input1").read.mapTo((0,1,2) -> ('x2, 'y2, 's2)) { input : (Int, Int, Int) => input } in0 - .blockJoinWithSmaller('y1 -> 'y2, in1, leftReplication = l, rightReplication = r, joiner = j) + .blockJoinWithSmaller('y1 -> 'y2, in1, leftReplication = l, rightReplication = r, joiners = j) .map(('s1, 's2) -> 'score) { v : (Int, Int) => v._1 * v._2 } diff --git a/src/test/scala/com/twitter/scalding/CoGroupTest.scala b/src/test/scala/com/twitter/scalding/CoGroupTest.scala index 6034b6fed3..cae084c3d1 100644 --- a/src/test/scala/com/twitter/scalding/CoGroupTest.scala +++ b/src/test/scala/com/twitter/scalding/CoGroupTest.scala @@ -9,11 +9,10 @@ class StarJoinJob(args : Args) extends Job(args) { val in2 = Tsv("input2").read.mapTo((0,1) -> ('x2, 'c)) { input : (Int, Int) => input } val in3 = Tsv("input3").read.mapTo((0,1) -> ('x3, 'd)) { input : (Int, Int) => input } - in0.groupBy('x0) { - _.coGroup('x1, in1) - .coGroup('x2, in2) - .coGroup('x3, in3) - .joiner(new MixedJoin(Array(true, false, false, false))) + in0.coGroupBy('x0) { + _.coGroup('x1, in1, OuterJoinMode) + .coGroup('x2, in2, OuterJoinMode) + .coGroup('x3, in3, OuterJoinMode) } .project('x0, 'a, 'b, 'c, 'd) .write(Tsv("output")) From 7a2a5859d16ddea9646f5d4d03792f5c30643d05 Mon Sep 17 00:00:00 2001 From: Argyris Zymnis Date: Thu, 5 Apr 2012 14:06:12 -0700 Subject: [PATCH 05/25] Add a CoGroupBuilder class. --- .../com/twitter/scalding/CoGroupBuilder.scala | 58 +++++++++++++++++++ 1 file changed, 58 insertions(+) create mode 100644 src/main/scala/com/twitter/scalding/CoGroupBuilder.scala diff --git a/src/main/scala/com/twitter/scalding/CoGroupBuilder.scala b/src/main/scala/com/twitter/scalding/CoGroupBuilder.scala new file mode 100644 index 0000000000..5d81b66de1 --- /dev/null +++ b/src/main/scala/com/twitter/scalding/CoGroupBuilder.scala @@ -0,0 +1,58 @@ +/* +Copyright 2012 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 + +import cascading.pipe.{CoGroup, Every, Pipe} +import cascading.pipe.joiner.MixedJoin +import cascading.tuple.Fields + +/** + * Builder classes used internally to implement coGroups (joins). + * Can also be used for more generalized joins, e.g., star joins. + * + */ +class CoGroupBuilder(groupFields : Fields, joinMode : JoinMode) extends GroupBuilder(groupFields) { + protected var coGroups : List[(Fields, Pipe, JoinMode)] = Nil + + // Joins (cogroups) with pipe p on fields f. + // Make sure that pipe p is smaller than the left side pipe, otherwise this + // might take a while. + def coGroup(f : Fields, p : Pipe, j : JoinMode = InnerJoinMode) = { + coGroups ::= (f, RichPipe.assignName(p), j) + this + } + + override def schedule(name : String, pipe : Pipe) : Pipe = { + assert(!sortBy.isDefined, "cannot use a sortBy when doing a coGroup") + val fields = (groupFields :: coGroups.map{ _._1 }).toArray + val pipes = (pipe :: coGroups.map{ _._2 }).toArray + val joinModes = (joinMode :: coGroups.map{ _._3 }).map{ _.booleanValue }.toArray + val mixedJoiner = new MixedJoin(joinModes) + val cg : Pipe = new CoGroup(pipes, fields, null, mixedJoiner) + overrideReducers(cg) + evs.foldRight(cg)( (op : Pipe => Every, p) => op(p) ) + } +} + +sealed abstract class JoinMode { + def booleanValue : Boolean +} +case object InnerJoinMode extends JoinMode { + override def booleanValue = true +} +case object OuterJoinMode extends JoinMode { + override def booleanValue = false +} From 1db425b33de28bcf912df596f1957a7f60fb0a60 Mon Sep 17 00:00:00 2001 From: Argyris Zymnis Date: Thu, 5 Apr 2012 14:59:24 -0700 Subject: [PATCH 06/25] Adding documentation and API cleanups. --- .../com/twitter/scalding/GroupBuilder.scala | 2 - .../com/twitter/scalding/JoinAlgorithms.scala | 55 +++++++++++++------ .../com/twitter/scalding/BlockJoinTest.scala | 10 ++-- 3 files changed, 43 insertions(+), 24 deletions(-) diff --git a/src/main/scala/com/twitter/scalding/GroupBuilder.scala b/src/main/scala/com/twitter/scalding/GroupBuilder.scala index 1ec2290d0b..2cf166527d 100644 --- a/src/main/scala/com/twitter/scalding/GroupBuilder.scala +++ b/src/main/scala/com/twitter/scalding/GroupBuilder.scala @@ -27,8 +27,6 @@ import scala.collection.JavaConverters._ import scala.annotation.tailrec import scala.math.Ordering -import java.lang.IllegalArgumentException - // This controls the sequence of reductions that happen inside a // particular grouping operation. Not all elements can be combined, // for instance, a scanLeft/foldLeft generally requires a sorting diff --git a/src/main/scala/com/twitter/scalding/JoinAlgorithms.scala b/src/main/scala/com/twitter/scalding/JoinAlgorithms.scala index 198a66af67..5097d1897b 100644 --- a/src/main/scala/com/twitter/scalding/JoinAlgorithms.scala +++ b/src/main/scala/com/twitter/scalding/JoinAlgorithms.scala @@ -19,7 +19,7 @@ import cascading.tap._ import cascading.scheme._ import cascading.pipe._ import cascading.pipe.assembly._ -import cascading.pipe.joiner.{InnerJoin => CInnerJoin, LeftJoin => CLeftJoin} +import cascading.pipe.joiner._ import cascading.flow._ import cascading.operation._ import cascading.operation.aggregator._ @@ -38,6 +38,16 @@ trait JoinAlgorithms { def pipe : Pipe + /** + * This method is used internally to implement all joins. + * You can use this directly if you want to implement something like a star join, + * e.g., when joining a single pipe to multiple other pipes. Make sure that you call this method + * on the larger pipe to make the grouping as efficient as possible. + * + * If you are only joining two pipes, then you are better off + * using joinWithSmaller/joinWithLarger/joinWithTiny/leftJoinWithTiny. + * + */ def coGroupBy(f : Fields, j : JoinMode = InnerJoinMode)(builder : CoGroupBuilder => GroupBuilder) : Pipe = { builder(new CoGroupBuilder(f, j)).schedule(pipe.getName, pipe) } @@ -85,6 +95,16 @@ trait JoinAlgorithms { (renamedPipe, newJoinKeys, temp) } + def joinerToJoinModes(j : Joiner) = { + j match { + case i : InnerJoin => (InnerJoinMode, InnerJoinMode) + case l : LeftJoin => (InnerJoinMode, OuterJoinMode) + case r : RightJoin => (OuterJoinMode, InnerJoinMode) + case o : OuterJoin => (OuterJoinMode, OuterJoinMode) + case _ => throw new InvalidJoinModeException("cannot convert joiner to joiner modes") + } + } + /** * joins the first set of keys in the first pipe to the second set of keys in the second pipe. * All keys must be unique UNLESS it is an inner join, then duplicated join keys are allowed, but @@ -96,8 +116,9 @@ trait JoinAlgorithms { * joinWithSmaller(('other1, 'other2)->('this1, 'this2), pipe, new FancyJoin) * } */ - def joinWithSmaller(fs :(Fields,Fields), that : Pipe, joiners : (JoinMode, JoinMode) = (InnerJoinMode, InnerJoinMode), reducers : Int = -1) = { + def joinWithSmaller(fs :(Fields,Fields), that : Pipe, joiner : Joiner = new InnerJoin, reducers : Int = -1) = { // If we are not doing an inner join, the join fields must be disjoint: + val joiners = joinerToJoinModes(joiner) val intersection = asSet(fs._1).intersect(asSet(fs._2)) if (intersection.size == 0) { // Common case: no intersection in names: just CoGroup, which duplicates the grouping fields: @@ -124,17 +145,17 @@ trait JoinAlgorithms { } } - def joinWithLarger(fs : (Fields, Fields), that : Pipe, joiners : (JoinMode, JoinMode) = (InnerJoinMode,InnerJoinMode), reducers : Int = -1) = { - that.joinWithSmaller((fs._2, fs._1), pipe, joiners, reducers) + def joinWithLarger(fs : (Fields, Fields), that : Pipe, joiner : Joiner = new InnerJoin, reducers : Int = -1) = { + that.joinWithSmaller((fs._2, fs._1), pipe, joiner, reducers) } def leftJoinWithSmaller(fs :(Fields,Fields), that : Pipe, reducers : Int = -1) = { - joinWithSmaller(fs, that, (InnerJoinMode, OuterJoinMode), reducers) + joinWithSmaller(fs, that, new LeftJoin, reducers) } def leftJoinWithLarger(fs :(Fields,Fields), that : Pipe, reducers : Int = -1) = { //We swap the order, and turn left into right: - that.joinWithSmaller((fs._2, fs._1), pipe, (OuterJoinMode, InnerJoinMode), reducers) + that.joinWithSmaller((fs._2, fs._1), pipe, new RightJoin, reducers) } /** @@ -151,18 +172,18 @@ trait JoinAlgorithms { def joinWithTiny(fs :(Fields,Fields), that : Pipe) = { val intersection = asSet(fs._1).intersect(asSet(fs._2)) if (intersection.size == 0) { - new Join(assignName(pipe), fs._1, assignName(that), fs._2, new CInnerJoin) + new Join(assignName(pipe), fs._1, assignName(that), fs._2, new InnerJoin) } else { val (renamedThat, newJoinFields, temp) = renameCollidingFields(that, fs._2, intersection) - (new Join(assignName(pipe), fs._1, assignName(renamedThat), newJoinFields, new CInnerJoin)) + (new Join(assignName(pipe), fs._1, assignName(renamedThat), newJoinFields, new InnerJoin)) .discard(temp) } } def leftJoinWithTiny(fs :(Fields,Fields), that : Pipe) = { //Rename these pipes to avoid cascading name conflicts - new Join(assignName(pipe), fs._1, assignName(that), fs._2, new CLeftJoin) + new Join(assignName(pipe), fs._1, assignName(that), fs._2, new LeftJoin) } /* @@ -193,11 +214,11 @@ trait JoinAlgorithms { */ def blockJoinWithSmaller(fs : (Fields, Fields), otherPipe : Pipe, rightReplication : Int = 1, leftReplication : Int = 1, - joiners : (JoinMode, JoinMode) = (InnerJoinMode, InnerJoinMode), reducers : Int = -1) : Pipe = { + joiner : Joiner = new InnerJoin, reducers : Int = -1) : Pipe = { assert(rightReplication > 0, "Must specify a positive number for the right replication in block join") assert(leftReplication > 0, "Must specify a positive number for the left replication in block join") - assertValidJoinMode(joiners, leftReplication, rightReplication) + assertValidJoinMode(joiner, leftReplication, rightReplication) // These are the new dummy fields used in the skew join val leftFields = new Fields("__LEFT_I__", "__LEFT_J__") @@ -211,7 +232,7 @@ trait JoinAlgorithms { val rightJoinFields = Fields.join(fs._2, rightFields) newLeft - .joinWithSmaller((leftJoinFields, rightJoinFields), newRight, joiners, reducers) + .joinWithSmaller((leftJoinFields, rightJoinFields), newRight, joiner, reducers) .discard(leftFields) .discard(rightFields) } @@ -228,11 +249,11 @@ trait JoinAlgorithms { } } - private def assertValidJoinMode(joiners : (JoinMode, JoinMode), left : Int, right : Int) { - (joiners, left, right) match { - case ((InnerJoinMode, InnerJoinMode), _, _) => true - case ((InnerJoinMode, OuterJoinMode), 1, _) => true - case ((OuterJoinMode, InnerJoinMode), _, 1) => true + private def assertValidJoinMode(joiner : Joiner, left : Int, right : Int) { + (joiner, left, right) match { + case (i : InnerJoin, _, _) => true + case (k : LeftJoin, 1, _) => true + case (m : RightJoin, _, 1) => true case (j, l, r) => throw new InvalidJoinModeException( "you cannot use joiner " + j + " with left replication " + l + " and right replication " + r diff --git a/src/test/scala/com/twitter/scalding/BlockJoinTest.scala b/src/test/scala/com/twitter/scalding/BlockJoinTest.scala index ae54135a49..32218cbd2c 100644 --- a/src/test/scala/com/twitter/scalding/BlockJoinTest.scala +++ b/src/test/scala/com/twitter/scalding/BlockJoinTest.scala @@ -12,16 +12,16 @@ class InnerProductJob(args : Args) extends Job(args) { val l = args.getOrElse("left", "1").toInt val r = args.getOrElse("right", "1").toInt val j = args.getOrElse("joiner", "i") match { - case "i" => (InnerJoinMode, InnerJoinMode) - case "l" => (InnerJoinMode, OuterJoinMode) - case "r" => (OuterJoinMode, InnerJoinMode) - case "o" => (OuterJoinMode, OuterJoinMode) + case "i" => new InnerJoin + case "l" => new LeftJoin + case "r" => new RightJoin + case "o" => new OuterJoin } val in0 = Tsv("input0").read.mapTo((0,1,2) -> ('x1, 'y1, 's1)) { input : (Int, Int, Int) => input } val in1 = Tsv("input1").read.mapTo((0,1,2) -> ('x2, 'y2, 's2)) { input : (Int, Int, Int) => input } in0 - .blockJoinWithSmaller('y1 -> 'y2, in1, leftReplication = l, rightReplication = r, joiners = j) + .blockJoinWithSmaller('y1 -> 'y2, in1, leftReplication = l, rightReplication = r, joiner = j) .map(('s1, 's2) -> 'score) { v : (Int, Int) => v._1 * v._2 } From 7715b5c4fedc652a465bf97dd50f5826bede705e Mon Sep 17 00:00:00 2001 From: Argyris Zymnis Date: Thu, 5 Apr 2012 17:59:13 -0700 Subject: [PATCH 07/25] Some cleanups for RichPipe method calls. --- src/main/scala/com/twitter/scalding/CoGroupBuilder.scala | 5 ++++- src/main/scala/com/twitter/scalding/JoinAlgorithms.scala | 4 ++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/src/main/scala/com/twitter/scalding/CoGroupBuilder.scala b/src/main/scala/com/twitter/scalding/CoGroupBuilder.scala index 5d81b66de1..e1cc0c80de 100644 --- a/src/main/scala/com/twitter/scalding/CoGroupBuilder.scala +++ b/src/main/scala/com/twitter/scalding/CoGroupBuilder.scala @@ -35,10 +35,13 @@ class CoGroupBuilder(groupFields : Fields, joinMode : JoinMode) extends GroupBui this } + // TODO: move the automatic renaming of fields here + // and remove it from joinWithSmaller/joinWithTiny override def schedule(name : String, pipe : Pipe) : Pipe = { assert(!sortBy.isDefined, "cannot use a sortBy when doing a coGroup") + assert(!coGroups.isEmpty, "coGroupBy requires at least one other pipe to .coGroup") val fields = (groupFields :: coGroups.map{ _._1 }).toArray - val pipes = (pipe :: coGroups.map{ _._2 }).toArray + val pipes = (pipe :: coGroups.map{ _._2 }).map{ RichPipe.assignName(_) }.toArray val joinModes = (joinMode :: coGroups.map{ _._3 }).map{ _.booleanValue }.toArray val mixedJoiner = new MixedJoin(joinModes) val cg : Pipe = new CoGroup(pipes, fields, null, mixedJoiner) diff --git a/src/main/scala/com/twitter/scalding/JoinAlgorithms.scala b/src/main/scala/com/twitter/scalding/JoinAlgorithms.scala index 5097d1897b..bf6f284153 100644 --- a/src/main/scala/com/twitter/scalding/JoinAlgorithms.scala +++ b/src/main/scala/com/twitter/scalding/JoinAlgorithms.scala @@ -122,7 +122,7 @@ trait JoinAlgorithms { val intersection = asSet(fs._1).intersect(asSet(fs._2)) if (intersection.size == 0) { // Common case: no intersection in names: just CoGroup, which duplicates the grouping fields: - assignName(pipe).coGroupBy(fs._1, joiners._1) { + pipe.coGroupBy(fs._1, joiners._1) { _.coGroup(fs._2, that, joiners._2) .reducers(reducers) } @@ -134,7 +134,7 @@ trait JoinAlgorithms { * So, we rename the right hand side to temporary names, then discard them after the operation */ val (renamedThat, newJoinFields, temp) = renameCollidingFields(that, fs._2, intersection) - assignName(pipe).coGroupBy(fs._1, joiners._1) { + pipe.coGroupBy(fs._1, joiners._1) { _.coGroup(newJoinFields, renamedThat, joiners._2) .reducers(reducers) }.discard(temp) From 67fcc601d90262e4bf0b38eadb206da16de4cd55 Mon Sep 17 00:00:00 2001 From: Sam Ritchie Date: Wed, 11 Apr 2012 20:49:34 -0700 Subject: [PATCH 08/25] upgrade KryoHadoopSerialization to Kryo v2. --- build.sbt | 2 +- .../scalding/KryoHadoopSerialization.scala | 65 +++++++++---------- 2 files changed, 31 insertions(+), 36 deletions(-) diff --git a/build.sbt b/build.sbt index bd6b25a43e..67a56337f5 100644 --- a/build.sbt +++ b/build.sbt @@ -16,7 +16,7 @@ libraryDependencies += "cascading" % "cascading-local" % "2.0.0-wip-238" libraryDependencies += "cascading" % "cascading-hadoop" % "2.0.0-wip-238" -libraryDependencies += "cascading.kryo" % "cascading.kryo" % "0.2.1" +libraryDependencies += "cascading.kryo" % "cascading.kryo" % "0.3.0-SNAPSHOT" libraryDependencies += "com.twitter" % "meat-locker" % "0.1.6" diff --git a/src/main/scala/com/twitter/scalding/KryoHadoopSerialization.scala b/src/main/scala/com/twitter/scalding/KryoHadoopSerialization.scala index 01f3938256..718d6a735e 100644 --- a/src/main/scala/com/twitter/scalding/KryoHadoopSerialization.scala +++ b/src/main/scala/com/twitter/scalding/KryoHadoopSerialization.scala @@ -22,13 +22,13 @@ import java.nio.ByteBuffer import org.apache.hadoop.io.serializer.{Serialization, Deserializer, Serializer, WritableSerialization} +import com.esotericsoftware.kryo.Kryo import com.esotericsoftware.kryo.{Serializer => KSerializer} -import com.esotericsoftware.kryo.serialize.DateSerializer +import com.esotericsoftware.kryo.io.{Input, Output} import cascading.kryo.KryoSerialization; import cascading.tuple.hadoop.BufferedInputStream import cascading.tuple.hadoop.TupleSerialization -import cascading.kryo.Kryo import scala.annotation.tailrec @@ -59,13 +59,12 @@ class KryoHadoopSerialization extends KryoSerialization { * but class scala.collection.immutable.$colon$colon for most concrete instances * This deals with the case of containers holding lists, even if klass is not directly one. */ - val listSer = new ListSerializer(newK) + val listSer = new ListSerializer() newK.register(List(1).getClass, listSer) //Make sure to register the Nil singleton, different from List(1).getClass newK.register(Nil.getClass, listSer) - newK.register(classOf[java.util.Date], new DateSerializer) - newK.register(classOf[RichDate], new RichDateSerializer(newK)) - newK.register(classOf[DateRange], new DateRangeSerializer(newK)) + newK.register(classOf[RichDate], new RichDateSerializer()) + newK.register(classOf[DateRange], new DateRangeSerializer()) //Add commonly used types with Fields serializer: registeredTypes.foreach { cls => newK.register(cls) } @@ -109,18 +108,18 @@ class KryoHadoopSerialization extends KryoSerialization { // Singletons are easy, you just return the singleton and don't read: // It's important you actually do this, or Kryo will generate Nil != Nil, or None != None class SingletonSerializer(obj: AnyRef) extends KSerializer { - override def writeObjectData(buf: ByteBuffer, obj: AnyRef) {} - override def readObjectData[T](buf: ByteBuffer, cls: Class[T]): T = obj.asInstanceOf[T] + def write(kser: Kryo, out: Output, obj: AnyRef) {} + def read(kser: Kryo, in: Input, cls: Class): AnyRef = obj } // Lists cause stack overflows for Kryo because they are cons cells. -class ListSerializer(kser : Kryo) extends KSerializer { - override def writeObjectData(buf: ByteBuffer, obj: AnyRef) { +class ListSerializer() extends KSerializer { + def write(kser: Kryo, out: Output, obj: AnyRef) { //Write the size: val list = obj.asInstanceOf[List[AnyRef]] - kser.writeObjectData(buf, new java.lang.Integer(list.size)) + out.writeInt(list.size, true);) /* - * An excellent question arrises at this point: + * An excellent question arises at this point: * How do we deal with List[List[T]]? * Since by the time this method is called, the ListSerializer has * already been registered, this iterative method will be used on @@ -128,11 +127,12 @@ class ListSerializer(kser : Kryo) extends KSerializer { * The only risk is List[List[List[List[..... * But anyone who tries that gets what they deserve */ - list.foreach { t => kser.writeClassAndObject(buf, t) } + list.foreach { t => kser.writeClassAndObject(out, t) } } - override def readObjectData[T](buf: ByteBuffer, cls: Class[T]) : T = { - val size = kser.readObjectData(buf, classOf[java.lang.Integer]).intValue +def read(kser: Kryo, in: Input, cls: Class) : AnyRef = { + val size = in.readInt(true); + //Produce the reversed list: if (size == 0) { /* @@ -143,9 +143,9 @@ class ListSerializer(kser : Kryo) extends KSerializer { } else { (0 until size).foldLeft(List[AnyRef]()) { (l, i) => - val iT = kser.readClassAndObject(buf) + val iT = kser.readClassAndObject(in) iT :: l - }.reverse.asInstanceOf[T] + }.reverse } } } @@ -153,28 +153,23 @@ class ListSerializer(kser : Kryo) extends KSerializer { /*** * Below are some serializers for objects in the scalding project. */ -class RichDateSerializer(kser : Kryo) extends KSerializer { - override def writeObjectData(buf: ByteBuffer, obj: AnyRef) { - kser.writeObjectData(buf, - new java.lang.Long(obj.asInstanceOf[RichDate].value.getTime)) +class RichDateSerializer() extends KSerializer[RichDate] { + def write(kser: Kryo, out: Output, date: RichDate) { + out.writeLong(date.value.getTime, true); } - override def readObjectData[T](buf: ByteBuffer, cls: Class[T]): T = { - RichDate(kser.readObjectData[java.lang.Long](buf,classOf[java.lang.Long]).longValue) - .asInstanceOf[T] + + def read(kser: Kryo, in: Input, cls: Class[RichDate]): RichDate = { + RichDate(in.readLong(in, true)) } } -class DateRangeSerializer(kser : Kryo) extends KSerializer { - override def writeObjectData(buf: ByteBuffer, obj: AnyRef) { - kser.writeObjectData(buf, - new java.lang.Long(obj.asInstanceOf[DateRange].start.value.getTime)) - kser.writeObjectData(buf, - new java.lang.Long(obj.asInstanceOf[DateRange].end.value.getTime)) +class DateRangeSerializer() extends KSerializer[DateRange] { + def write(kser: Kryo, out: Output, range: DateRange) { + out.writeLong(range.start.value.getTime, true); + out.writeLong(range.end.value.getTime, true); } - override def readObjectData[T](buf: ByteBuffer, cls: Class[T]): T = { - val start = kser.readObjectData[java.lang.Long](buf,classOf[java.lang.Long]) - val end = kser.readObjectData[java.lang.Long](buf,classOf[java.lang.Long]) - DateRange(RichDate(start.longValue), RichDate(end.longValue)) - .asInstanceOf[T] + + def read(kser: Kryo, in: Input, cls: Class[DateRange]): DateRange = { + DateRange(RichDate(in.readLong(true)), RichDate(in.readLong(true))); } } From 7f74b27dc78f0ee5a98e3683b5d38146dc67519c Mon Sep 17 00:00:00 2001 From: Oscar Boykin Date: Wed, 11 Apr 2012 21:50:11 -0700 Subject: [PATCH 09/25] Sets the flow name --- src/main/scala/com/twitter/scalding/Job.scala | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/main/scala/com/twitter/scalding/Job.scala b/src/main/scala/com/twitter/scalding/Job.scala index 2ff4341705..d5e3425d29 100644 --- a/src/main/scala/com/twitter/scalding/Job.scala +++ b/src/main/scala/com/twitter/scalding/Job.scala @@ -55,9 +55,17 @@ class Job(val args : Args) extends TupleConversions with FieldConversions { implicit def iterToRichPipe[T](iter : Iterable[T])(implicit set: TupleSetter[T]) : RichPipe = { RichPipe(iterToPipe(iter)(set)) } + + // Override this if you want change how the mapred.job.name is written in Hadoop + def name : String = getClass.getCanonicalName + //This is the FlowDef used by all Sources this job creates @transient - implicit val flowDef = new FlowDef + implicit val flowDef = { + val fd = new FlowDef + fd.setName(name) + fd + } // Use reflection to copy this job: def clone(nextargs : Args) : Job = { From 00cf7b50d5fb836170df3cf7795f01113d4c7b6a Mon Sep 17 00:00:00 2001 From: Sam Ritchie Date: Wed, 11 Apr 2012 22:01:58 -0700 Subject: [PATCH 10/25] I fired up the compiler and fixed up a few issues. (Weak, I know.) --- .../scalding/KryoHadoopSerialization.scala | 21 +++++++++---------- 1 file changed, 10 insertions(+), 11 deletions(-) diff --git a/src/main/scala/com/twitter/scalding/KryoHadoopSerialization.scala b/src/main/scala/com/twitter/scalding/KryoHadoopSerialization.scala index 718d6a735e..98e0bc16ab 100644 --- a/src/main/scala/com/twitter/scalding/KryoHadoopSerialization.scala +++ b/src/main/scala/com/twitter/scalding/KryoHadoopSerialization.scala @@ -107,17 +107,16 @@ class KryoHadoopSerialization extends KryoSerialization { // Singletons are easy, you just return the singleton and don't read: // It's important you actually do this, or Kryo will generate Nil != Nil, or None != None -class SingletonSerializer(obj: AnyRef) extends KSerializer { - def write(kser: Kryo, out: Output, obj: AnyRef) {} - def read(kser: Kryo, in: Input, cls: Class): AnyRef = obj +class SingletonSerializer[T](obj: T) extends KSerializer[T] { + def write(kser: Kryo, out: Output, obj: T) {} + def read(kser: Kryo, in: Input, cls: Class[T]): T = obj } // Lists cause stack overflows for Kryo because they are cons cells. -class ListSerializer() extends KSerializer { - def write(kser: Kryo, out: Output, obj: AnyRef) { +class ListSerializer[T] extends KSerializer[List[T]] { + def write(kser: Kryo, out: Output, list: List[T]) { //Write the size: - val list = obj.asInstanceOf[List[AnyRef]] - out.writeInt(list.size, true);) + out.writeInt(list.size, true) /* * An excellent question arises at this point: * How do we deal with List[List[T]]? @@ -130,7 +129,7 @@ class ListSerializer() extends KSerializer { list.foreach { t => kser.writeClassAndObject(out, t) } } -def read(kser: Kryo, in: Input, cls: Class) : AnyRef = { +def read(kser: Kryo, in: Input, cls: Class[List[T]]) : List[T] = { val size = in.readInt(true); //Produce the reversed list: @@ -139,13 +138,13 @@ def read(kser: Kryo, in: Input, cls: Class) : AnyRef = { * this is only here at compile time. The type T is erased, but the * compiler verifies that we are intending to return a type T here. */ - Nil.asInstanceOf[T] + Nil.asInstanceOf[List[T]] } else { (0 until size).foldLeft(List[AnyRef]()) { (l, i) => val iT = kser.readClassAndObject(in) iT :: l - }.reverse + }.reverse.asInstanceOf[List[T]] } } } @@ -159,7 +158,7 @@ class RichDateSerializer() extends KSerializer[RichDate] { } def read(kser: Kryo, in: Input, cls: Class[RichDate]): RichDate = { - RichDate(in.readLong(in, true)) + RichDate(in.readLong(true)) } } From 20db41f940fc425590a7db6a5b0239ea76956c61 Mon Sep 17 00:00:00 2001 From: Sam Ritchie Date: Wed, 11 Apr 2012 22:09:54 -0700 Subject: [PATCH 11/25] Removed type parametrization on ListSerializer. --- .../twitter/scalding/KryoHadoopSerialization.scala | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/src/main/scala/com/twitter/scalding/KryoHadoopSerialization.scala b/src/main/scala/com/twitter/scalding/KryoHadoopSerialization.scala index 98e0bc16ab..d39641008e 100644 --- a/src/main/scala/com/twitter/scalding/KryoHadoopSerialization.scala +++ b/src/main/scala/com/twitter/scalding/KryoHadoopSerialization.scala @@ -113,8 +113,9 @@ class SingletonSerializer[T](obj: T) extends KSerializer[T] { } // Lists cause stack overflows for Kryo because they are cons cells. -class ListSerializer[T] extends KSerializer[List[T]] { - def write(kser: Kryo, out: Output, list: List[T]) { +class ListSerializer extends KSerializer[AnyRef] { + def write(kser: Kryo, out: Output, obj: AnyRef) { + val list = obj.asInstanceOf[List[AnyRef]] //Write the size: out.writeInt(list.size, true) /* @@ -129,7 +130,7 @@ class ListSerializer[T] extends KSerializer[List[T]] { list.foreach { t => kser.writeClassAndObject(out, t) } } -def read(kser: Kryo, in: Input, cls: Class[List[T]]) : List[T] = { +def read(kser: Kryo, in: Input, cls: Class[AnyRef]) : AnyRef = { val size = in.readInt(true); //Produce the reversed list: @@ -138,13 +139,13 @@ def read(kser: Kryo, in: Input, cls: Class[List[T]]) : List[T] = { * this is only here at compile time. The type T is erased, but the * compiler verifies that we are intending to return a type T here. */ - Nil.asInstanceOf[List[T]] + Nil } else { (0 until size).foldLeft(List[AnyRef]()) { (l, i) => val iT = kser.readClassAndObject(in) iT :: l - }.reverse.asInstanceOf[List[T]] + }.reverse } } } From ca91b917ddb7d1cb1476365866dad8c773f9f874 Mon Sep 17 00:00:00 2001 From: Sam Ritchie Date: Wed, 11 Apr 2012 22:15:11 -0700 Subject: [PATCH 12/25] Released cascading.kryo 0.3.0 proper to Conjars. --- README.md | 1 + build.sbt | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index b4e904873a..dbd8ce6bfc 100644 --- a/README.md +++ b/README.md @@ -74,6 +74,7 @@ Thanks for assistance and contributions: * Dmitriy Ryaboy * Dong Wang * Edwin Chen +* Sam Ritchie ## License Copyright 2012 Twitter, Inc. diff --git a/build.sbt b/build.sbt index 67a56337f5..611697c041 100644 --- a/build.sbt +++ b/build.sbt @@ -16,7 +16,7 @@ libraryDependencies += "cascading" % "cascading-local" % "2.0.0-wip-238" libraryDependencies += "cascading" % "cascading-hadoop" % "2.0.0-wip-238" -libraryDependencies += "cascading.kryo" % "cascading.kryo" % "0.3.0-SNAPSHOT" +libraryDependencies += "cascading.kryo" % "cascading.kryo" % "0.3.0" libraryDependencies += "com.twitter" % "meat-locker" % "0.1.6" From f05fc24a1f8de4bdf82732f9f5c71700f216515a Mon Sep 17 00:00:00 2001 From: Oscar Boykin Date: Thu, 12 Apr 2012 13:48:44 -0700 Subject: [PATCH 13/25] Updates meat-locker and maple --- build.sbt | 4 +++- src/main/scala/com/twitter/scalding/IterableSource.scala | 2 +- src/main/scala/com/twitter/scalding/Job.scala | 7 +++++-- src/main/scala/com/twitter/scalding/Mode.scala | 1 + src/main/scala/com/twitter/scalding/Source.scala | 4 ++-- 5 files changed, 12 insertions(+), 6 deletions(-) diff --git a/build.sbt b/build.sbt index 611697c041..141a37e75b 100644 --- a/build.sbt +++ b/build.sbt @@ -18,7 +18,9 @@ libraryDependencies += "cascading" % "cascading-hadoop" % "2.0.0-wip-238" libraryDependencies += "cascading.kryo" % "cascading.kryo" % "0.3.0" -libraryDependencies += "com.twitter" % "meat-locker" % "0.1.6" +libraryDependencies += "com.twitter" % "meat-locker" % "0.2.0" + +libraryDependencies += "com.twitter" % "maple" % "0.1.1" libraryDependencies += "commons-lang" % "commons-lang" % "2.4" diff --git a/src/main/scala/com/twitter/scalding/IterableSource.scala b/src/main/scala/com/twitter/scalding/IterableSource.scala index 719e66ddd8..dc6478e68f 100644 --- a/src/main/scala/com/twitter/scalding/IterableSource.scala +++ b/src/main/scala/com/twitter/scalding/IterableSource.scala @@ -15,7 +15,7 @@ limitations under the License. */ package com.twitter.scalding -import com.twitter.meatlocker.tap.MemorySourceTap +import com.twitter.maple.tap.MemorySourceTap import cascading.flow.FlowProcess import cascading.scheme.local.{TextDelimited => CLTextDelimited} diff --git a/src/main/scala/com/twitter/scalding/Job.scala b/src/main/scala/com/twitter/scalding/Job.scala index 2ff4341705..8e47274d58 100644 --- a/src/main/scala/com/twitter/scalding/Job.scala +++ b/src/main/scala/com/twitter/scalding/Job.scala @@ -105,8 +105,11 @@ class Job(val args : Args) extends TupleConversions with FieldConversions { flow.complete flow.getFlowStats.isSuccessful } - // Add any serializations you need to deal with here: - def ioSerializations = List[String]() + // Add any serializations you need to deal with here (after these) + def ioSerializations = List[String]( + "org.apache.hadoop.io.serializer.WritableSerialization", + "cascading.tuple.hadoop.TupleSerialization" + ) // Override this if you want to customize comparisons/hashing for your job def defaultComparator : Option[String] = { Some("com.twitter.scalding.IntegralComparator") diff --git a/src/main/scala/com/twitter/scalding/Mode.scala b/src/main/scala/com/twitter/scalding/Mode.scala index e4fc974c2e..7969e0c577 100644 --- a/src/main/scala/com/twitter/scalding/Mode.scala +++ b/src/main/scala/com/twitter/scalding/Mode.scala @@ -113,6 +113,7 @@ trait HadoopMode extends Mode { new HadoopFlowConnector(unionValues(jobConf, props)) } + // TODO unlike newFlowConnector, this does not look at the Job.config override def openForRead(tap : Tap[_,_,_,_]) = { val htap = tap.asInstanceOf[Tap[HadoopFlowProcess,_,_,_]] val fp = new HadoopFlowProcess(new JobConf(jobConf)) diff --git a/src/main/scala/com/twitter/scalding/Source.scala b/src/main/scala/com/twitter/scalding/Source.scala index a28bea557a..7491f8be32 100644 --- a/src/main/scala/com/twitter/scalding/Source.scala +++ b/src/main/scala/com/twitter/scalding/Source.scala @@ -15,7 +15,7 @@ limitations under the License. */ package com.twitter.scalding -import com.twitter.meatlocker.tap.MemorySourceTap +import com.twitter.maple.tap.MemorySourceTap import java.io.File import java.util.TimeZone @@ -137,7 +137,7 @@ abstract class Source extends java.io.Serializable { case Read => { val buffer = buffers(this) val fields = hdfsScheme.getSourceFields - new MemorySourceTap(buffer.toList.asJava, fields) + (new MemorySourceTap(buffer.toList.asJava, fields)).asInstanceOf[RawTap] } case Write => { val path = hdfsTest.getWritePathFor(this) From ba2571c32811442440af6a3f3892c4d89ad1fa97 Mon Sep 17 00:00:00 2001 From: Oscar Boykin Date: Fri, 13 Apr 2012 12:24:54 -0700 Subject: [PATCH 14/25] Adds Abstract Algebra --- scripts/ntuple_generators.rb | 173 +++ .../com/twitter/scalding/GroupBuilder.scala | 46 + .../scala/com/twitter/scalding/Source.scala | 2 +- .../mathematics/AbstractAlgebra.scala | 1109 +++++++++++++++++ .../scalding/AlgebraicReductionsTest.scala | 33 + 5 files changed, 1362 insertions(+), 1 deletion(-) create mode 100644 scripts/ntuple_generators.rb create mode 100644 src/main/scala/com/twitter/scalding/mathematics/AbstractAlgebra.scala create mode 100644 src/test/scala/com/twitter/scalding/AlgebraicReductionsTest.scala diff --git a/scripts/ntuple_generators.rb b/scripts/ntuple_generators.rb new file mode 100644 index 0000000000..1ee4d1a3f4 --- /dev/null +++ b/scripts/ntuple_generators.rb @@ -0,0 +1,173 @@ +# @author Edwin Chen (@echen) +# Automatically write product monoid, product group, and product ring +# classes for tuples up to size 22. + +# The tuple sizes we want. +TUPLE_SIZES = (3..22).to_a + +# Each element in a product tuple is of a certain type. +# This provides an alphabet to draw types from. +TYPE_SYMBOLS = ("A".."Z").to_a + +# This returns the comment for each product monoid/group/ring definition. +# n is the size of the product. +# algebraic_structure is "monoid", "group", "ring", etc. +# +# Example return: +# "/** +# * Combine two monoids into a product monoid +# */" +def get_comment(n, algebraic_structure) + ret = < CTuple} +import com.twitter.scalding.mathematics.{Monoid, Ring} + import scala.collection.JavaConverters._ import scala.annotation.tailrec import scala.math.Ordering @@ -387,6 +389,50 @@ class GroupBuilder(val groupFields : Fields) extends FieldConversions reduce(fieldDef -> fieldDef)(fn)(setter,conv) } + // Abstract algebra reductions (plus, times, dot): + + /** use Monoid.plus to compute a sum. Not called sum to avoid conflicting with standard sum + * Your Monoid[T] should be associated and commutative, else this doesn't make sense + */ + def plus[T](fd : (Fields,Fields)) + (implicit monoid : Monoid[T], tconv : TupleConverter[T], tset : TupleSetter[T]) : GroupBuilder = { + // We reverse the order because the left is the old value in reduce, and for list concat + // we are much better off concatenating into the bigger list + reduce[T](fd)({ (left, right) => monoid.plus(right, left) })(tset, tconv) + } + + // The same as plus(fs -> fs) + def plus[T](fs : Symbol*) + (implicit monoid : Monoid[T], tconv : TupleConverter[T], tset : TupleSetter[T]) : GroupBuilder = { + plus[T](fs -> fs)(monoid,tconv,tset) + } + + // Returns the product of all the items in this grouping + def times[T](fd : (Fields,Fields)) + (implicit ring : Ring[T], tconv : TupleConverter[T], tset : TupleSetter[T]) : GroupBuilder = { + // We reverse the order because the left is the old value in reduce, and for list concat + // we are much better off concatenating into the bigger list + reduce[T](fd)({ (left, right) => ring.times(right, left) })(tset, tconv) + } + + // The same as times(fs -> fs) + def times[T](fs : Symbol*) + (implicit ring : Ring[T], tconv : TupleConverter[T], tset : TupleSetter[T]) : GroupBuilder = { + times[T](fs -> fs)(ring,tconv,tset) + } + + // First do "times" on each pair, then "plus" them all together. + // Example: groupBy('x) { _.dot('y,'z, 'ydotz) } + def dot[T](left : Fields, right : Fields, result : Fields) + (implicit ttconv : TupleConverter[Tuple2[T,T]], ring : Ring[T], + tconv : TupleConverter[T], tset : TupleSetter[T]) : GroupBuilder = { + mapReduceMap[(T,T),T,T](Fields.merge(left, right) -> result) { init : (T,T) => + ring.times(init._1, init._2) + } { (left : T, right: T) => + ring.plus(left, right) + } { result => result } + } + def reverse : GroupBuilder = { assert(reds.isEmpty, "Cannot sort when reducing") assert(!isReversed, "Reverse called a second time! Only one allowed") diff --git a/src/main/scala/com/twitter/scalding/Source.scala b/src/main/scala/com/twitter/scalding/Source.scala index a28bea557a..34682564eb 100644 --- a/src/main/scala/com/twitter/scalding/Source.scala +++ b/src/main/scala/com/twitter/scalding/Source.scala @@ -137,7 +137,7 @@ abstract class Source extends java.io.Serializable { case Read => { val buffer = buffers(this) val fields = hdfsScheme.getSourceFields - new MemorySourceTap(buffer.toList.asJava, fields) + (new MemorySourceTap(buffer.toList.asJava, fields)).asInstanceOf[RawTap] } case Write => { val path = hdfsTest.getWritePathFor(this) diff --git a/src/main/scala/com/twitter/scalding/mathematics/AbstractAlgebra.scala b/src/main/scala/com/twitter/scalding/mathematics/AbstractAlgebra.scala new file mode 100644 index 0000000000..a16341c861 --- /dev/null +++ b/src/main/scala/com/twitter/scalding/mathematics/AbstractAlgebra.scala @@ -0,0 +1,1109 @@ +/* +Copyright 2012 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.mathematics + +/** + * Monoid (take a deep breath, and relax about the weird name): + * This is a class that has an additive identify (called zero), and plus method that is + * associative: a+(b+c) = (a+b)+c and a+0=a, 0+a=a + * + * Group: this is a monoid that also has subtraction (and negation): + * So, you can do (a-b), or -a (which is equal to 0 - a). + * + * Ring: Group + multiplication (see: http://en.wikipedia.org/wiki/Ring_%28mathematics%29) + * and the three elements it defines: + * - additive identity aka zero + * - addition + * - multiplication + * + * The ring is to be passed as an argument to Matrix innerproduct and + * provides the definitions for addition and multiplication + * + * Field: Ring + division. It is a generalization of Ring and adds support for inversion and + * multiplicative identity. + */ + +trait Monoid[@specialized(Int,Long,Float,Double) T] extends java.io.Serializable { + def zero : T //additive identity + def assertNotZero(v : T) { + if(zero == v) { + throw new java.lang.IllegalArgumentException("argument should not be zero") + } + } + + def isNonZero(v : T) = (v != zero) + + def nonZeroOption(v : T): Option[T] = { + if (isNonZero(v)) { + Some(v) + } + else { + None + } + } + def plus(l : T, r : T) : T +} + +trait Group[@specialized(Int,Long,Float,Double) T] extends Monoid[T] { + // must override negate or minus (or both) + def negate(v : T) : T = minus(zero, v) + def minus(l : T, r : T) : T = plus(l, negate(r)) +} + +trait Ring[@specialized(Int,Long,Float,Double) T] extends Group[T] { + def one : T // Multiplicative identity + def times(l : T, r : T) : T +} + +trait Field[@specialized(Int,Long,Float,Double) T] extends Ring[T] { + // default implementation uses div YOU MUST OVERRIDE ONE OF THESE + def inverse(v : T) : T = { + assertNotZero(v) + div(one, v) + } + // default implementation uses inverse: + def div(l : T, r : T) : T = { + assertNotZero(r) + times(l, inverse(r)) + } +} + +/** List concatenation monoid. + * plus means concatenation, zero is empty list + */ +class ListMonoid[T] extends Monoid[List[T]] { + override def zero = List[T]() + override def plus(left : List[T], right : List[T]) = left ++ right +} + +/** Set union monoid. + * plus means union, zero is empty set + */ +class SetMonoid[T] extends Monoid[Set[T]] { + override def zero = Set[T]() + override def plus(left : Set[T], right : Set[T]) = left ++ right +} + +/** You can think of this as a Sparse vector monoid + */ +class MapMonoid[K,V](implicit monoid : Monoid[V]) extends Monoid[Map[K,V]] { + override def zero = Map[K,V]() + override def plus(left : Map[K,V], right : Map[K,V]) = { + (left.keys.toSet ++ right.keys.toSet).foldLeft(Map[K,V]()) { (oldMap, newK) => + val leftV = left.getOrElse(newK, monoid.zero) + val rightV = right.getOrElse(newK, monoid.zero) + val newValue = monoid.plus(leftV, rightV) + if (monoid.isNonZero(newValue)) { + oldMap + (newK -> newValue) + } + else { + // Keep it sparse + oldMap - newK + } + } + } + // This is not part of the typeclass, but it is useful: + def sumValues(elem : Map[K,V]) : V = elem.values.reduceLeft { monoid.plus(_,_) } +} + +/** You can think of this as a Sparse vector group + */ +class MapGroup[K,V](implicit vgrp : Group[V]) extends MapMonoid[K,V]()(vgrp) with Group[Map[K,V]] { + override def negate(kv : Map[K,V]) = kv.mapValues { v => vgrp.negate(v) } +} + +/** You can think of this as a Sparse vector ring + */ +class MapRing[K,V](implicit ring : Ring[V]) extends MapGroup[K,V]()(ring) with Ring[Map[K,V]] { + // It is possible to implement this, but we need a special "identity map" which we + // deal with as if it were map with all possible keys (.get(x) == ring.one for all x). + // Then we have to manage the delta from this map as we add elements. That said, it + // is not actually needed in matrix multiplication, so we are punting on it for now. + override def one = error("multiplicative identity for Map unimplemented") + override def times(left : Map[K,V], right : Map[K,V]) : Map[K,V] = { + (left.keys.toSet & right.keys.toSet).foldLeft(Map[K,V]()) { (oldMap, newK) => + // The key is on both sides, so it is safe to get it out: + val leftV = left(newK) + val rightV = right(newK) + val newValue = ring.times(leftV, rightV) + if (ring.isNonZero(newValue)) { + oldMap + (newK -> newValue) + } + else { + // Keep it sparse + oldMap - newK + } + } + } + // This is not part of the typeclass, but it is useful: + def productValues(elem : Map[K,V]) : V = elem.values.reduceLeft { ring.times(_,_) } + def dot(left : Map[K,V], right : Map[K,V]) : V = sumValues(times(left, right)) +} + +object IntRing extends Ring[Int] { + override def zero = 0 + override def one = 1 + override def negate(v : Int) = -v + override def plus(l : Int, r : Int) = l + r + override def minus(l : Int, r : Int) = l - r + override def times(l : Int, r : Int) = l * r +} + +object LongRing extends Ring[Long] { + override def zero = 0L + override def one = 1L + override def negate(v : Long) = -v + override def plus(l : Long, r : Long) = l + r + override def minus(l : Long, r : Long) = l - r + override def times(l : Long, r : Long) = l * r +} + +object FloatField extends Field[Float] { + override def one = 1.0f + override def zero = 0.0f + override def negate(v : Float) = -v + override def plus(l : Float, r : Float) = l + r + override def minus(l : Float, r : Float) = l - r + override def times(l : Float, r : Float) = l * r + override def div(l : Float, r : Float) = { + assertNotZero(r) + l / r + } +} + +object DoubleField extends Field[Double] { + override def one = 1.0 + override def zero = 0.0 + override def negate(v : Double) = -v + override def plus(l : Double, r : Double) = l + r + override def minus(l : Double, r : Double) = l - r + override def times(l : Double, r : Double) = l * r + override def div(l : Double, r : Double) = { + assertNotZero(r) + l / r + } +} + +object BooleanField extends Field[Boolean] { + override def one = true + override def zero = false + override def negate(v : Boolean) = v + override def plus(l : Boolean, r : Boolean) = l ^ r + override def minus(l : Boolean, r : Boolean) = l ^ r + override def times(l : Boolean, r : Boolean) = l && r + override def inverse(l : Boolean) = { + assertNotZero(l) + true + } + override def div(l : Boolean, r : Boolean) = { + assertNotZero(r) + l + } +} + +/** +* Combine two monoids into a product monoid +*/ +class Tuple2Monoid[T,U](implicit tmonoid : Monoid[T], umonoid : Monoid[U]) extends Monoid[(T,U)] { + override def zero = (tmonoid.zero, umonoid.zero) + override def plus(l : (T,U), r : (T,U)) = (tmonoid.plus(l._1,r._1), umonoid.plus(l._2, r._2)) +} + +/** +* Combine two groups into a product group +*/ +class Tuple2Group[T,U](implicit tgroup : Group[T], ugroup : Group[U]) extends Group[(T,U)] { + override def zero = (tgroup.zero, ugroup.zero) + override def negate(v : (T,U)) = (tgroup.negate(v._1), ugroup.negate(v._2)) + override def plus(l : (T,U), r : (T,U)) = (tgroup.plus(l._1,r._1), ugroup.plus(l._2, r._2)) + override def minus(l : (T,U), r : (T,U)) = (tgroup.minus(l._1,r._1), ugroup.minus(l._2, r._2)) +} + +/** +* Combine two rings into a product ring +*/ +class Tuple2Ring[T,U](implicit tring : Ring[T], uring : Ring[U]) extends Ring[(T,U)] { + override def zero = (tring.zero, uring.zero) + override def one = (tring.one, uring.one) + override def negate(v : (T,U)) = (tring.negate(v._1), uring.negate(v._2)) + override def plus(l : (T,U), r : (T,U)) = (tring.plus(l._1,r._1), uring.plus(l._2, r._2)) + override def minus(l : (T,U), r : (T,U)) = (tring.minus(l._1,r._1), uring.minus(l._2, r._2)) + override def times(l : (T,U), r : (T,U)) = (tring.times(l._1,r._1), uring.times(l._2, r._2)) +} + +/** +* Combine 3 monoids into a product monoid +*/ +class Tuple3Monoid[A, B, C](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C]) extends Monoid[(A, B, C)] { + override def zero = (amonoid.zero, bmonoid.zero, cmonoid.zero) + override def plus(l : (A, B, C), r : (A, B, C)) = (amonoid.plus(l._1, r._1), bmonoid.plus(l._2, r._2), cmonoid.plus(l._3, r._3)) +} + +/** +* Combine 3 groups into a product group +*/ +class Tuple3Group[A, B, C](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C]) extends Group[(A, B, C)] { + override def zero = (agroup.zero, bgroup.zero, cgroup.zero) + override def negate(v : (A, B, C)) = (agroup.negate(v._1), bgroup.negate(v._2), cgroup.negate(v._3)) + override def plus(l : (A, B, C), r : (A, B, C)) = (agroup.plus(l._1, r._1), bgroup.plus(l._2, r._2), cgroup.plus(l._3, r._3)) + override def minus(l : (A, B, C), r : (A, B, C)) = (agroup.minus(l._1, r._1), bgroup.minus(l._2, r._2), cgroup.minus(l._3, r._3)) +} + +/** +* Combine 3 rings into a product ring +*/ +class Tuple3Ring[A, B, C](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C]) extends Ring[(A, B, C)] { + override def zero = (aring.zero, bring.zero, cring.zero) + override def one = (aring.one, bring.one, cring.one) + override def negate(v : (A, B, C)) = (aring.negate(v._1), bring.negate(v._2), cring.negate(v._3)) + override def plus(l : (A, B, C), r : (A, B, C)) = (aring.plus(l._1, r._1), bring.plus(l._2, r._2), cring.plus(l._3, r._3)) + override def minus(l : (A, B, C), r : (A, B, C)) = (aring.minus(l._1, r._1), bring.minus(l._2, r._2), cring.minus(l._3, r._3)) + override def times(l : (A, B, C), r : (A, B, C)) = (aring.times(l._1, r._1), bring.times(l._2, r._2), cring.times(l._3, r._3)) +} +/** +* Combine 4 monoids into a product monoid +*/ +class Tuple4Monoid[A, B, C, D](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D]) extends Monoid[(A, B, C, D)] { + override def zero = (amonoid.zero, bmonoid.zero, cmonoid.zero, dmonoid.zero) + override def plus(l : (A, B, C, D), r : (A, B, C, D)) = (amonoid.plus(l._1, r._1), bmonoid.plus(l._2, r._2), cmonoid.plus(l._3, r._3), dmonoid.plus(l._4, r._4)) +} + +/** +* Combine 4 groups into a product group +*/ +class Tuple4Group[A, B, C, D](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D]) extends Group[(A, B, C, D)] { + override def zero = (agroup.zero, bgroup.zero, cgroup.zero, dgroup.zero) + override def negate(v : (A, B, C, D)) = (agroup.negate(v._1), bgroup.negate(v._2), cgroup.negate(v._3), dgroup.negate(v._4)) + override def plus(l : (A, B, C, D), r : (A, B, C, D)) = (agroup.plus(l._1, r._1), bgroup.plus(l._2, r._2), cgroup.plus(l._3, r._3), dgroup.plus(l._4, r._4)) + override def minus(l : (A, B, C, D), r : (A, B, C, D)) = (agroup.minus(l._1, r._1), bgroup.minus(l._2, r._2), cgroup.minus(l._3, r._3), dgroup.minus(l._4, r._4)) +} + +/** +* Combine 4 rings into a product ring +*/ +class Tuple4Ring[A, B, C, D](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D]) extends Ring[(A, B, C, D)] { + override def zero = (aring.zero, bring.zero, cring.zero, dring.zero) + override def one = (aring.one, bring.one, cring.one, dring.one) + override def negate(v : (A, B, C, D)) = (aring.negate(v._1), bring.negate(v._2), cring.negate(v._3), dring.negate(v._4)) + override def plus(l : (A, B, C, D), r : (A, B, C, D)) = (aring.plus(l._1, r._1), bring.plus(l._2, r._2), cring.plus(l._3, r._3), dring.plus(l._4, r._4)) + override def minus(l : (A, B, C, D), r : (A, B, C, D)) = (aring.minus(l._1, r._1), bring.minus(l._2, r._2), cring.minus(l._3, r._3), dring.minus(l._4, r._4)) + override def times(l : (A, B, C, D), r : (A, B, C, D)) = (aring.times(l._1, r._1), bring.times(l._2, r._2), cring.times(l._3, r._3), dring.times(l._4, r._4)) +} +/** +* Combine 5 monoids into a product monoid +*/ +class Tuple5Monoid[A, B, C, D, E](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E]) extends Monoid[(A, B, C, D, E)] { + override def zero = (amonoid.zero, bmonoid.zero, cmonoid.zero, dmonoid.zero, emonoid.zero) + override def plus(l : (A, B, C, D, E), r : (A, B, C, D, E)) = (amonoid.plus(l._1, r._1), bmonoid.plus(l._2, r._2), cmonoid.plus(l._3, r._3), dmonoid.plus(l._4, r._4), emonoid.plus(l._5, r._5)) +} + +/** +* Combine 5 groups into a product group +*/ +class Tuple5Group[A, B, C, D, E](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E]) extends Group[(A, B, C, D, E)] { + override def zero = (agroup.zero, bgroup.zero, cgroup.zero, dgroup.zero, egroup.zero) + override def negate(v : (A, B, C, D, E)) = (agroup.negate(v._1), bgroup.negate(v._2), cgroup.negate(v._3), dgroup.negate(v._4), egroup.negate(v._5)) + override def plus(l : (A, B, C, D, E), r : (A, B, C, D, E)) = (agroup.plus(l._1, r._1), bgroup.plus(l._2, r._2), cgroup.plus(l._3, r._3), dgroup.plus(l._4, r._4), egroup.plus(l._5, r._5)) + override def minus(l : (A, B, C, D, E), r : (A, B, C, D, E)) = (agroup.minus(l._1, r._1), bgroup.minus(l._2, r._2), cgroup.minus(l._3, r._3), dgroup.minus(l._4, r._4), egroup.minus(l._5, r._5)) +} + +/** +* Combine 5 rings into a product ring +*/ +class Tuple5Ring[A, B, C, D, E](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E]) extends Ring[(A, B, C, D, E)] { + override def zero = (aring.zero, bring.zero, cring.zero, dring.zero, ering.zero) + override def one = (aring.one, bring.one, cring.one, dring.one, ering.one) + override def negate(v : (A, B, C, D, E)) = (aring.negate(v._1), bring.negate(v._2), cring.negate(v._3), dring.negate(v._4), ering.negate(v._5)) + override def plus(l : (A, B, C, D, E), r : (A, B, C, D, E)) = (aring.plus(l._1, r._1), bring.plus(l._2, r._2), cring.plus(l._3, r._3), dring.plus(l._4, r._4), ering.plus(l._5, r._5)) + override def minus(l : (A, B, C, D, E), r : (A, B, C, D, E)) = (aring.minus(l._1, r._1), bring.minus(l._2, r._2), cring.minus(l._3, r._3), dring.minus(l._4, r._4), ering.minus(l._5, r._5)) + override def times(l : (A, B, C, D, E), r : (A, B, C, D, E)) = (aring.times(l._1, r._1), bring.times(l._2, r._2), cring.times(l._3, r._3), dring.times(l._4, r._4), ering.times(l._5, r._5)) +} +/** +* Combine 6 monoids into a product monoid +*/ +class Tuple6Monoid[A, B, C, D, E, F](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F]) extends Monoid[(A, B, C, D, E, F)] { + override def zero = (amonoid.zero, bmonoid.zero, cmonoid.zero, dmonoid.zero, emonoid.zero, fmonoid.zero) + override def plus(l : (A, B, C, D, E, F), r : (A, B, C, D, E, F)) = (amonoid.plus(l._1, r._1), bmonoid.plus(l._2, r._2), cmonoid.plus(l._3, r._3), dmonoid.plus(l._4, r._4), emonoid.plus(l._5, r._5), fmonoid.plus(l._6, r._6)) +} + +/** +* Combine 6 groups into a product group +*/ +class Tuple6Group[A, B, C, D, E, F](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F]) extends Group[(A, B, C, D, E, F)] { + override def zero = (agroup.zero, bgroup.zero, cgroup.zero, dgroup.zero, egroup.zero, fgroup.zero) + override def negate(v : (A, B, C, D, E, F)) = (agroup.negate(v._1), bgroup.negate(v._2), cgroup.negate(v._3), dgroup.negate(v._4), egroup.negate(v._5), fgroup.negate(v._6)) + override def plus(l : (A, B, C, D, E, F), r : (A, B, C, D, E, F)) = (agroup.plus(l._1, r._1), bgroup.plus(l._2, r._2), cgroup.plus(l._3, r._3), dgroup.plus(l._4, r._4), egroup.plus(l._5, r._5), fgroup.plus(l._6, r._6)) + override def minus(l : (A, B, C, D, E, F), r : (A, B, C, D, E, F)) = (agroup.minus(l._1, r._1), bgroup.minus(l._2, r._2), cgroup.minus(l._3, r._3), dgroup.minus(l._4, r._4), egroup.minus(l._5, r._5), fgroup.minus(l._6, r._6)) +} + +/** +* Combine 6 rings into a product ring +*/ +class Tuple6Ring[A, B, C, D, E, F](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E], fring : Ring[F]) extends Ring[(A, B, C, D, E, F)] { + override def zero = (aring.zero, bring.zero, cring.zero, dring.zero, ering.zero, fring.zero) + override def one = (aring.one, bring.one, cring.one, dring.one, ering.one, fring.one) + override def negate(v : (A, B, C, D, E, F)) = (aring.negate(v._1), bring.negate(v._2), cring.negate(v._3), dring.negate(v._4), ering.negate(v._5), fring.negate(v._6)) + override def plus(l : (A, B, C, D, E, F), r : (A, B, C, D, E, F)) = (aring.plus(l._1, r._1), bring.plus(l._2, r._2), cring.plus(l._3, r._3), dring.plus(l._4, r._4), ering.plus(l._5, r._5), fring.plus(l._6, r._6)) + override def minus(l : (A, B, C, D, E, F), r : (A, B, C, D, E, F)) = (aring.minus(l._1, r._1), bring.minus(l._2, r._2), cring.minus(l._3, r._3), dring.minus(l._4, r._4), ering.minus(l._5, r._5), fring.minus(l._6, r._6)) + override def times(l : (A, B, C, D, E, F), r : (A, B, C, D, E, F)) = (aring.times(l._1, r._1), bring.times(l._2, r._2), cring.times(l._3, r._3), dring.times(l._4, r._4), ering.times(l._5, r._5), fring.times(l._6, r._6)) +} +/** +* Combine 7 monoids into a product monoid +*/ +class Tuple7Monoid[A, B, C, D, E, F, G](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F], gmonoid : Monoid[G]) extends Monoid[(A, B, C, D, E, F, G)] { + override def zero = (amonoid.zero, bmonoid.zero, cmonoid.zero, dmonoid.zero, emonoid.zero, fmonoid.zero, gmonoid.zero) + override def plus(l : (A, B, C, D, E, F, G), r : (A, B, C, D, E, F, G)) = (amonoid.plus(l._1, r._1), bmonoid.plus(l._2, r._2), cmonoid.plus(l._3, r._3), dmonoid.plus(l._4, r._4), emonoid.plus(l._5, r._5), fmonoid.plus(l._6, r._6), gmonoid.plus(l._7, r._7)) +} + +/** +* Combine 7 groups into a product group +*/ +class Tuple7Group[A, B, C, D, E, F, G](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F], ggroup : Group[G]) extends Group[(A, B, C, D, E, F, G)] { + override def zero = (agroup.zero, bgroup.zero, cgroup.zero, dgroup.zero, egroup.zero, fgroup.zero, ggroup.zero) + override def negate(v : (A, B, C, D, E, F, G)) = (agroup.negate(v._1), bgroup.negate(v._2), cgroup.negate(v._3), dgroup.negate(v._4), egroup.negate(v._5), fgroup.negate(v._6), ggroup.negate(v._7)) + override def plus(l : (A, B, C, D, E, F, G), r : (A, B, C, D, E, F, G)) = (agroup.plus(l._1, r._1), bgroup.plus(l._2, r._2), cgroup.plus(l._3, r._3), dgroup.plus(l._4, r._4), egroup.plus(l._5, r._5), fgroup.plus(l._6, r._6), ggroup.plus(l._7, r._7)) + override def minus(l : (A, B, C, D, E, F, G), r : (A, B, C, D, E, F, G)) = (agroup.minus(l._1, r._1), bgroup.minus(l._2, r._2), cgroup.minus(l._3, r._3), dgroup.minus(l._4, r._4), egroup.minus(l._5, r._5), fgroup.minus(l._6, r._6), ggroup.minus(l._7, r._7)) +} + +/** +* Combine 7 rings into a product ring +*/ +class Tuple7Ring[A, B, C, D, E, F, G](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E], fring : Ring[F], gring : Ring[G]) extends Ring[(A, B, C, D, E, F, G)] { + override def zero = (aring.zero, bring.zero, cring.zero, dring.zero, ering.zero, fring.zero, gring.zero) + override def one = (aring.one, bring.one, cring.one, dring.one, ering.one, fring.one, gring.one) + override def negate(v : (A, B, C, D, E, F, G)) = (aring.negate(v._1), bring.negate(v._2), cring.negate(v._3), dring.negate(v._4), ering.negate(v._5), fring.negate(v._6), gring.negate(v._7)) + override def plus(l : (A, B, C, D, E, F, G), r : (A, B, C, D, E, F, G)) = (aring.plus(l._1, r._1), bring.plus(l._2, r._2), cring.plus(l._3, r._3), dring.plus(l._4, r._4), ering.plus(l._5, r._5), fring.plus(l._6, r._6), gring.plus(l._7, r._7)) + override def minus(l : (A, B, C, D, E, F, G), r : (A, B, C, D, E, F, G)) = (aring.minus(l._1, r._1), bring.minus(l._2, r._2), cring.minus(l._3, r._3), dring.minus(l._4, r._4), ering.minus(l._5, r._5), fring.minus(l._6, r._6), gring.minus(l._7, r._7)) + override def times(l : (A, B, C, D, E, F, G), r : (A, B, C, D, E, F, G)) = (aring.times(l._1, r._1), bring.times(l._2, r._2), cring.times(l._3, r._3), dring.times(l._4, r._4), ering.times(l._5, r._5), fring.times(l._6, r._6), gring.times(l._7, r._7)) +} +/** +* Combine 8 monoids into a product monoid +*/ +class Tuple8Monoid[A, B, C, D, E, F, G, H](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F], gmonoid : Monoid[G], hmonoid : Monoid[H]) extends Monoid[(A, B, C, D, E, F, G, H)] { + override def zero = (amonoid.zero, bmonoid.zero, cmonoid.zero, dmonoid.zero, emonoid.zero, fmonoid.zero, gmonoid.zero, hmonoid.zero) + override def plus(l : (A, B, C, D, E, F, G, H), r : (A, B, C, D, E, F, G, H)) = (amonoid.plus(l._1, r._1), bmonoid.plus(l._2, r._2), cmonoid.plus(l._3, r._3), dmonoid.plus(l._4, r._4), emonoid.plus(l._5, r._5), fmonoid.plus(l._6, r._6), gmonoid.plus(l._7, r._7), hmonoid.plus(l._8, r._8)) +} + +/** +* Combine 8 groups into a product group +*/ +class Tuple8Group[A, B, C, D, E, F, G, H](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F], ggroup : Group[G], hgroup : Group[H]) extends Group[(A, B, C, D, E, F, G, H)] { + override def zero = (agroup.zero, bgroup.zero, cgroup.zero, dgroup.zero, egroup.zero, fgroup.zero, ggroup.zero, hgroup.zero) + override def negate(v : (A, B, C, D, E, F, G, H)) = (agroup.negate(v._1), bgroup.negate(v._2), cgroup.negate(v._3), dgroup.negate(v._4), egroup.negate(v._5), fgroup.negate(v._6), ggroup.negate(v._7), hgroup.negate(v._8)) + override def plus(l : (A, B, C, D, E, F, G, H), r : (A, B, C, D, E, F, G, H)) = (agroup.plus(l._1, r._1), bgroup.plus(l._2, r._2), cgroup.plus(l._3, r._3), dgroup.plus(l._4, r._4), egroup.plus(l._5, r._5), fgroup.plus(l._6, r._6), ggroup.plus(l._7, r._7), hgroup.plus(l._8, r._8)) + override def minus(l : (A, B, C, D, E, F, G, H), r : (A, B, C, D, E, F, G, H)) = (agroup.minus(l._1, r._1), bgroup.minus(l._2, r._2), cgroup.minus(l._3, r._3), dgroup.minus(l._4, r._4), egroup.minus(l._5, r._5), fgroup.minus(l._6, r._6), ggroup.minus(l._7, r._7), hgroup.minus(l._8, r._8)) +} + +/** +* Combine 8 rings into a product ring +*/ +class Tuple8Ring[A, B, C, D, E, F, G, H](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E], fring : Ring[F], gring : Ring[G], hring : Ring[H]) extends Ring[(A, B, C, D, E, F, G, H)] { + override def zero = (aring.zero, bring.zero, cring.zero, dring.zero, ering.zero, fring.zero, gring.zero, hring.zero) + override def one = (aring.one, bring.one, cring.one, dring.one, ering.one, fring.one, gring.one, hring.one) + override def negate(v : (A, B, C, D, E, F, G, H)) = (aring.negate(v._1), bring.negate(v._2), cring.negate(v._3), dring.negate(v._4), ering.negate(v._5), fring.negate(v._6), gring.negate(v._7), hring.negate(v._8)) + override def plus(l : (A, B, C, D, E, F, G, H), r : (A, B, C, D, E, F, G, H)) = (aring.plus(l._1, r._1), bring.plus(l._2, r._2), cring.plus(l._3, r._3), dring.plus(l._4, r._4), ering.plus(l._5, r._5), fring.plus(l._6, r._6), gring.plus(l._7, r._7), hring.plus(l._8, r._8)) + override def minus(l : (A, B, C, D, E, F, G, H), r : (A, B, C, D, E, F, G, H)) = (aring.minus(l._1, r._1), bring.minus(l._2, r._2), cring.minus(l._3, r._3), dring.minus(l._4, r._4), ering.minus(l._5, r._5), fring.minus(l._6, r._6), gring.minus(l._7, r._7), hring.minus(l._8, r._8)) + override def times(l : (A, B, C, D, E, F, G, H), r : (A, B, C, D, E, F, G, H)) = (aring.times(l._1, r._1), bring.times(l._2, r._2), cring.times(l._3, r._3), dring.times(l._4, r._4), ering.times(l._5, r._5), fring.times(l._6, r._6), gring.times(l._7, r._7), hring.times(l._8, r._8)) +} +/** +* Combine 9 monoids into a product monoid +*/ +class Tuple9Monoid[A, B, C, D, E, F, G, H, I](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F], gmonoid : Monoid[G], hmonoid : Monoid[H], imonoid : Monoid[I]) extends Monoid[(A, B, C, D, E, F, G, H, I)] { + override def zero = (amonoid.zero, bmonoid.zero, cmonoid.zero, dmonoid.zero, emonoid.zero, fmonoid.zero, gmonoid.zero, hmonoid.zero, imonoid.zero) + override def plus(l : (A, B, C, D, E, F, G, H, I), r : (A, B, C, D, E, F, G, H, I)) = (amonoid.plus(l._1, r._1), bmonoid.plus(l._2, r._2), cmonoid.plus(l._3, r._3), dmonoid.plus(l._4, r._4), emonoid.plus(l._5, r._5), fmonoid.plus(l._6, r._6), gmonoid.plus(l._7, r._7), hmonoid.plus(l._8, r._8), imonoid.plus(l._9, r._9)) +} + +/** +* Combine 9 groups into a product group +*/ +class Tuple9Group[A, B, C, D, E, F, G, H, I](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F], ggroup : Group[G], hgroup : Group[H], igroup : Group[I]) extends Group[(A, B, C, D, E, F, G, H, I)] { + override def zero = (agroup.zero, bgroup.zero, cgroup.zero, dgroup.zero, egroup.zero, fgroup.zero, ggroup.zero, hgroup.zero, igroup.zero) + override def negate(v : (A, B, C, D, E, F, G, H, I)) = (agroup.negate(v._1), bgroup.negate(v._2), cgroup.negate(v._3), dgroup.negate(v._4), egroup.negate(v._5), fgroup.negate(v._6), ggroup.negate(v._7), hgroup.negate(v._8), igroup.negate(v._9)) + override def plus(l : (A, B, C, D, E, F, G, H, I), r : (A, B, C, D, E, F, G, H, I)) = (agroup.plus(l._1, r._1), bgroup.plus(l._2, r._2), cgroup.plus(l._3, r._3), dgroup.plus(l._4, r._4), egroup.plus(l._5, r._5), fgroup.plus(l._6, r._6), ggroup.plus(l._7, r._7), hgroup.plus(l._8, r._8), igroup.plus(l._9, r._9)) + override def minus(l : (A, B, C, D, E, F, G, H, I), r : (A, B, C, D, E, F, G, H, I)) = (agroup.minus(l._1, r._1), bgroup.minus(l._2, r._2), cgroup.minus(l._3, r._3), dgroup.minus(l._4, r._4), egroup.minus(l._5, r._5), fgroup.minus(l._6, r._6), ggroup.minus(l._7, r._7), hgroup.minus(l._8, r._8), igroup.minus(l._9, r._9)) +} + +/** +* Combine 9 rings into a product ring +*/ +class Tuple9Ring[A, B, C, D, E, F, G, H, I](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E], fring : Ring[F], gring : Ring[G], hring : Ring[H], iring : Ring[I]) extends Ring[(A, B, C, D, E, F, G, H, I)] { + override def zero = (aring.zero, bring.zero, cring.zero, dring.zero, ering.zero, fring.zero, gring.zero, hring.zero, iring.zero) + override def one = (aring.one, bring.one, cring.one, dring.one, ering.one, fring.one, gring.one, hring.one, iring.one) + override def negate(v : (A, B, C, D, E, F, G, H, I)) = (aring.negate(v._1), bring.negate(v._2), cring.negate(v._3), dring.negate(v._4), ering.negate(v._5), fring.negate(v._6), gring.negate(v._7), hring.negate(v._8), iring.negate(v._9)) + override def plus(l : (A, B, C, D, E, F, G, H, I), r : (A, B, C, D, E, F, G, H, I)) = (aring.plus(l._1, r._1), bring.plus(l._2, r._2), cring.plus(l._3, r._3), dring.plus(l._4, r._4), ering.plus(l._5, r._5), fring.plus(l._6, r._6), gring.plus(l._7, r._7), hring.plus(l._8, r._8), iring.plus(l._9, r._9)) + override def minus(l : (A, B, C, D, E, F, G, H, I), r : (A, B, C, D, E, F, G, H, I)) = (aring.minus(l._1, r._1), bring.minus(l._2, r._2), cring.minus(l._3, r._3), dring.minus(l._4, r._4), ering.minus(l._5, r._5), fring.minus(l._6, r._6), gring.minus(l._7, r._7), hring.minus(l._8, r._8), iring.minus(l._9, r._9)) + override def times(l : (A, B, C, D, E, F, G, H, I), r : (A, B, C, D, E, F, G, H, I)) = (aring.times(l._1, r._1), bring.times(l._2, r._2), cring.times(l._3, r._3), dring.times(l._4, r._4), ering.times(l._5, r._5), fring.times(l._6, r._6), gring.times(l._7, r._7), hring.times(l._8, r._8), iring.times(l._9, r._9)) +} +/** +* Combine 10 monoids into a product monoid +*/ +class Tuple10Monoid[A, B, C, D, E, F, G, H, I, J](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F], gmonoid : Monoid[G], hmonoid : Monoid[H], imonoid : Monoid[I], jmonoid : Monoid[J]) extends Monoid[(A, B, C, D, E, F, G, H, I, J)] { + override def zero = (amonoid.zero, bmonoid.zero, cmonoid.zero, dmonoid.zero, emonoid.zero, fmonoid.zero, gmonoid.zero, hmonoid.zero, imonoid.zero, jmonoid.zero) + override def plus(l : (A, B, C, D, E, F, G, H, I, J), r : (A, B, C, D, E, F, G, H, I, J)) = (amonoid.plus(l._1, r._1), bmonoid.plus(l._2, r._2), cmonoid.plus(l._3, r._3), dmonoid.plus(l._4, r._4), emonoid.plus(l._5, r._5), fmonoid.plus(l._6, r._6), gmonoid.plus(l._7, r._7), hmonoid.plus(l._8, r._8), imonoid.plus(l._9, r._9), jmonoid.plus(l._10, r._10)) +} + +/** +* Combine 10 groups into a product group +*/ +class Tuple10Group[A, B, C, D, E, F, G, H, I, J](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F], ggroup : Group[G], hgroup : Group[H], igroup : Group[I], jgroup : Group[J]) extends Group[(A, B, C, D, E, F, G, H, I, J)] { + override def zero = (agroup.zero, bgroup.zero, cgroup.zero, dgroup.zero, egroup.zero, fgroup.zero, ggroup.zero, hgroup.zero, igroup.zero, jgroup.zero) + override def negate(v : (A, B, C, D, E, F, G, H, I, J)) = (agroup.negate(v._1), bgroup.negate(v._2), cgroup.negate(v._3), dgroup.negate(v._4), egroup.negate(v._5), fgroup.negate(v._6), ggroup.negate(v._7), hgroup.negate(v._8), igroup.negate(v._9), jgroup.negate(v._10)) + override def plus(l : (A, B, C, D, E, F, G, H, I, J), r : (A, B, C, D, E, F, G, H, I, J)) = (agroup.plus(l._1, r._1), bgroup.plus(l._2, r._2), cgroup.plus(l._3, r._3), dgroup.plus(l._4, r._4), egroup.plus(l._5, r._5), fgroup.plus(l._6, r._6), ggroup.plus(l._7, r._7), hgroup.plus(l._8, r._8), igroup.plus(l._9, r._9), jgroup.plus(l._10, r._10)) + override def minus(l : (A, B, C, D, E, F, G, H, I, J), r : (A, B, C, D, E, F, G, H, I, J)) = (agroup.minus(l._1, r._1), bgroup.minus(l._2, r._2), cgroup.minus(l._3, r._3), dgroup.minus(l._4, r._4), egroup.minus(l._5, r._5), fgroup.minus(l._6, r._6), ggroup.minus(l._7, r._7), hgroup.minus(l._8, r._8), igroup.minus(l._9, r._9), jgroup.minus(l._10, r._10)) +} + +/** +* Combine 10 rings into a product ring +*/ +class Tuple10Ring[A, B, C, D, E, F, G, H, I, J](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E], fring : Ring[F], gring : Ring[G], hring : Ring[H], iring : Ring[I], jring : Ring[J]) extends Ring[(A, B, C, D, E, F, G, H, I, J)] { + override def zero = (aring.zero, bring.zero, cring.zero, dring.zero, ering.zero, fring.zero, gring.zero, hring.zero, iring.zero, jring.zero) + override def one = (aring.one, bring.one, cring.one, dring.one, ering.one, fring.one, gring.one, hring.one, iring.one, jring.one) + override def negate(v : (A, B, C, D, E, F, G, H, I, J)) = (aring.negate(v._1), bring.negate(v._2), cring.negate(v._3), dring.negate(v._4), ering.negate(v._5), fring.negate(v._6), gring.negate(v._7), hring.negate(v._8), iring.negate(v._9), jring.negate(v._10)) + override def plus(l : (A, B, C, D, E, F, G, H, I, J), r : (A, B, C, D, E, F, G, H, I, J)) = (aring.plus(l._1, r._1), bring.plus(l._2, r._2), cring.plus(l._3, r._3), dring.plus(l._4, r._4), ering.plus(l._5, r._5), fring.plus(l._6, r._6), gring.plus(l._7, r._7), hring.plus(l._8, r._8), iring.plus(l._9, r._9), jring.plus(l._10, r._10)) + override def minus(l : (A, B, C, D, E, F, G, H, I, J), r : (A, B, C, D, E, F, G, H, I, J)) = (aring.minus(l._1, r._1), bring.minus(l._2, r._2), cring.minus(l._3, r._3), dring.minus(l._4, r._4), ering.minus(l._5, r._5), fring.minus(l._6, r._6), gring.minus(l._7, r._7), hring.minus(l._8, r._8), iring.minus(l._9, r._9), jring.minus(l._10, r._10)) + override def times(l : (A, B, C, D, E, F, G, H, I, J), r : (A, B, C, D, E, F, G, H, I, J)) = (aring.times(l._1, r._1), bring.times(l._2, r._2), cring.times(l._3, r._3), dring.times(l._4, r._4), ering.times(l._5, r._5), fring.times(l._6, r._6), gring.times(l._7, r._7), hring.times(l._8, r._8), iring.times(l._9, r._9), jring.times(l._10, r._10)) +} +/** +* Combine 11 monoids into a product monoid +*/ +class Tuple11Monoid[A, B, C, D, E, F, G, H, I, J, K](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F], gmonoid : Monoid[G], hmonoid : Monoid[H], imonoid : Monoid[I], jmonoid : Monoid[J], kmonoid : Monoid[K]) extends Monoid[(A, B, C, D, E, F, G, H, I, J, K)] { + override def zero = (amonoid.zero, bmonoid.zero, cmonoid.zero, dmonoid.zero, emonoid.zero, fmonoid.zero, gmonoid.zero, hmonoid.zero, imonoid.zero, jmonoid.zero, kmonoid.zero) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K), r : (A, B, C, D, E, F, G, H, I, J, K)) = (amonoid.plus(l._1, r._1), bmonoid.plus(l._2, r._2), cmonoid.plus(l._3, r._3), dmonoid.plus(l._4, r._4), emonoid.plus(l._5, r._5), fmonoid.plus(l._6, r._6), gmonoid.plus(l._7, r._7), hmonoid.plus(l._8, r._8), imonoid.plus(l._9, r._9), jmonoid.plus(l._10, r._10), kmonoid.plus(l._11, r._11)) +} + +/** +* Combine 11 groups into a product group +*/ +class Tuple11Group[A, B, C, D, E, F, G, H, I, J, K](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F], ggroup : Group[G], hgroup : Group[H], igroup : Group[I], jgroup : Group[J], kgroup : Group[K]) extends Group[(A, B, C, D, E, F, G, H, I, J, K)] { + override def zero = (agroup.zero, bgroup.zero, cgroup.zero, dgroup.zero, egroup.zero, fgroup.zero, ggroup.zero, hgroup.zero, igroup.zero, jgroup.zero, kgroup.zero) + override def negate(v : (A, B, C, D, E, F, G, H, I, J, K)) = (agroup.negate(v._1), bgroup.negate(v._2), cgroup.negate(v._3), dgroup.negate(v._4), egroup.negate(v._5), fgroup.negate(v._6), ggroup.negate(v._7), hgroup.negate(v._8), igroup.negate(v._9), jgroup.negate(v._10), kgroup.negate(v._11)) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K), r : (A, B, C, D, E, F, G, H, I, J, K)) = (agroup.plus(l._1, r._1), bgroup.plus(l._2, r._2), cgroup.plus(l._3, r._3), dgroup.plus(l._4, r._4), egroup.plus(l._5, r._5), fgroup.plus(l._6, r._6), ggroup.plus(l._7, r._7), hgroup.plus(l._8, r._8), igroup.plus(l._9, r._9), jgroup.plus(l._10, r._10), kgroup.plus(l._11, r._11)) + override def minus(l : (A, B, C, D, E, F, G, H, I, J, K), r : (A, B, C, D, E, F, G, H, I, J, K)) = (agroup.minus(l._1, r._1), bgroup.minus(l._2, r._2), cgroup.minus(l._3, r._3), dgroup.minus(l._4, r._4), egroup.minus(l._5, r._5), fgroup.minus(l._6, r._6), ggroup.minus(l._7, r._7), hgroup.minus(l._8, r._8), igroup.minus(l._9, r._9), jgroup.minus(l._10, r._10), kgroup.minus(l._11, r._11)) +} + +/** +* Combine 11 rings into a product ring +*/ +class Tuple11Ring[A, B, C, D, E, F, G, H, I, J, K](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E], fring : Ring[F], gring : Ring[G], hring : Ring[H], iring : Ring[I], jring : Ring[J], kring : Ring[K]) extends Ring[(A, B, C, D, E, F, G, H, I, J, K)] { + override def zero = (aring.zero, bring.zero, cring.zero, dring.zero, ering.zero, fring.zero, gring.zero, hring.zero, iring.zero, jring.zero, kring.zero) + override def one = (aring.one, bring.one, cring.one, dring.one, ering.one, fring.one, gring.one, hring.one, iring.one, jring.one, kring.one) + override def negate(v : (A, B, C, D, E, F, G, H, I, J, K)) = (aring.negate(v._1), bring.negate(v._2), cring.negate(v._3), dring.negate(v._4), ering.negate(v._5), fring.negate(v._6), gring.negate(v._7), hring.negate(v._8), iring.negate(v._9), jring.negate(v._10), kring.negate(v._11)) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K), r : (A, B, C, D, E, F, G, H, I, J, K)) = (aring.plus(l._1, r._1), bring.plus(l._2, r._2), cring.plus(l._3, r._3), dring.plus(l._4, r._4), ering.plus(l._5, r._5), fring.plus(l._6, r._6), gring.plus(l._7, r._7), hring.plus(l._8, r._8), iring.plus(l._9, r._9), jring.plus(l._10, r._10), kring.plus(l._11, r._11)) + override def minus(l : (A, B, C, D, E, F, G, H, I, J, K), r : (A, B, C, D, E, F, G, H, I, J, K)) = (aring.minus(l._1, r._1), bring.minus(l._2, r._2), cring.minus(l._3, r._3), dring.minus(l._4, r._4), ering.minus(l._5, r._5), fring.minus(l._6, r._6), gring.minus(l._7, r._7), hring.minus(l._8, r._8), iring.minus(l._9, r._9), jring.minus(l._10, r._10), kring.minus(l._11, r._11)) + override def times(l : (A, B, C, D, E, F, G, H, I, J, K), r : (A, B, C, D, E, F, G, H, I, J, K)) = (aring.times(l._1, r._1), bring.times(l._2, r._2), cring.times(l._3, r._3), dring.times(l._4, r._4), ering.times(l._5, r._5), fring.times(l._6, r._6), gring.times(l._7, r._7), hring.times(l._8, r._8), iring.times(l._9, r._9), jring.times(l._10, r._10), kring.times(l._11, r._11)) +} +/** +* Combine 12 monoids into a product monoid +*/ +class Tuple12Monoid[A, B, C, D, E, F, G, H, I, J, K, L](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F], gmonoid : Monoid[G], hmonoid : Monoid[H], imonoid : Monoid[I], jmonoid : Monoid[J], kmonoid : Monoid[K], lmonoid : Monoid[L]) extends Monoid[(A, B, C, D, E, F, G, H, I, J, K, L)] { + override def zero = (amonoid.zero, bmonoid.zero, cmonoid.zero, dmonoid.zero, emonoid.zero, fmonoid.zero, gmonoid.zero, hmonoid.zero, imonoid.zero, jmonoid.zero, kmonoid.zero, lmonoid.zero) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K, L), r : (A, B, C, D, E, F, G, H, I, J, K, L)) = (amonoid.plus(l._1, r._1), bmonoid.plus(l._2, r._2), cmonoid.plus(l._3, r._3), dmonoid.plus(l._4, r._4), emonoid.plus(l._5, r._5), fmonoid.plus(l._6, r._6), gmonoid.plus(l._7, r._7), hmonoid.plus(l._8, r._8), imonoid.plus(l._9, r._9), jmonoid.plus(l._10, r._10), kmonoid.plus(l._11, r._11), lmonoid.plus(l._12, r._12)) +} + +/** +* Combine 12 groups into a product group +*/ +class Tuple12Group[A, B, C, D, E, F, G, H, I, J, K, L](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F], ggroup : Group[G], hgroup : Group[H], igroup : Group[I], jgroup : Group[J], kgroup : Group[K], lgroup : Group[L]) extends Group[(A, B, C, D, E, F, G, H, I, J, K, L)] { + override def zero = (agroup.zero, bgroup.zero, cgroup.zero, dgroup.zero, egroup.zero, fgroup.zero, ggroup.zero, hgroup.zero, igroup.zero, jgroup.zero, kgroup.zero, lgroup.zero) + override def negate(v : (A, B, C, D, E, F, G, H, I, J, K, L)) = (agroup.negate(v._1), bgroup.negate(v._2), cgroup.negate(v._3), dgroup.negate(v._4), egroup.negate(v._5), fgroup.negate(v._6), ggroup.negate(v._7), hgroup.negate(v._8), igroup.negate(v._9), jgroup.negate(v._10), kgroup.negate(v._11), lgroup.negate(v._12)) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K, L), r : (A, B, C, D, E, F, G, H, I, J, K, L)) = (agroup.plus(l._1, r._1), bgroup.plus(l._2, r._2), cgroup.plus(l._3, r._3), dgroup.plus(l._4, r._4), egroup.plus(l._5, r._5), fgroup.plus(l._6, r._6), ggroup.plus(l._7, r._7), hgroup.plus(l._8, r._8), igroup.plus(l._9, r._9), jgroup.plus(l._10, r._10), kgroup.plus(l._11, r._11), lgroup.plus(l._12, r._12)) + override def minus(l : (A, B, C, D, E, F, G, H, I, J, K, L), r : (A, B, C, D, E, F, G, H, I, J, K, L)) = (agroup.minus(l._1, r._1), bgroup.minus(l._2, r._2), cgroup.minus(l._3, r._3), dgroup.minus(l._4, r._4), egroup.minus(l._5, r._5), fgroup.minus(l._6, r._6), ggroup.minus(l._7, r._7), hgroup.minus(l._8, r._8), igroup.minus(l._9, r._9), jgroup.minus(l._10, r._10), kgroup.minus(l._11, r._11), lgroup.minus(l._12, r._12)) +} + +/** +* Combine 12 rings into a product ring +*/ +class Tuple12Ring[A, B, C, D, E, F, G, H, I, J, K, L](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E], fring : Ring[F], gring : Ring[G], hring : Ring[H], iring : Ring[I], jring : Ring[J], kring : Ring[K], lring : Ring[L]) extends Ring[(A, B, C, D, E, F, G, H, I, J, K, L)] { + override def zero = (aring.zero, bring.zero, cring.zero, dring.zero, ering.zero, fring.zero, gring.zero, hring.zero, iring.zero, jring.zero, kring.zero, lring.zero) + override def one = (aring.one, bring.one, cring.one, dring.one, ering.one, fring.one, gring.one, hring.one, iring.one, jring.one, kring.one, lring.one) + override def negate(v : (A, B, C, D, E, F, G, H, I, J, K, L)) = (aring.negate(v._1), bring.negate(v._2), cring.negate(v._3), dring.negate(v._4), ering.negate(v._5), fring.negate(v._6), gring.negate(v._7), hring.negate(v._8), iring.negate(v._9), jring.negate(v._10), kring.negate(v._11), lring.negate(v._12)) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K, L), r : (A, B, C, D, E, F, G, H, I, J, K, L)) = (aring.plus(l._1, r._1), bring.plus(l._2, r._2), cring.plus(l._3, r._3), dring.plus(l._4, r._4), ering.plus(l._5, r._5), fring.plus(l._6, r._6), gring.plus(l._7, r._7), hring.plus(l._8, r._8), iring.plus(l._9, r._9), jring.plus(l._10, r._10), kring.plus(l._11, r._11), lring.plus(l._12, r._12)) + override def minus(l : (A, B, C, D, E, F, G, H, I, J, K, L), r : (A, B, C, D, E, F, G, H, I, J, K, L)) = (aring.minus(l._1, r._1), bring.minus(l._2, r._2), cring.minus(l._3, r._3), dring.minus(l._4, r._4), ering.minus(l._5, r._5), fring.minus(l._6, r._6), gring.minus(l._7, r._7), hring.minus(l._8, r._8), iring.minus(l._9, r._9), jring.minus(l._10, r._10), kring.minus(l._11, r._11), lring.minus(l._12, r._12)) + override def times(l : (A, B, C, D, E, F, G, H, I, J, K, L), r : (A, B, C, D, E, F, G, H, I, J, K, L)) = (aring.times(l._1, r._1), bring.times(l._2, r._2), cring.times(l._3, r._3), dring.times(l._4, r._4), ering.times(l._5, r._5), fring.times(l._6, r._6), gring.times(l._7, r._7), hring.times(l._8, r._8), iring.times(l._9, r._9), jring.times(l._10, r._10), kring.times(l._11, r._11), lring.times(l._12, r._12)) +} +/** +* Combine 13 monoids into a product monoid +*/ +class Tuple13Monoid[A, B, C, D, E, F, G, H, I, J, K, L, M](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F], gmonoid : Monoid[G], hmonoid : Monoid[H], imonoid : Monoid[I], jmonoid : Monoid[J], kmonoid : Monoid[K], lmonoid : Monoid[L], mmonoid : Monoid[M]) extends Monoid[(A, B, C, D, E, F, G, H, I, J, K, L, M)] { + override def zero = (amonoid.zero, bmonoid.zero, cmonoid.zero, dmonoid.zero, emonoid.zero, fmonoid.zero, gmonoid.zero, hmonoid.zero, imonoid.zero, jmonoid.zero, kmonoid.zero, lmonoid.zero, mmonoid.zero) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M), r : (A, B, C, D, E, F, G, H, I, J, K, L, M)) = (amonoid.plus(l._1, r._1), bmonoid.plus(l._2, r._2), cmonoid.plus(l._3, r._3), dmonoid.plus(l._4, r._4), emonoid.plus(l._5, r._5), fmonoid.plus(l._6, r._6), gmonoid.plus(l._7, r._7), hmonoid.plus(l._8, r._8), imonoid.plus(l._9, r._9), jmonoid.plus(l._10, r._10), kmonoid.plus(l._11, r._11), lmonoid.plus(l._12, r._12), mmonoid.plus(l._13, r._13)) +} + +/** +* Combine 13 groups into a product group +*/ +class Tuple13Group[A, B, C, D, E, F, G, H, I, J, K, L, M](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F], ggroup : Group[G], hgroup : Group[H], igroup : Group[I], jgroup : Group[J], kgroup : Group[K], lgroup : Group[L], mgroup : Group[M]) extends Group[(A, B, C, D, E, F, G, H, I, J, K, L, M)] { + override def zero = (agroup.zero, bgroup.zero, cgroup.zero, dgroup.zero, egroup.zero, fgroup.zero, ggroup.zero, hgroup.zero, igroup.zero, jgroup.zero, kgroup.zero, lgroup.zero, mgroup.zero) + override def negate(v : (A, B, C, D, E, F, G, H, I, J, K, L, M)) = (agroup.negate(v._1), bgroup.negate(v._2), cgroup.negate(v._3), dgroup.negate(v._4), egroup.negate(v._5), fgroup.negate(v._6), ggroup.negate(v._7), hgroup.negate(v._8), igroup.negate(v._9), jgroup.negate(v._10), kgroup.negate(v._11), lgroup.negate(v._12), mgroup.negate(v._13)) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M), r : (A, B, C, D, E, F, G, H, I, J, K, L, M)) = (agroup.plus(l._1, r._1), bgroup.plus(l._2, r._2), cgroup.plus(l._3, r._3), dgroup.plus(l._4, r._4), egroup.plus(l._5, r._5), fgroup.plus(l._6, r._6), ggroup.plus(l._7, r._7), hgroup.plus(l._8, r._8), igroup.plus(l._9, r._9), jgroup.plus(l._10, r._10), kgroup.plus(l._11, r._11), lgroup.plus(l._12, r._12), mgroup.plus(l._13, r._13)) + override def minus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M), r : (A, B, C, D, E, F, G, H, I, J, K, L, M)) = (agroup.minus(l._1, r._1), bgroup.minus(l._2, r._2), cgroup.minus(l._3, r._3), dgroup.minus(l._4, r._4), egroup.minus(l._5, r._5), fgroup.minus(l._6, r._6), ggroup.minus(l._7, r._7), hgroup.minus(l._8, r._8), igroup.minus(l._9, r._9), jgroup.minus(l._10, r._10), kgroup.minus(l._11, r._11), lgroup.minus(l._12, r._12), mgroup.minus(l._13, r._13)) +} + +/** +* Combine 13 rings into a product ring +*/ +class Tuple13Ring[A, B, C, D, E, F, G, H, I, J, K, L, M](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E], fring : Ring[F], gring : Ring[G], hring : Ring[H], iring : Ring[I], jring : Ring[J], kring : Ring[K], lring : Ring[L], mring : Ring[M]) extends Ring[(A, B, C, D, E, F, G, H, I, J, K, L, M)] { + override def zero = (aring.zero, bring.zero, cring.zero, dring.zero, ering.zero, fring.zero, gring.zero, hring.zero, iring.zero, jring.zero, kring.zero, lring.zero, mring.zero) + override def one = (aring.one, bring.one, cring.one, dring.one, ering.one, fring.one, gring.one, hring.one, iring.one, jring.one, kring.one, lring.one, mring.one) + override def negate(v : (A, B, C, D, E, F, G, H, I, J, K, L, M)) = (aring.negate(v._1), bring.negate(v._2), cring.negate(v._3), dring.negate(v._4), ering.negate(v._5), fring.negate(v._6), gring.negate(v._7), hring.negate(v._8), iring.negate(v._9), jring.negate(v._10), kring.negate(v._11), lring.negate(v._12), mring.negate(v._13)) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M), r : (A, B, C, D, E, F, G, H, I, J, K, L, M)) = (aring.plus(l._1, r._1), bring.plus(l._2, r._2), cring.plus(l._3, r._3), dring.plus(l._4, r._4), ering.plus(l._5, r._5), fring.plus(l._6, r._6), gring.plus(l._7, r._7), hring.plus(l._8, r._8), iring.plus(l._9, r._9), jring.plus(l._10, r._10), kring.plus(l._11, r._11), lring.plus(l._12, r._12), mring.plus(l._13, r._13)) + override def minus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M), r : (A, B, C, D, E, F, G, H, I, J, K, L, M)) = (aring.minus(l._1, r._1), bring.minus(l._2, r._2), cring.minus(l._3, r._3), dring.minus(l._4, r._4), ering.minus(l._5, r._5), fring.minus(l._6, r._6), gring.minus(l._7, r._7), hring.minus(l._8, r._8), iring.minus(l._9, r._9), jring.minus(l._10, r._10), kring.minus(l._11, r._11), lring.minus(l._12, r._12), mring.minus(l._13, r._13)) + override def times(l : (A, B, C, D, E, F, G, H, I, J, K, L, M), r : (A, B, C, D, E, F, G, H, I, J, K, L, M)) = (aring.times(l._1, r._1), bring.times(l._2, r._2), cring.times(l._3, r._3), dring.times(l._4, r._4), ering.times(l._5, r._5), fring.times(l._6, r._6), gring.times(l._7, r._7), hring.times(l._8, r._8), iring.times(l._9, r._9), jring.times(l._10, r._10), kring.times(l._11, r._11), lring.times(l._12, r._12), mring.times(l._13, r._13)) +} +/** +* Combine 14 monoids into a product monoid +*/ +class Tuple14Monoid[A, B, C, D, E, F, G, H, I, J, K, L, M, N](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F], gmonoid : Monoid[G], hmonoid : Monoid[H], imonoid : Monoid[I], jmonoid : Monoid[J], kmonoid : Monoid[K], lmonoid : Monoid[L], mmonoid : Monoid[M], nmonoid : Monoid[N]) extends Monoid[(A, B, C, D, E, F, G, H, I, J, K, L, M, N)] { + override def zero = (amonoid.zero, bmonoid.zero, cmonoid.zero, dmonoid.zero, emonoid.zero, fmonoid.zero, gmonoid.zero, hmonoid.zero, imonoid.zero, jmonoid.zero, kmonoid.zero, lmonoid.zero, mmonoid.zero, nmonoid.zero) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N)) = (amonoid.plus(l._1, r._1), bmonoid.plus(l._2, r._2), cmonoid.plus(l._3, r._3), dmonoid.plus(l._4, r._4), emonoid.plus(l._5, r._5), fmonoid.plus(l._6, r._6), gmonoid.plus(l._7, r._7), hmonoid.plus(l._8, r._8), imonoid.plus(l._9, r._9), jmonoid.plus(l._10, r._10), kmonoid.plus(l._11, r._11), lmonoid.plus(l._12, r._12), mmonoid.plus(l._13, r._13), nmonoid.plus(l._14, r._14)) +} + +/** +* Combine 14 groups into a product group +*/ +class Tuple14Group[A, B, C, D, E, F, G, H, I, J, K, L, M, N](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F], ggroup : Group[G], hgroup : Group[H], igroup : Group[I], jgroup : Group[J], kgroup : Group[K], lgroup : Group[L], mgroup : Group[M], ngroup : Group[N]) extends Group[(A, B, C, D, E, F, G, H, I, J, K, L, M, N)] { + override def zero = (agroup.zero, bgroup.zero, cgroup.zero, dgroup.zero, egroup.zero, fgroup.zero, ggroup.zero, hgroup.zero, igroup.zero, jgroup.zero, kgroup.zero, lgroup.zero, mgroup.zero, ngroup.zero) + override def negate(v : (A, B, C, D, E, F, G, H, I, J, K, L, M, N)) = (agroup.negate(v._1), bgroup.negate(v._2), cgroup.negate(v._3), dgroup.negate(v._4), egroup.negate(v._5), fgroup.negate(v._6), ggroup.negate(v._7), hgroup.negate(v._8), igroup.negate(v._9), jgroup.negate(v._10), kgroup.negate(v._11), lgroup.negate(v._12), mgroup.negate(v._13), ngroup.negate(v._14)) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N)) = (agroup.plus(l._1, r._1), bgroup.plus(l._2, r._2), cgroup.plus(l._3, r._3), dgroup.plus(l._4, r._4), egroup.plus(l._5, r._5), fgroup.plus(l._6, r._6), ggroup.plus(l._7, r._7), hgroup.plus(l._8, r._8), igroup.plus(l._9, r._9), jgroup.plus(l._10, r._10), kgroup.plus(l._11, r._11), lgroup.plus(l._12, r._12), mgroup.plus(l._13, r._13), ngroup.plus(l._14, r._14)) + override def minus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N)) = (agroup.minus(l._1, r._1), bgroup.minus(l._2, r._2), cgroup.minus(l._3, r._3), dgroup.minus(l._4, r._4), egroup.minus(l._5, r._5), fgroup.minus(l._6, r._6), ggroup.minus(l._7, r._7), hgroup.minus(l._8, r._8), igroup.minus(l._9, r._9), jgroup.minus(l._10, r._10), kgroup.minus(l._11, r._11), lgroup.minus(l._12, r._12), mgroup.minus(l._13, r._13), ngroup.minus(l._14, r._14)) +} + +/** +* Combine 14 rings into a product ring +*/ +class Tuple14Ring[A, B, C, D, E, F, G, H, I, J, K, L, M, N](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E], fring : Ring[F], gring : Ring[G], hring : Ring[H], iring : Ring[I], jring : Ring[J], kring : Ring[K], lring : Ring[L], mring : Ring[M], nring : Ring[N]) extends Ring[(A, B, C, D, E, F, G, H, I, J, K, L, M, N)] { + override def zero = (aring.zero, bring.zero, cring.zero, dring.zero, ering.zero, fring.zero, gring.zero, hring.zero, iring.zero, jring.zero, kring.zero, lring.zero, mring.zero, nring.zero) + override def one = (aring.one, bring.one, cring.one, dring.one, ering.one, fring.one, gring.one, hring.one, iring.one, jring.one, kring.one, lring.one, mring.one, nring.one) + override def negate(v : (A, B, C, D, E, F, G, H, I, J, K, L, M, N)) = (aring.negate(v._1), bring.negate(v._2), cring.negate(v._3), dring.negate(v._4), ering.negate(v._5), fring.negate(v._6), gring.negate(v._7), hring.negate(v._8), iring.negate(v._9), jring.negate(v._10), kring.negate(v._11), lring.negate(v._12), mring.negate(v._13), nring.negate(v._14)) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N)) = (aring.plus(l._1, r._1), bring.plus(l._2, r._2), cring.plus(l._3, r._3), dring.plus(l._4, r._4), ering.plus(l._5, r._5), fring.plus(l._6, r._6), gring.plus(l._7, r._7), hring.plus(l._8, r._8), iring.plus(l._9, r._9), jring.plus(l._10, r._10), kring.plus(l._11, r._11), lring.plus(l._12, r._12), mring.plus(l._13, r._13), nring.plus(l._14, r._14)) + override def minus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N)) = (aring.minus(l._1, r._1), bring.minus(l._2, r._2), cring.minus(l._3, r._3), dring.minus(l._4, r._4), ering.minus(l._5, r._5), fring.minus(l._6, r._6), gring.minus(l._7, r._7), hring.minus(l._8, r._8), iring.minus(l._9, r._9), jring.minus(l._10, r._10), kring.minus(l._11, r._11), lring.minus(l._12, r._12), mring.minus(l._13, r._13), nring.minus(l._14, r._14)) + override def times(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N)) = (aring.times(l._1, r._1), bring.times(l._2, r._2), cring.times(l._3, r._3), dring.times(l._4, r._4), ering.times(l._5, r._5), fring.times(l._6, r._6), gring.times(l._7, r._7), hring.times(l._8, r._8), iring.times(l._9, r._9), jring.times(l._10, r._10), kring.times(l._11, r._11), lring.times(l._12, r._12), mring.times(l._13, r._13), nring.times(l._14, r._14)) +} +/** +* Combine 15 monoids into a product monoid +*/ +class Tuple15Monoid[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F], gmonoid : Monoid[G], hmonoid : Monoid[H], imonoid : Monoid[I], jmonoid : Monoid[J], kmonoid : Monoid[K], lmonoid : Monoid[L], mmonoid : Monoid[M], nmonoid : Monoid[N], omonoid : Monoid[O]) extends Monoid[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O)] { + override def zero = (amonoid.zero, bmonoid.zero, cmonoid.zero, dmonoid.zero, emonoid.zero, fmonoid.zero, gmonoid.zero, hmonoid.zero, imonoid.zero, jmonoid.zero, kmonoid.zero, lmonoid.zero, mmonoid.zero, nmonoid.zero, omonoid.zero) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O)) = (amonoid.plus(l._1, r._1), bmonoid.plus(l._2, r._2), cmonoid.plus(l._3, r._3), dmonoid.plus(l._4, r._4), emonoid.plus(l._5, r._5), fmonoid.plus(l._6, r._6), gmonoid.plus(l._7, r._7), hmonoid.plus(l._8, r._8), imonoid.plus(l._9, r._9), jmonoid.plus(l._10, r._10), kmonoid.plus(l._11, r._11), lmonoid.plus(l._12, r._12), mmonoid.plus(l._13, r._13), nmonoid.plus(l._14, r._14), omonoid.plus(l._15, r._15)) +} + +/** +* Combine 15 groups into a product group +*/ +class Tuple15Group[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F], ggroup : Group[G], hgroup : Group[H], igroup : Group[I], jgroup : Group[J], kgroup : Group[K], lgroup : Group[L], mgroup : Group[M], ngroup : Group[N], ogroup : Group[O]) extends Group[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O)] { + override def zero = (agroup.zero, bgroup.zero, cgroup.zero, dgroup.zero, egroup.zero, fgroup.zero, ggroup.zero, hgroup.zero, igroup.zero, jgroup.zero, kgroup.zero, lgroup.zero, mgroup.zero, ngroup.zero, ogroup.zero) + override def negate(v : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O)) = (agroup.negate(v._1), bgroup.negate(v._2), cgroup.negate(v._3), dgroup.negate(v._4), egroup.negate(v._5), fgroup.negate(v._6), ggroup.negate(v._7), hgroup.negate(v._8), igroup.negate(v._9), jgroup.negate(v._10), kgroup.negate(v._11), lgroup.negate(v._12), mgroup.negate(v._13), ngroup.negate(v._14), ogroup.negate(v._15)) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O)) = (agroup.plus(l._1, r._1), bgroup.plus(l._2, r._2), cgroup.plus(l._3, r._3), dgroup.plus(l._4, r._4), egroup.plus(l._5, r._5), fgroup.plus(l._6, r._6), ggroup.plus(l._7, r._7), hgroup.plus(l._8, r._8), igroup.plus(l._9, r._9), jgroup.plus(l._10, r._10), kgroup.plus(l._11, r._11), lgroup.plus(l._12, r._12), mgroup.plus(l._13, r._13), ngroup.plus(l._14, r._14), ogroup.plus(l._15, r._15)) + override def minus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O)) = (agroup.minus(l._1, r._1), bgroup.minus(l._2, r._2), cgroup.minus(l._3, r._3), dgroup.minus(l._4, r._4), egroup.minus(l._5, r._5), fgroup.minus(l._6, r._6), ggroup.minus(l._7, r._7), hgroup.minus(l._8, r._8), igroup.minus(l._9, r._9), jgroup.minus(l._10, r._10), kgroup.minus(l._11, r._11), lgroup.minus(l._12, r._12), mgroup.minus(l._13, r._13), ngroup.minus(l._14, r._14), ogroup.minus(l._15, r._15)) +} + +/** +* Combine 15 rings into a product ring +*/ +class Tuple15Ring[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E], fring : Ring[F], gring : Ring[G], hring : Ring[H], iring : Ring[I], jring : Ring[J], kring : Ring[K], lring : Ring[L], mring : Ring[M], nring : Ring[N], oring : Ring[O]) extends Ring[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O)] { + override def zero = (aring.zero, bring.zero, cring.zero, dring.zero, ering.zero, fring.zero, gring.zero, hring.zero, iring.zero, jring.zero, kring.zero, lring.zero, mring.zero, nring.zero, oring.zero) + override def one = (aring.one, bring.one, cring.one, dring.one, ering.one, fring.one, gring.one, hring.one, iring.one, jring.one, kring.one, lring.one, mring.one, nring.one, oring.one) + override def negate(v : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O)) = (aring.negate(v._1), bring.negate(v._2), cring.negate(v._3), dring.negate(v._4), ering.negate(v._5), fring.negate(v._6), gring.negate(v._7), hring.negate(v._8), iring.negate(v._9), jring.negate(v._10), kring.negate(v._11), lring.negate(v._12), mring.negate(v._13), nring.negate(v._14), oring.negate(v._15)) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O)) = (aring.plus(l._1, r._1), bring.plus(l._2, r._2), cring.plus(l._3, r._3), dring.plus(l._4, r._4), ering.plus(l._5, r._5), fring.plus(l._6, r._6), gring.plus(l._7, r._7), hring.plus(l._8, r._8), iring.plus(l._9, r._9), jring.plus(l._10, r._10), kring.plus(l._11, r._11), lring.plus(l._12, r._12), mring.plus(l._13, r._13), nring.plus(l._14, r._14), oring.plus(l._15, r._15)) + override def minus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O)) = (aring.minus(l._1, r._1), bring.minus(l._2, r._2), cring.minus(l._3, r._3), dring.minus(l._4, r._4), ering.minus(l._5, r._5), fring.minus(l._6, r._6), gring.minus(l._7, r._7), hring.minus(l._8, r._8), iring.minus(l._9, r._9), jring.minus(l._10, r._10), kring.minus(l._11, r._11), lring.minus(l._12, r._12), mring.minus(l._13, r._13), nring.minus(l._14, r._14), oring.minus(l._15, r._15)) + override def times(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O)) = (aring.times(l._1, r._1), bring.times(l._2, r._2), cring.times(l._3, r._3), dring.times(l._4, r._4), ering.times(l._5, r._5), fring.times(l._6, r._6), gring.times(l._7, r._7), hring.times(l._8, r._8), iring.times(l._9, r._9), jring.times(l._10, r._10), kring.times(l._11, r._11), lring.times(l._12, r._12), mring.times(l._13, r._13), nring.times(l._14, r._14), oring.times(l._15, r._15)) +} +/** +* Combine 16 monoids into a product monoid +*/ +class Tuple16Monoid[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F], gmonoid : Monoid[G], hmonoid : Monoid[H], imonoid : Monoid[I], jmonoid : Monoid[J], kmonoid : Monoid[K], lmonoid : Monoid[L], mmonoid : Monoid[M], nmonoid : Monoid[N], omonoid : Monoid[O], pmonoid : Monoid[P]) extends Monoid[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P)] { + override def zero = (amonoid.zero, bmonoid.zero, cmonoid.zero, dmonoid.zero, emonoid.zero, fmonoid.zero, gmonoid.zero, hmonoid.zero, imonoid.zero, jmonoid.zero, kmonoid.zero, lmonoid.zero, mmonoid.zero, nmonoid.zero, omonoid.zero, pmonoid.zero) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P)) = (amonoid.plus(l._1, r._1), bmonoid.plus(l._2, r._2), cmonoid.plus(l._3, r._3), dmonoid.plus(l._4, r._4), emonoid.plus(l._5, r._5), fmonoid.plus(l._6, r._6), gmonoid.plus(l._7, r._7), hmonoid.plus(l._8, r._8), imonoid.plus(l._9, r._9), jmonoid.plus(l._10, r._10), kmonoid.plus(l._11, r._11), lmonoid.plus(l._12, r._12), mmonoid.plus(l._13, r._13), nmonoid.plus(l._14, r._14), omonoid.plus(l._15, r._15), pmonoid.plus(l._16, r._16)) +} + +/** +* Combine 16 groups into a product group +*/ +class Tuple16Group[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F], ggroup : Group[G], hgroup : Group[H], igroup : Group[I], jgroup : Group[J], kgroup : Group[K], lgroup : Group[L], mgroup : Group[M], ngroup : Group[N], ogroup : Group[O], pgroup : Group[P]) extends Group[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P)] { + override def zero = (agroup.zero, bgroup.zero, cgroup.zero, dgroup.zero, egroup.zero, fgroup.zero, ggroup.zero, hgroup.zero, igroup.zero, jgroup.zero, kgroup.zero, lgroup.zero, mgroup.zero, ngroup.zero, ogroup.zero, pgroup.zero) + override def negate(v : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P)) = (agroup.negate(v._1), bgroup.negate(v._2), cgroup.negate(v._3), dgroup.negate(v._4), egroup.negate(v._5), fgroup.negate(v._6), ggroup.negate(v._7), hgroup.negate(v._8), igroup.negate(v._9), jgroup.negate(v._10), kgroup.negate(v._11), lgroup.negate(v._12), mgroup.negate(v._13), ngroup.negate(v._14), ogroup.negate(v._15), pgroup.negate(v._16)) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P)) = (agroup.plus(l._1, r._1), bgroup.plus(l._2, r._2), cgroup.plus(l._3, r._3), dgroup.plus(l._4, r._4), egroup.plus(l._5, r._5), fgroup.plus(l._6, r._6), ggroup.plus(l._7, r._7), hgroup.plus(l._8, r._8), igroup.plus(l._9, r._9), jgroup.plus(l._10, r._10), kgroup.plus(l._11, r._11), lgroup.plus(l._12, r._12), mgroup.plus(l._13, r._13), ngroup.plus(l._14, r._14), ogroup.plus(l._15, r._15), pgroup.plus(l._16, r._16)) + override def minus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P)) = (agroup.minus(l._1, r._1), bgroup.minus(l._2, r._2), cgroup.minus(l._3, r._3), dgroup.minus(l._4, r._4), egroup.minus(l._5, r._5), fgroup.minus(l._6, r._6), ggroup.minus(l._7, r._7), hgroup.minus(l._8, r._8), igroup.minus(l._9, r._9), jgroup.minus(l._10, r._10), kgroup.minus(l._11, r._11), lgroup.minus(l._12, r._12), mgroup.minus(l._13, r._13), ngroup.minus(l._14, r._14), ogroup.minus(l._15, r._15), pgroup.minus(l._16, r._16)) +} + +/** +* Combine 16 rings into a product ring +*/ +class Tuple16Ring[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E], fring : Ring[F], gring : Ring[G], hring : Ring[H], iring : Ring[I], jring : Ring[J], kring : Ring[K], lring : Ring[L], mring : Ring[M], nring : Ring[N], oring : Ring[O], pring : Ring[P]) extends Ring[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P)] { + override def zero = (aring.zero, bring.zero, cring.zero, dring.zero, ering.zero, fring.zero, gring.zero, hring.zero, iring.zero, jring.zero, kring.zero, lring.zero, mring.zero, nring.zero, oring.zero, pring.zero) + override def one = (aring.one, bring.one, cring.one, dring.one, ering.one, fring.one, gring.one, hring.one, iring.one, jring.one, kring.one, lring.one, mring.one, nring.one, oring.one, pring.one) + override def negate(v : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P)) = (aring.negate(v._1), bring.negate(v._2), cring.negate(v._3), dring.negate(v._4), ering.negate(v._5), fring.negate(v._6), gring.negate(v._7), hring.negate(v._8), iring.negate(v._9), jring.negate(v._10), kring.negate(v._11), lring.negate(v._12), mring.negate(v._13), nring.negate(v._14), oring.negate(v._15), pring.negate(v._16)) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P)) = (aring.plus(l._1, r._1), bring.plus(l._2, r._2), cring.plus(l._3, r._3), dring.plus(l._4, r._4), ering.plus(l._5, r._5), fring.plus(l._6, r._6), gring.plus(l._7, r._7), hring.plus(l._8, r._8), iring.plus(l._9, r._9), jring.plus(l._10, r._10), kring.plus(l._11, r._11), lring.plus(l._12, r._12), mring.plus(l._13, r._13), nring.plus(l._14, r._14), oring.plus(l._15, r._15), pring.plus(l._16, r._16)) + override def minus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P)) = (aring.minus(l._1, r._1), bring.minus(l._2, r._2), cring.minus(l._3, r._3), dring.minus(l._4, r._4), ering.minus(l._5, r._5), fring.minus(l._6, r._6), gring.minus(l._7, r._7), hring.minus(l._8, r._8), iring.minus(l._9, r._9), jring.minus(l._10, r._10), kring.minus(l._11, r._11), lring.minus(l._12, r._12), mring.minus(l._13, r._13), nring.minus(l._14, r._14), oring.minus(l._15, r._15), pring.minus(l._16, r._16)) + override def times(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P)) = (aring.times(l._1, r._1), bring.times(l._2, r._2), cring.times(l._3, r._3), dring.times(l._4, r._4), ering.times(l._5, r._5), fring.times(l._6, r._6), gring.times(l._7, r._7), hring.times(l._8, r._8), iring.times(l._9, r._9), jring.times(l._10, r._10), kring.times(l._11, r._11), lring.times(l._12, r._12), mring.times(l._13, r._13), nring.times(l._14, r._14), oring.times(l._15, r._15), pring.times(l._16, r._16)) +} +/** +* Combine 17 monoids into a product monoid +*/ +class Tuple17Monoid[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F], gmonoid : Monoid[G], hmonoid : Monoid[H], imonoid : Monoid[I], jmonoid : Monoid[J], kmonoid : Monoid[K], lmonoid : Monoid[L], mmonoid : Monoid[M], nmonoid : Monoid[N], omonoid : Monoid[O], pmonoid : Monoid[P], qmonoid : Monoid[Q]) extends Monoid[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q)] { + override def zero = (amonoid.zero, bmonoid.zero, cmonoid.zero, dmonoid.zero, emonoid.zero, fmonoid.zero, gmonoid.zero, hmonoid.zero, imonoid.zero, jmonoid.zero, kmonoid.zero, lmonoid.zero, mmonoid.zero, nmonoid.zero, omonoid.zero, pmonoid.zero, qmonoid.zero) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q)) = (amonoid.plus(l._1, r._1), bmonoid.plus(l._2, r._2), cmonoid.plus(l._3, r._3), dmonoid.plus(l._4, r._4), emonoid.plus(l._5, r._5), fmonoid.plus(l._6, r._6), gmonoid.plus(l._7, r._7), hmonoid.plus(l._8, r._8), imonoid.plus(l._9, r._9), jmonoid.plus(l._10, r._10), kmonoid.plus(l._11, r._11), lmonoid.plus(l._12, r._12), mmonoid.plus(l._13, r._13), nmonoid.plus(l._14, r._14), omonoid.plus(l._15, r._15), pmonoid.plus(l._16, r._16), qmonoid.plus(l._17, r._17)) +} + +/** +* Combine 17 groups into a product group +*/ +class Tuple17Group[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F], ggroup : Group[G], hgroup : Group[H], igroup : Group[I], jgroup : Group[J], kgroup : Group[K], lgroup : Group[L], mgroup : Group[M], ngroup : Group[N], ogroup : Group[O], pgroup : Group[P], qgroup : Group[Q]) extends Group[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q)] { + override def zero = (agroup.zero, bgroup.zero, cgroup.zero, dgroup.zero, egroup.zero, fgroup.zero, ggroup.zero, hgroup.zero, igroup.zero, jgroup.zero, kgroup.zero, lgroup.zero, mgroup.zero, ngroup.zero, ogroup.zero, pgroup.zero, qgroup.zero) + override def negate(v : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q)) = (agroup.negate(v._1), bgroup.negate(v._2), cgroup.negate(v._3), dgroup.negate(v._4), egroup.negate(v._5), fgroup.negate(v._6), ggroup.negate(v._7), hgroup.negate(v._8), igroup.negate(v._9), jgroup.negate(v._10), kgroup.negate(v._11), lgroup.negate(v._12), mgroup.negate(v._13), ngroup.negate(v._14), ogroup.negate(v._15), pgroup.negate(v._16), qgroup.negate(v._17)) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q)) = (agroup.plus(l._1, r._1), bgroup.plus(l._2, r._2), cgroup.plus(l._3, r._3), dgroup.plus(l._4, r._4), egroup.plus(l._5, r._5), fgroup.plus(l._6, r._6), ggroup.plus(l._7, r._7), hgroup.plus(l._8, r._8), igroup.plus(l._9, r._9), jgroup.plus(l._10, r._10), kgroup.plus(l._11, r._11), lgroup.plus(l._12, r._12), mgroup.plus(l._13, r._13), ngroup.plus(l._14, r._14), ogroup.plus(l._15, r._15), pgroup.plus(l._16, r._16), qgroup.plus(l._17, r._17)) + override def minus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q)) = (agroup.minus(l._1, r._1), bgroup.minus(l._2, r._2), cgroup.minus(l._3, r._3), dgroup.minus(l._4, r._4), egroup.minus(l._5, r._5), fgroup.minus(l._6, r._6), ggroup.minus(l._7, r._7), hgroup.minus(l._8, r._8), igroup.minus(l._9, r._9), jgroup.minus(l._10, r._10), kgroup.minus(l._11, r._11), lgroup.minus(l._12, r._12), mgroup.minus(l._13, r._13), ngroup.minus(l._14, r._14), ogroup.minus(l._15, r._15), pgroup.minus(l._16, r._16), qgroup.minus(l._17, r._17)) +} + +/** +* Combine 17 rings into a product ring +*/ +class Tuple17Ring[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E], fring : Ring[F], gring : Ring[G], hring : Ring[H], iring : Ring[I], jring : Ring[J], kring : Ring[K], lring : Ring[L], mring : Ring[M], nring : Ring[N], oring : Ring[O], pring : Ring[P], qring : Ring[Q]) extends Ring[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q)] { + override def zero = (aring.zero, bring.zero, cring.zero, dring.zero, ering.zero, fring.zero, gring.zero, hring.zero, iring.zero, jring.zero, kring.zero, lring.zero, mring.zero, nring.zero, oring.zero, pring.zero, qring.zero) + override def one = (aring.one, bring.one, cring.one, dring.one, ering.one, fring.one, gring.one, hring.one, iring.one, jring.one, kring.one, lring.one, mring.one, nring.one, oring.one, pring.one, qring.one) + override def negate(v : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q)) = (aring.negate(v._1), bring.negate(v._2), cring.negate(v._3), dring.negate(v._4), ering.negate(v._5), fring.negate(v._6), gring.negate(v._7), hring.negate(v._8), iring.negate(v._9), jring.negate(v._10), kring.negate(v._11), lring.negate(v._12), mring.negate(v._13), nring.negate(v._14), oring.negate(v._15), pring.negate(v._16), qring.negate(v._17)) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q)) = (aring.plus(l._1, r._1), bring.plus(l._2, r._2), cring.plus(l._3, r._3), dring.plus(l._4, r._4), ering.plus(l._5, r._5), fring.plus(l._6, r._6), gring.plus(l._7, r._7), hring.plus(l._8, r._8), iring.plus(l._9, r._9), jring.plus(l._10, r._10), kring.plus(l._11, r._11), lring.plus(l._12, r._12), mring.plus(l._13, r._13), nring.plus(l._14, r._14), oring.plus(l._15, r._15), pring.plus(l._16, r._16), qring.plus(l._17, r._17)) + override def minus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q)) = (aring.minus(l._1, r._1), bring.minus(l._2, r._2), cring.minus(l._3, r._3), dring.minus(l._4, r._4), ering.minus(l._5, r._5), fring.minus(l._6, r._6), gring.minus(l._7, r._7), hring.minus(l._8, r._8), iring.minus(l._9, r._9), jring.minus(l._10, r._10), kring.minus(l._11, r._11), lring.minus(l._12, r._12), mring.minus(l._13, r._13), nring.minus(l._14, r._14), oring.minus(l._15, r._15), pring.minus(l._16, r._16), qring.minus(l._17, r._17)) + override def times(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q)) = (aring.times(l._1, r._1), bring.times(l._2, r._2), cring.times(l._3, r._3), dring.times(l._4, r._4), ering.times(l._5, r._5), fring.times(l._6, r._6), gring.times(l._7, r._7), hring.times(l._8, r._8), iring.times(l._9, r._9), jring.times(l._10, r._10), kring.times(l._11, r._11), lring.times(l._12, r._12), mring.times(l._13, r._13), nring.times(l._14, r._14), oring.times(l._15, r._15), pring.times(l._16, r._16), qring.times(l._17, r._17)) +} +/** +* Combine 18 monoids into a product monoid +*/ +class Tuple18Monoid[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F], gmonoid : Monoid[G], hmonoid : Monoid[H], imonoid : Monoid[I], jmonoid : Monoid[J], kmonoid : Monoid[K], lmonoid : Monoid[L], mmonoid : Monoid[M], nmonoid : Monoid[N], omonoid : Monoid[O], pmonoid : Monoid[P], qmonoid : Monoid[Q], rmonoid : Monoid[R]) extends Monoid[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R)] { + override def zero = (amonoid.zero, bmonoid.zero, cmonoid.zero, dmonoid.zero, emonoid.zero, fmonoid.zero, gmonoid.zero, hmonoid.zero, imonoid.zero, jmonoid.zero, kmonoid.zero, lmonoid.zero, mmonoid.zero, nmonoid.zero, omonoid.zero, pmonoid.zero, qmonoid.zero, rmonoid.zero) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R)) = (amonoid.plus(l._1, r._1), bmonoid.plus(l._2, r._2), cmonoid.plus(l._3, r._3), dmonoid.plus(l._4, r._4), emonoid.plus(l._5, r._5), fmonoid.plus(l._6, r._6), gmonoid.plus(l._7, r._7), hmonoid.plus(l._8, r._8), imonoid.plus(l._9, r._9), jmonoid.plus(l._10, r._10), kmonoid.plus(l._11, r._11), lmonoid.plus(l._12, r._12), mmonoid.plus(l._13, r._13), nmonoid.plus(l._14, r._14), omonoid.plus(l._15, r._15), pmonoid.plus(l._16, r._16), qmonoid.plus(l._17, r._17), rmonoid.plus(l._18, r._18)) +} + +/** +* Combine 18 groups into a product group +*/ +class Tuple18Group[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F], ggroup : Group[G], hgroup : Group[H], igroup : Group[I], jgroup : Group[J], kgroup : Group[K], lgroup : Group[L], mgroup : Group[M], ngroup : Group[N], ogroup : Group[O], pgroup : Group[P], qgroup : Group[Q], rgroup : Group[R]) extends Group[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R)] { + override def zero = (agroup.zero, bgroup.zero, cgroup.zero, dgroup.zero, egroup.zero, fgroup.zero, ggroup.zero, hgroup.zero, igroup.zero, jgroup.zero, kgroup.zero, lgroup.zero, mgroup.zero, ngroup.zero, ogroup.zero, pgroup.zero, qgroup.zero, rgroup.zero) + override def negate(v : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R)) = (agroup.negate(v._1), bgroup.negate(v._2), cgroup.negate(v._3), dgroup.negate(v._4), egroup.negate(v._5), fgroup.negate(v._6), ggroup.negate(v._7), hgroup.negate(v._8), igroup.negate(v._9), jgroup.negate(v._10), kgroup.negate(v._11), lgroup.negate(v._12), mgroup.negate(v._13), ngroup.negate(v._14), ogroup.negate(v._15), pgroup.negate(v._16), qgroup.negate(v._17), rgroup.negate(v._18)) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R)) = (agroup.plus(l._1, r._1), bgroup.plus(l._2, r._2), cgroup.plus(l._3, r._3), dgroup.plus(l._4, r._4), egroup.plus(l._5, r._5), fgroup.plus(l._6, r._6), ggroup.plus(l._7, r._7), hgroup.plus(l._8, r._8), igroup.plus(l._9, r._9), jgroup.plus(l._10, r._10), kgroup.plus(l._11, r._11), lgroup.plus(l._12, r._12), mgroup.plus(l._13, r._13), ngroup.plus(l._14, r._14), ogroup.plus(l._15, r._15), pgroup.plus(l._16, r._16), qgroup.plus(l._17, r._17), rgroup.plus(l._18, r._18)) + override def minus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R)) = (agroup.minus(l._1, r._1), bgroup.minus(l._2, r._2), cgroup.minus(l._3, r._3), dgroup.minus(l._4, r._4), egroup.minus(l._5, r._5), fgroup.minus(l._6, r._6), ggroup.minus(l._7, r._7), hgroup.minus(l._8, r._8), igroup.minus(l._9, r._9), jgroup.minus(l._10, r._10), kgroup.minus(l._11, r._11), lgroup.minus(l._12, r._12), mgroup.minus(l._13, r._13), ngroup.minus(l._14, r._14), ogroup.minus(l._15, r._15), pgroup.minus(l._16, r._16), qgroup.minus(l._17, r._17), rgroup.minus(l._18, r._18)) +} + +/** +* Combine 18 rings into a product ring +*/ +class Tuple18Ring[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E], fring : Ring[F], gring : Ring[G], hring : Ring[H], iring : Ring[I], jring : Ring[J], kring : Ring[K], lring : Ring[L], mring : Ring[M], nring : Ring[N], oring : Ring[O], pring : Ring[P], qring : Ring[Q], rring : Ring[R]) extends Ring[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R)] { + override def zero = (aring.zero, bring.zero, cring.zero, dring.zero, ering.zero, fring.zero, gring.zero, hring.zero, iring.zero, jring.zero, kring.zero, lring.zero, mring.zero, nring.zero, oring.zero, pring.zero, qring.zero, rring.zero) + override def one = (aring.one, bring.one, cring.one, dring.one, ering.one, fring.one, gring.one, hring.one, iring.one, jring.one, kring.one, lring.one, mring.one, nring.one, oring.one, pring.one, qring.one, rring.one) + override def negate(v : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R)) = (aring.negate(v._1), bring.negate(v._2), cring.negate(v._3), dring.negate(v._4), ering.negate(v._5), fring.negate(v._6), gring.negate(v._7), hring.negate(v._8), iring.negate(v._9), jring.negate(v._10), kring.negate(v._11), lring.negate(v._12), mring.negate(v._13), nring.negate(v._14), oring.negate(v._15), pring.negate(v._16), qring.negate(v._17), rring.negate(v._18)) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R)) = (aring.plus(l._1, r._1), bring.plus(l._2, r._2), cring.plus(l._3, r._3), dring.plus(l._4, r._4), ering.plus(l._5, r._5), fring.plus(l._6, r._6), gring.plus(l._7, r._7), hring.plus(l._8, r._8), iring.plus(l._9, r._9), jring.plus(l._10, r._10), kring.plus(l._11, r._11), lring.plus(l._12, r._12), mring.plus(l._13, r._13), nring.plus(l._14, r._14), oring.plus(l._15, r._15), pring.plus(l._16, r._16), qring.plus(l._17, r._17), rring.plus(l._18, r._18)) + override def minus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R)) = (aring.minus(l._1, r._1), bring.minus(l._2, r._2), cring.minus(l._3, r._3), dring.minus(l._4, r._4), ering.minus(l._5, r._5), fring.minus(l._6, r._6), gring.minus(l._7, r._7), hring.minus(l._8, r._8), iring.minus(l._9, r._9), jring.minus(l._10, r._10), kring.minus(l._11, r._11), lring.minus(l._12, r._12), mring.minus(l._13, r._13), nring.minus(l._14, r._14), oring.minus(l._15, r._15), pring.minus(l._16, r._16), qring.minus(l._17, r._17), rring.minus(l._18, r._18)) + override def times(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R)) = (aring.times(l._1, r._1), bring.times(l._2, r._2), cring.times(l._3, r._3), dring.times(l._4, r._4), ering.times(l._5, r._5), fring.times(l._6, r._6), gring.times(l._7, r._7), hring.times(l._8, r._8), iring.times(l._9, r._9), jring.times(l._10, r._10), kring.times(l._11, r._11), lring.times(l._12, r._12), mring.times(l._13, r._13), nring.times(l._14, r._14), oring.times(l._15, r._15), pring.times(l._16, r._16), qring.times(l._17, r._17), rring.times(l._18, r._18)) +} +/** +* Combine 19 monoids into a product monoid +*/ +class Tuple19Monoid[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F], gmonoid : Monoid[G], hmonoid : Monoid[H], imonoid : Monoid[I], jmonoid : Monoid[J], kmonoid : Monoid[K], lmonoid : Monoid[L], mmonoid : Monoid[M], nmonoid : Monoid[N], omonoid : Monoid[O], pmonoid : Monoid[P], qmonoid : Monoid[Q], rmonoid : Monoid[R], smonoid : Monoid[S]) extends Monoid[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S)] { + override def zero = (amonoid.zero, bmonoid.zero, cmonoid.zero, dmonoid.zero, emonoid.zero, fmonoid.zero, gmonoid.zero, hmonoid.zero, imonoid.zero, jmonoid.zero, kmonoid.zero, lmonoid.zero, mmonoid.zero, nmonoid.zero, omonoid.zero, pmonoid.zero, qmonoid.zero, rmonoid.zero, smonoid.zero) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S)) = (amonoid.plus(l._1, r._1), bmonoid.plus(l._2, r._2), cmonoid.plus(l._3, r._3), dmonoid.plus(l._4, r._4), emonoid.plus(l._5, r._5), fmonoid.plus(l._6, r._6), gmonoid.plus(l._7, r._7), hmonoid.plus(l._8, r._8), imonoid.plus(l._9, r._9), jmonoid.plus(l._10, r._10), kmonoid.plus(l._11, r._11), lmonoid.plus(l._12, r._12), mmonoid.plus(l._13, r._13), nmonoid.plus(l._14, r._14), omonoid.plus(l._15, r._15), pmonoid.plus(l._16, r._16), qmonoid.plus(l._17, r._17), rmonoid.plus(l._18, r._18), smonoid.plus(l._19, r._19)) +} + +/** +* Combine 19 groups into a product group +*/ +class Tuple19Group[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F], ggroup : Group[G], hgroup : Group[H], igroup : Group[I], jgroup : Group[J], kgroup : Group[K], lgroup : Group[L], mgroup : Group[M], ngroup : Group[N], ogroup : Group[O], pgroup : Group[P], qgroup : Group[Q], rgroup : Group[R], sgroup : Group[S]) extends Group[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S)] { + override def zero = (agroup.zero, bgroup.zero, cgroup.zero, dgroup.zero, egroup.zero, fgroup.zero, ggroup.zero, hgroup.zero, igroup.zero, jgroup.zero, kgroup.zero, lgroup.zero, mgroup.zero, ngroup.zero, ogroup.zero, pgroup.zero, qgroup.zero, rgroup.zero, sgroup.zero) + override def negate(v : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S)) = (agroup.negate(v._1), bgroup.negate(v._2), cgroup.negate(v._3), dgroup.negate(v._4), egroup.negate(v._5), fgroup.negate(v._6), ggroup.negate(v._7), hgroup.negate(v._8), igroup.negate(v._9), jgroup.negate(v._10), kgroup.negate(v._11), lgroup.negate(v._12), mgroup.negate(v._13), ngroup.negate(v._14), ogroup.negate(v._15), pgroup.negate(v._16), qgroup.negate(v._17), rgroup.negate(v._18), sgroup.negate(v._19)) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S)) = (agroup.plus(l._1, r._1), bgroup.plus(l._2, r._2), cgroup.plus(l._3, r._3), dgroup.plus(l._4, r._4), egroup.plus(l._5, r._5), fgroup.plus(l._6, r._6), ggroup.plus(l._7, r._7), hgroup.plus(l._8, r._8), igroup.plus(l._9, r._9), jgroup.plus(l._10, r._10), kgroup.plus(l._11, r._11), lgroup.plus(l._12, r._12), mgroup.plus(l._13, r._13), ngroup.plus(l._14, r._14), ogroup.plus(l._15, r._15), pgroup.plus(l._16, r._16), qgroup.plus(l._17, r._17), rgroup.plus(l._18, r._18), sgroup.plus(l._19, r._19)) + override def minus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S)) = (agroup.minus(l._1, r._1), bgroup.minus(l._2, r._2), cgroup.minus(l._3, r._3), dgroup.minus(l._4, r._4), egroup.minus(l._5, r._5), fgroup.minus(l._6, r._6), ggroup.minus(l._7, r._7), hgroup.minus(l._8, r._8), igroup.minus(l._9, r._9), jgroup.minus(l._10, r._10), kgroup.minus(l._11, r._11), lgroup.minus(l._12, r._12), mgroup.minus(l._13, r._13), ngroup.minus(l._14, r._14), ogroup.minus(l._15, r._15), pgroup.minus(l._16, r._16), qgroup.minus(l._17, r._17), rgroup.minus(l._18, r._18), sgroup.minus(l._19, r._19)) +} + +/** +* Combine 19 rings into a product ring +*/ +class Tuple19Ring[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E], fring : Ring[F], gring : Ring[G], hring : Ring[H], iring : Ring[I], jring : Ring[J], kring : Ring[K], lring : Ring[L], mring : Ring[M], nring : Ring[N], oring : Ring[O], pring : Ring[P], qring : Ring[Q], rring : Ring[R], sring : Ring[S]) extends Ring[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S)] { + override def zero = (aring.zero, bring.zero, cring.zero, dring.zero, ering.zero, fring.zero, gring.zero, hring.zero, iring.zero, jring.zero, kring.zero, lring.zero, mring.zero, nring.zero, oring.zero, pring.zero, qring.zero, rring.zero, sring.zero) + override def one = (aring.one, bring.one, cring.one, dring.one, ering.one, fring.one, gring.one, hring.one, iring.one, jring.one, kring.one, lring.one, mring.one, nring.one, oring.one, pring.one, qring.one, rring.one, sring.one) + override def negate(v : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S)) = (aring.negate(v._1), bring.negate(v._2), cring.negate(v._3), dring.negate(v._4), ering.negate(v._5), fring.negate(v._6), gring.negate(v._7), hring.negate(v._8), iring.negate(v._9), jring.negate(v._10), kring.negate(v._11), lring.negate(v._12), mring.negate(v._13), nring.negate(v._14), oring.negate(v._15), pring.negate(v._16), qring.negate(v._17), rring.negate(v._18), sring.negate(v._19)) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S)) = (aring.plus(l._1, r._1), bring.plus(l._2, r._2), cring.plus(l._3, r._3), dring.plus(l._4, r._4), ering.plus(l._5, r._5), fring.plus(l._6, r._6), gring.plus(l._7, r._7), hring.plus(l._8, r._8), iring.plus(l._9, r._9), jring.plus(l._10, r._10), kring.plus(l._11, r._11), lring.plus(l._12, r._12), mring.plus(l._13, r._13), nring.plus(l._14, r._14), oring.plus(l._15, r._15), pring.plus(l._16, r._16), qring.plus(l._17, r._17), rring.plus(l._18, r._18), sring.plus(l._19, r._19)) + override def minus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S)) = (aring.minus(l._1, r._1), bring.minus(l._2, r._2), cring.minus(l._3, r._3), dring.minus(l._4, r._4), ering.minus(l._5, r._5), fring.minus(l._6, r._6), gring.minus(l._7, r._7), hring.minus(l._8, r._8), iring.minus(l._9, r._9), jring.minus(l._10, r._10), kring.minus(l._11, r._11), lring.minus(l._12, r._12), mring.minus(l._13, r._13), nring.minus(l._14, r._14), oring.minus(l._15, r._15), pring.minus(l._16, r._16), qring.minus(l._17, r._17), rring.minus(l._18, r._18), sring.minus(l._19, r._19)) + override def times(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S)) = (aring.times(l._1, r._1), bring.times(l._2, r._2), cring.times(l._3, r._3), dring.times(l._4, r._4), ering.times(l._5, r._5), fring.times(l._6, r._6), gring.times(l._7, r._7), hring.times(l._8, r._8), iring.times(l._9, r._9), jring.times(l._10, r._10), kring.times(l._11, r._11), lring.times(l._12, r._12), mring.times(l._13, r._13), nring.times(l._14, r._14), oring.times(l._15, r._15), pring.times(l._16, r._16), qring.times(l._17, r._17), rring.times(l._18, r._18), sring.times(l._19, r._19)) +} +/** +* Combine 20 monoids into a product monoid +*/ +class Tuple20Monoid[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F], gmonoid : Monoid[G], hmonoid : Monoid[H], imonoid : Monoid[I], jmonoid : Monoid[J], kmonoid : Monoid[K], lmonoid : Monoid[L], mmonoid : Monoid[M], nmonoid : Monoid[N], omonoid : Monoid[O], pmonoid : Monoid[P], qmonoid : Monoid[Q], rmonoid : Monoid[R], smonoid : Monoid[S], tmonoid : Monoid[T]) extends Monoid[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T)] { + override def zero = (amonoid.zero, bmonoid.zero, cmonoid.zero, dmonoid.zero, emonoid.zero, fmonoid.zero, gmonoid.zero, hmonoid.zero, imonoid.zero, jmonoid.zero, kmonoid.zero, lmonoid.zero, mmonoid.zero, nmonoid.zero, omonoid.zero, pmonoid.zero, qmonoid.zero, rmonoid.zero, smonoid.zero, tmonoid.zero) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T)) = (amonoid.plus(l._1, r._1), bmonoid.plus(l._2, r._2), cmonoid.plus(l._3, r._3), dmonoid.plus(l._4, r._4), emonoid.plus(l._5, r._5), fmonoid.plus(l._6, r._6), gmonoid.plus(l._7, r._7), hmonoid.plus(l._8, r._8), imonoid.plus(l._9, r._9), jmonoid.plus(l._10, r._10), kmonoid.plus(l._11, r._11), lmonoid.plus(l._12, r._12), mmonoid.plus(l._13, r._13), nmonoid.plus(l._14, r._14), omonoid.plus(l._15, r._15), pmonoid.plus(l._16, r._16), qmonoid.plus(l._17, r._17), rmonoid.plus(l._18, r._18), smonoid.plus(l._19, r._19), tmonoid.plus(l._20, r._20)) +} + +/** +* Combine 20 groups into a product group +*/ +class Tuple20Group[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F], ggroup : Group[G], hgroup : Group[H], igroup : Group[I], jgroup : Group[J], kgroup : Group[K], lgroup : Group[L], mgroup : Group[M], ngroup : Group[N], ogroup : Group[O], pgroup : Group[P], qgroup : Group[Q], rgroup : Group[R], sgroup : Group[S], tgroup : Group[T]) extends Group[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T)] { + override def zero = (agroup.zero, bgroup.zero, cgroup.zero, dgroup.zero, egroup.zero, fgroup.zero, ggroup.zero, hgroup.zero, igroup.zero, jgroup.zero, kgroup.zero, lgroup.zero, mgroup.zero, ngroup.zero, ogroup.zero, pgroup.zero, qgroup.zero, rgroup.zero, sgroup.zero, tgroup.zero) + override def negate(v : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T)) = (agroup.negate(v._1), bgroup.negate(v._2), cgroup.negate(v._3), dgroup.negate(v._4), egroup.negate(v._5), fgroup.negate(v._6), ggroup.negate(v._7), hgroup.negate(v._8), igroup.negate(v._9), jgroup.negate(v._10), kgroup.negate(v._11), lgroup.negate(v._12), mgroup.negate(v._13), ngroup.negate(v._14), ogroup.negate(v._15), pgroup.negate(v._16), qgroup.negate(v._17), rgroup.negate(v._18), sgroup.negate(v._19), tgroup.negate(v._20)) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T)) = (agroup.plus(l._1, r._1), bgroup.plus(l._2, r._2), cgroup.plus(l._3, r._3), dgroup.plus(l._4, r._4), egroup.plus(l._5, r._5), fgroup.plus(l._6, r._6), ggroup.plus(l._7, r._7), hgroup.plus(l._8, r._8), igroup.plus(l._9, r._9), jgroup.plus(l._10, r._10), kgroup.plus(l._11, r._11), lgroup.plus(l._12, r._12), mgroup.plus(l._13, r._13), ngroup.plus(l._14, r._14), ogroup.plus(l._15, r._15), pgroup.plus(l._16, r._16), qgroup.plus(l._17, r._17), rgroup.plus(l._18, r._18), sgroup.plus(l._19, r._19), tgroup.plus(l._20, r._20)) + override def minus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T)) = (agroup.minus(l._1, r._1), bgroup.minus(l._2, r._2), cgroup.minus(l._3, r._3), dgroup.minus(l._4, r._4), egroup.minus(l._5, r._5), fgroup.minus(l._6, r._6), ggroup.minus(l._7, r._7), hgroup.minus(l._8, r._8), igroup.minus(l._9, r._9), jgroup.minus(l._10, r._10), kgroup.minus(l._11, r._11), lgroup.minus(l._12, r._12), mgroup.minus(l._13, r._13), ngroup.minus(l._14, r._14), ogroup.minus(l._15, r._15), pgroup.minus(l._16, r._16), qgroup.minus(l._17, r._17), rgroup.minus(l._18, r._18), sgroup.minus(l._19, r._19), tgroup.minus(l._20, r._20)) +} + +/** +* Combine 20 rings into a product ring +*/ +class Tuple20Ring[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E], fring : Ring[F], gring : Ring[G], hring : Ring[H], iring : Ring[I], jring : Ring[J], kring : Ring[K], lring : Ring[L], mring : Ring[M], nring : Ring[N], oring : Ring[O], pring : Ring[P], qring : Ring[Q], rring : Ring[R], sring : Ring[S], tring : Ring[T]) extends Ring[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T)] { + override def zero = (aring.zero, bring.zero, cring.zero, dring.zero, ering.zero, fring.zero, gring.zero, hring.zero, iring.zero, jring.zero, kring.zero, lring.zero, mring.zero, nring.zero, oring.zero, pring.zero, qring.zero, rring.zero, sring.zero, tring.zero) + override def one = (aring.one, bring.one, cring.one, dring.one, ering.one, fring.one, gring.one, hring.one, iring.one, jring.one, kring.one, lring.one, mring.one, nring.one, oring.one, pring.one, qring.one, rring.one, sring.one, tring.one) + override def negate(v : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T)) = (aring.negate(v._1), bring.negate(v._2), cring.negate(v._3), dring.negate(v._4), ering.negate(v._5), fring.negate(v._6), gring.negate(v._7), hring.negate(v._8), iring.negate(v._9), jring.negate(v._10), kring.negate(v._11), lring.negate(v._12), mring.negate(v._13), nring.negate(v._14), oring.negate(v._15), pring.negate(v._16), qring.negate(v._17), rring.negate(v._18), sring.negate(v._19), tring.negate(v._20)) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T)) = (aring.plus(l._1, r._1), bring.plus(l._2, r._2), cring.plus(l._3, r._3), dring.plus(l._4, r._4), ering.plus(l._5, r._5), fring.plus(l._6, r._6), gring.plus(l._7, r._7), hring.plus(l._8, r._8), iring.plus(l._9, r._9), jring.plus(l._10, r._10), kring.plus(l._11, r._11), lring.plus(l._12, r._12), mring.plus(l._13, r._13), nring.plus(l._14, r._14), oring.plus(l._15, r._15), pring.plus(l._16, r._16), qring.plus(l._17, r._17), rring.plus(l._18, r._18), sring.plus(l._19, r._19), tring.plus(l._20, r._20)) + override def minus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T)) = (aring.minus(l._1, r._1), bring.minus(l._2, r._2), cring.minus(l._3, r._3), dring.minus(l._4, r._4), ering.minus(l._5, r._5), fring.minus(l._6, r._6), gring.minus(l._7, r._7), hring.minus(l._8, r._8), iring.minus(l._9, r._9), jring.minus(l._10, r._10), kring.minus(l._11, r._11), lring.minus(l._12, r._12), mring.minus(l._13, r._13), nring.minus(l._14, r._14), oring.minus(l._15, r._15), pring.minus(l._16, r._16), qring.minus(l._17, r._17), rring.minus(l._18, r._18), sring.minus(l._19, r._19), tring.minus(l._20, r._20)) + override def times(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T)) = (aring.times(l._1, r._1), bring.times(l._2, r._2), cring.times(l._3, r._3), dring.times(l._4, r._4), ering.times(l._5, r._5), fring.times(l._6, r._6), gring.times(l._7, r._7), hring.times(l._8, r._8), iring.times(l._9, r._9), jring.times(l._10, r._10), kring.times(l._11, r._11), lring.times(l._12, r._12), mring.times(l._13, r._13), nring.times(l._14, r._14), oring.times(l._15, r._15), pring.times(l._16, r._16), qring.times(l._17, r._17), rring.times(l._18, r._18), sring.times(l._19, r._19), tring.times(l._20, r._20)) +} +/** +* Combine 21 monoids into a product monoid +*/ +class Tuple21Monoid[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F], gmonoid : Monoid[G], hmonoid : Monoid[H], imonoid : Monoid[I], jmonoid : Monoid[J], kmonoid : Monoid[K], lmonoid : Monoid[L], mmonoid : Monoid[M], nmonoid : Monoid[N], omonoid : Monoid[O], pmonoid : Monoid[P], qmonoid : Monoid[Q], rmonoid : Monoid[R], smonoid : Monoid[S], tmonoid : Monoid[T], umonoid : Monoid[U]) extends Monoid[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U)] { + override def zero = (amonoid.zero, bmonoid.zero, cmonoid.zero, dmonoid.zero, emonoid.zero, fmonoid.zero, gmonoid.zero, hmonoid.zero, imonoid.zero, jmonoid.zero, kmonoid.zero, lmonoid.zero, mmonoid.zero, nmonoid.zero, omonoid.zero, pmonoid.zero, qmonoid.zero, rmonoid.zero, smonoid.zero, tmonoid.zero, umonoid.zero) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U)) = (amonoid.plus(l._1, r._1), bmonoid.plus(l._2, r._2), cmonoid.plus(l._3, r._3), dmonoid.plus(l._4, r._4), emonoid.plus(l._5, r._5), fmonoid.plus(l._6, r._6), gmonoid.plus(l._7, r._7), hmonoid.plus(l._8, r._8), imonoid.plus(l._9, r._9), jmonoid.plus(l._10, r._10), kmonoid.plus(l._11, r._11), lmonoid.plus(l._12, r._12), mmonoid.plus(l._13, r._13), nmonoid.plus(l._14, r._14), omonoid.plus(l._15, r._15), pmonoid.plus(l._16, r._16), qmonoid.plus(l._17, r._17), rmonoid.plus(l._18, r._18), smonoid.plus(l._19, r._19), tmonoid.plus(l._20, r._20), umonoid.plus(l._21, r._21)) +} + +/** +* Combine 21 groups into a product group +*/ +class Tuple21Group[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F], ggroup : Group[G], hgroup : Group[H], igroup : Group[I], jgroup : Group[J], kgroup : Group[K], lgroup : Group[L], mgroup : Group[M], ngroup : Group[N], ogroup : Group[O], pgroup : Group[P], qgroup : Group[Q], rgroup : Group[R], sgroup : Group[S], tgroup : Group[T], ugroup : Group[U]) extends Group[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U)] { + override def zero = (agroup.zero, bgroup.zero, cgroup.zero, dgroup.zero, egroup.zero, fgroup.zero, ggroup.zero, hgroup.zero, igroup.zero, jgroup.zero, kgroup.zero, lgroup.zero, mgroup.zero, ngroup.zero, ogroup.zero, pgroup.zero, qgroup.zero, rgroup.zero, sgroup.zero, tgroup.zero, ugroup.zero) + override def negate(v : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U)) = (agroup.negate(v._1), bgroup.negate(v._2), cgroup.negate(v._3), dgroup.negate(v._4), egroup.negate(v._5), fgroup.negate(v._6), ggroup.negate(v._7), hgroup.negate(v._8), igroup.negate(v._9), jgroup.negate(v._10), kgroup.negate(v._11), lgroup.negate(v._12), mgroup.negate(v._13), ngroup.negate(v._14), ogroup.negate(v._15), pgroup.negate(v._16), qgroup.negate(v._17), rgroup.negate(v._18), sgroup.negate(v._19), tgroup.negate(v._20), ugroup.negate(v._21)) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U)) = (agroup.plus(l._1, r._1), bgroup.plus(l._2, r._2), cgroup.plus(l._3, r._3), dgroup.plus(l._4, r._4), egroup.plus(l._5, r._5), fgroup.plus(l._6, r._6), ggroup.plus(l._7, r._7), hgroup.plus(l._8, r._8), igroup.plus(l._9, r._9), jgroup.plus(l._10, r._10), kgroup.plus(l._11, r._11), lgroup.plus(l._12, r._12), mgroup.plus(l._13, r._13), ngroup.plus(l._14, r._14), ogroup.plus(l._15, r._15), pgroup.plus(l._16, r._16), qgroup.plus(l._17, r._17), rgroup.plus(l._18, r._18), sgroup.plus(l._19, r._19), tgroup.plus(l._20, r._20), ugroup.plus(l._21, r._21)) + override def minus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U)) = (agroup.minus(l._1, r._1), bgroup.minus(l._2, r._2), cgroup.minus(l._3, r._3), dgroup.minus(l._4, r._4), egroup.minus(l._5, r._5), fgroup.minus(l._6, r._6), ggroup.minus(l._7, r._7), hgroup.minus(l._8, r._8), igroup.minus(l._9, r._9), jgroup.minus(l._10, r._10), kgroup.minus(l._11, r._11), lgroup.minus(l._12, r._12), mgroup.minus(l._13, r._13), ngroup.minus(l._14, r._14), ogroup.minus(l._15, r._15), pgroup.minus(l._16, r._16), qgroup.minus(l._17, r._17), rgroup.minus(l._18, r._18), sgroup.minus(l._19, r._19), tgroup.minus(l._20, r._20), ugroup.minus(l._21, r._21)) +} + +/** +* Combine 21 rings into a product ring +*/ +class Tuple21Ring[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E], fring : Ring[F], gring : Ring[G], hring : Ring[H], iring : Ring[I], jring : Ring[J], kring : Ring[K], lring : Ring[L], mring : Ring[M], nring : Ring[N], oring : Ring[O], pring : Ring[P], qring : Ring[Q], rring : Ring[R], sring : Ring[S], tring : Ring[T], uring : Ring[U]) extends Ring[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U)] { + override def zero = (aring.zero, bring.zero, cring.zero, dring.zero, ering.zero, fring.zero, gring.zero, hring.zero, iring.zero, jring.zero, kring.zero, lring.zero, mring.zero, nring.zero, oring.zero, pring.zero, qring.zero, rring.zero, sring.zero, tring.zero, uring.zero) + override def one = (aring.one, bring.one, cring.one, dring.one, ering.one, fring.one, gring.one, hring.one, iring.one, jring.one, kring.one, lring.one, mring.one, nring.one, oring.one, pring.one, qring.one, rring.one, sring.one, tring.one, uring.one) + override def negate(v : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U)) = (aring.negate(v._1), bring.negate(v._2), cring.negate(v._3), dring.negate(v._4), ering.negate(v._5), fring.negate(v._6), gring.negate(v._7), hring.negate(v._8), iring.negate(v._9), jring.negate(v._10), kring.negate(v._11), lring.negate(v._12), mring.negate(v._13), nring.negate(v._14), oring.negate(v._15), pring.negate(v._16), qring.negate(v._17), rring.negate(v._18), sring.negate(v._19), tring.negate(v._20), uring.negate(v._21)) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U)) = (aring.plus(l._1, r._1), bring.plus(l._2, r._2), cring.plus(l._3, r._3), dring.plus(l._4, r._4), ering.plus(l._5, r._5), fring.plus(l._6, r._6), gring.plus(l._7, r._7), hring.plus(l._8, r._8), iring.plus(l._9, r._9), jring.plus(l._10, r._10), kring.plus(l._11, r._11), lring.plus(l._12, r._12), mring.plus(l._13, r._13), nring.plus(l._14, r._14), oring.plus(l._15, r._15), pring.plus(l._16, r._16), qring.plus(l._17, r._17), rring.plus(l._18, r._18), sring.plus(l._19, r._19), tring.plus(l._20, r._20), uring.plus(l._21, r._21)) + override def minus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U)) = (aring.minus(l._1, r._1), bring.minus(l._2, r._2), cring.minus(l._3, r._3), dring.minus(l._4, r._4), ering.minus(l._5, r._5), fring.minus(l._6, r._6), gring.minus(l._7, r._7), hring.minus(l._8, r._8), iring.minus(l._9, r._9), jring.minus(l._10, r._10), kring.minus(l._11, r._11), lring.minus(l._12, r._12), mring.minus(l._13, r._13), nring.minus(l._14, r._14), oring.minus(l._15, r._15), pring.minus(l._16, r._16), qring.minus(l._17, r._17), rring.minus(l._18, r._18), sring.minus(l._19, r._19), tring.minus(l._20, r._20), uring.minus(l._21, r._21)) + override def times(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U)) = (aring.times(l._1, r._1), bring.times(l._2, r._2), cring.times(l._3, r._3), dring.times(l._4, r._4), ering.times(l._5, r._5), fring.times(l._6, r._6), gring.times(l._7, r._7), hring.times(l._8, r._8), iring.times(l._9, r._9), jring.times(l._10, r._10), kring.times(l._11, r._11), lring.times(l._12, r._12), mring.times(l._13, r._13), nring.times(l._14, r._14), oring.times(l._15, r._15), pring.times(l._16, r._16), qring.times(l._17, r._17), rring.times(l._18, r._18), sring.times(l._19, r._19), tring.times(l._20, r._20), uring.times(l._21, r._21)) +} +/** +* Combine 22 monoids into a product monoid +*/ +class Tuple22Monoid[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F], gmonoid : Monoid[G], hmonoid : Monoid[H], imonoid : Monoid[I], jmonoid : Monoid[J], kmonoid : Monoid[K], lmonoid : Monoid[L], mmonoid : Monoid[M], nmonoid : Monoid[N], omonoid : Monoid[O], pmonoid : Monoid[P], qmonoid : Monoid[Q], rmonoid : Monoid[R], smonoid : Monoid[S], tmonoid : Monoid[T], umonoid : Monoid[U], vmonoid : Monoid[V]) extends Monoid[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V)] { + override def zero = (amonoid.zero, bmonoid.zero, cmonoid.zero, dmonoid.zero, emonoid.zero, fmonoid.zero, gmonoid.zero, hmonoid.zero, imonoid.zero, jmonoid.zero, kmonoid.zero, lmonoid.zero, mmonoid.zero, nmonoid.zero, omonoid.zero, pmonoid.zero, qmonoid.zero, rmonoid.zero, smonoid.zero, tmonoid.zero, umonoid.zero, vmonoid.zero) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V)) = (amonoid.plus(l._1, r._1), bmonoid.plus(l._2, r._2), cmonoid.plus(l._3, r._3), dmonoid.plus(l._4, r._4), emonoid.plus(l._5, r._5), fmonoid.plus(l._6, r._6), gmonoid.plus(l._7, r._7), hmonoid.plus(l._8, r._8), imonoid.plus(l._9, r._9), jmonoid.plus(l._10, r._10), kmonoid.plus(l._11, r._11), lmonoid.plus(l._12, r._12), mmonoid.plus(l._13, r._13), nmonoid.plus(l._14, r._14), omonoid.plus(l._15, r._15), pmonoid.plus(l._16, r._16), qmonoid.plus(l._17, r._17), rmonoid.plus(l._18, r._18), smonoid.plus(l._19, r._19), tmonoid.plus(l._20, r._20), umonoid.plus(l._21, r._21), vmonoid.plus(l._22, r._22)) +} + +/** +* Combine 22 groups into a product group +*/ +class Tuple22Group[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F], ggroup : Group[G], hgroup : Group[H], igroup : Group[I], jgroup : Group[J], kgroup : Group[K], lgroup : Group[L], mgroup : Group[M], ngroup : Group[N], ogroup : Group[O], pgroup : Group[P], qgroup : Group[Q], rgroup : Group[R], sgroup : Group[S], tgroup : Group[T], ugroup : Group[U], vgroup : Group[V]) extends Group[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V)] { + override def zero = (agroup.zero, bgroup.zero, cgroup.zero, dgroup.zero, egroup.zero, fgroup.zero, ggroup.zero, hgroup.zero, igroup.zero, jgroup.zero, kgroup.zero, lgroup.zero, mgroup.zero, ngroup.zero, ogroup.zero, pgroup.zero, qgroup.zero, rgroup.zero, sgroup.zero, tgroup.zero, ugroup.zero, vgroup.zero) + override def negate(v : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V)) = (agroup.negate(v._1), bgroup.negate(v._2), cgroup.negate(v._3), dgroup.negate(v._4), egroup.negate(v._5), fgroup.negate(v._6), ggroup.negate(v._7), hgroup.negate(v._8), igroup.negate(v._9), jgroup.negate(v._10), kgroup.negate(v._11), lgroup.negate(v._12), mgroup.negate(v._13), ngroup.negate(v._14), ogroup.negate(v._15), pgroup.negate(v._16), qgroup.negate(v._17), rgroup.negate(v._18), sgroup.negate(v._19), tgroup.negate(v._20), ugroup.negate(v._21), vgroup.negate(v._22)) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V)) = (agroup.plus(l._1, r._1), bgroup.plus(l._2, r._2), cgroup.plus(l._3, r._3), dgroup.plus(l._4, r._4), egroup.plus(l._5, r._5), fgroup.plus(l._6, r._6), ggroup.plus(l._7, r._7), hgroup.plus(l._8, r._8), igroup.plus(l._9, r._9), jgroup.plus(l._10, r._10), kgroup.plus(l._11, r._11), lgroup.plus(l._12, r._12), mgroup.plus(l._13, r._13), ngroup.plus(l._14, r._14), ogroup.plus(l._15, r._15), pgroup.plus(l._16, r._16), qgroup.plus(l._17, r._17), rgroup.plus(l._18, r._18), sgroup.plus(l._19, r._19), tgroup.plus(l._20, r._20), ugroup.plus(l._21, r._21), vgroup.plus(l._22, r._22)) + override def minus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V)) = (agroup.minus(l._1, r._1), bgroup.minus(l._2, r._2), cgroup.minus(l._3, r._3), dgroup.minus(l._4, r._4), egroup.minus(l._5, r._5), fgroup.minus(l._6, r._6), ggroup.minus(l._7, r._7), hgroup.minus(l._8, r._8), igroup.minus(l._9, r._9), jgroup.minus(l._10, r._10), kgroup.minus(l._11, r._11), lgroup.minus(l._12, r._12), mgroup.minus(l._13, r._13), ngroup.minus(l._14, r._14), ogroup.minus(l._15, r._15), pgroup.minus(l._16, r._16), qgroup.minus(l._17, r._17), rgroup.minus(l._18, r._18), sgroup.minus(l._19, r._19), tgroup.minus(l._20, r._20), ugroup.minus(l._21, r._21), vgroup.minus(l._22, r._22)) +} + +/** +* Combine 22 rings into a product ring +*/ +class Tuple22Ring[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E], fring : Ring[F], gring : Ring[G], hring : Ring[H], iring : Ring[I], jring : Ring[J], kring : Ring[K], lring : Ring[L], mring : Ring[M], nring : Ring[N], oring : Ring[O], pring : Ring[P], qring : Ring[Q], rring : Ring[R], sring : Ring[S], tring : Ring[T], uring : Ring[U], vring : Ring[V]) extends Ring[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V)] { + override def zero = (aring.zero, bring.zero, cring.zero, dring.zero, ering.zero, fring.zero, gring.zero, hring.zero, iring.zero, jring.zero, kring.zero, lring.zero, mring.zero, nring.zero, oring.zero, pring.zero, qring.zero, rring.zero, sring.zero, tring.zero, uring.zero, vring.zero) + override def one = (aring.one, bring.one, cring.one, dring.one, ering.one, fring.one, gring.one, hring.one, iring.one, jring.one, kring.one, lring.one, mring.one, nring.one, oring.one, pring.one, qring.one, rring.one, sring.one, tring.one, uring.one, vring.one) + override def negate(v : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V)) = (aring.negate(v._1), bring.negate(v._2), cring.negate(v._3), dring.negate(v._4), ering.negate(v._5), fring.negate(v._6), gring.negate(v._7), hring.negate(v._8), iring.negate(v._9), jring.negate(v._10), kring.negate(v._11), lring.negate(v._12), mring.negate(v._13), nring.negate(v._14), oring.negate(v._15), pring.negate(v._16), qring.negate(v._17), rring.negate(v._18), sring.negate(v._19), tring.negate(v._20), uring.negate(v._21), vring.negate(v._22)) + override def plus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V)) = (aring.plus(l._1, r._1), bring.plus(l._2, r._2), cring.plus(l._3, r._3), dring.plus(l._4, r._4), ering.plus(l._5, r._5), fring.plus(l._6, r._6), gring.plus(l._7, r._7), hring.plus(l._8, r._8), iring.plus(l._9, r._9), jring.plus(l._10, r._10), kring.plus(l._11, r._11), lring.plus(l._12, r._12), mring.plus(l._13, r._13), nring.plus(l._14, r._14), oring.plus(l._15, r._15), pring.plus(l._16, r._16), qring.plus(l._17, r._17), rring.plus(l._18, r._18), sring.plus(l._19, r._19), tring.plus(l._20, r._20), uring.plus(l._21, r._21), vring.plus(l._22, r._22)) + override def minus(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V)) = (aring.minus(l._1, r._1), bring.minus(l._2, r._2), cring.minus(l._3, r._3), dring.minus(l._4, r._4), ering.minus(l._5, r._5), fring.minus(l._6, r._6), gring.minus(l._7, r._7), hring.minus(l._8, r._8), iring.minus(l._9, r._9), jring.minus(l._10, r._10), kring.minus(l._11, r._11), lring.minus(l._12, r._12), mring.minus(l._13, r._13), nring.minus(l._14, r._14), oring.minus(l._15, r._15), pring.minus(l._16, r._16), qring.minus(l._17, r._17), rring.minus(l._18, r._18), sring.minus(l._19, r._19), tring.minus(l._20, r._20), uring.minus(l._21, r._21), vring.minus(l._22, r._22)) + override def times(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V)) = (aring.times(l._1, r._1), bring.times(l._2, r._2), cring.times(l._3, r._3), dring.times(l._4, r._4), ering.times(l._5, r._5), fring.times(l._6, r._6), gring.times(l._7, r._7), hring.times(l._8, r._8), iring.times(l._9, r._9), jring.times(l._10, r._10), kring.times(l._11, r._11), lring.times(l._12, r._12), mring.times(l._13, r._13), nring.times(l._14, r._14), oring.times(l._15, r._15), pring.times(l._16, r._16), qring.times(l._17, r._17), rring.times(l._18, r._18), sring.times(l._19, r._19), tring.times(l._20, r._20), uring.times(l._21, r._21), vring.times(l._22, r._22)) +} + +// Here we complete the typeclasses by added the implicits to the companions: + +object Monoid { + implicit val boolMonoid : Monoid[Boolean] = BooleanField + implicit val intMonoid : Monoid[Int] = IntRing + implicit val longMonoid : Monoid[Long] = LongRing + implicit val floatMonoid : Monoid[Float] = FloatField + implicit val doubleMonoid : Monoid[Double] = DoubleField + implicit def listMonoid[T] : Monoid[List[T]] = new ListMonoid[T] + implicit def setMonoid[T] : Monoid[Set[T]] = new SetMonoid[T] + implicit def mapMonoid[K,V](implicit monoid : Monoid[V]) = new MapMonoid[K,V]()(monoid) + + implicit def pairMonoid[T,U](implicit tg : Monoid[T], ug : Monoid[U]) : Monoid[(T,U)] = { + new Tuple2Monoid[T,U]()(tg,ug) + } + implicit def monoid3[A, B, C](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C]) : Monoid[(A, B, C)] = { + new Tuple3Monoid[A, B, C]()(amonoid, bmonoid, cmonoid) + } + + implicit def monoid4[A, B, C, D](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D]) : Monoid[(A, B, C, D)] = { + new Tuple4Monoid[A, B, C, D]()(amonoid, bmonoid, cmonoid, dmonoid) + } + + implicit def monoid5[A, B, C, D, E](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E]) : Monoid[(A, B, C, D, E)] = { + new Tuple5Monoid[A, B, C, D, E]()(amonoid, bmonoid, cmonoid, dmonoid, emonoid) + } + + implicit def monoid6[A, B, C, D, E, F](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F]) : Monoid[(A, B, C, D, E, F)] = { + new Tuple6Monoid[A, B, C, D, E, F]()(amonoid, bmonoid, cmonoid, dmonoid, emonoid, fmonoid) + } + + implicit def monoid7[A, B, C, D, E, F, G](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F], gmonoid : Monoid[G]) : Monoid[(A, B, C, D, E, F, G)] = { + new Tuple7Monoid[A, B, C, D, E, F, G]()(amonoid, bmonoid, cmonoid, dmonoid, emonoid, fmonoid, gmonoid) + } + + implicit def monoid8[A, B, C, D, E, F, G, H](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F], gmonoid : Monoid[G], hmonoid : Monoid[H]) : Monoid[(A, B, C, D, E, F, G, H)] = { + new Tuple8Monoid[A, B, C, D, E, F, G, H]()(amonoid, bmonoid, cmonoid, dmonoid, emonoid, fmonoid, gmonoid, hmonoid) + } + + implicit def monoid9[A, B, C, D, E, F, G, H, I](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F], gmonoid : Monoid[G], hmonoid : Monoid[H], imonoid : Monoid[I]) : Monoid[(A, B, C, D, E, F, G, H, I)] = { + new Tuple9Monoid[A, B, C, D, E, F, G, H, I]()(amonoid, bmonoid, cmonoid, dmonoid, emonoid, fmonoid, gmonoid, hmonoid, imonoid) + } + + implicit def monoid10[A, B, C, D, E, F, G, H, I, J](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F], gmonoid : Monoid[G], hmonoid : Monoid[H], imonoid : Monoid[I], jmonoid : Monoid[J]) : Monoid[(A, B, C, D, E, F, G, H, I, J)] = { + new Tuple10Monoid[A, B, C, D, E, F, G, H, I, J]()(amonoid, bmonoid, cmonoid, dmonoid, emonoid, fmonoid, gmonoid, hmonoid, imonoid, jmonoid) + } + + implicit def monoid11[A, B, C, D, E, F, G, H, I, J, K](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F], gmonoid : Monoid[G], hmonoid : Monoid[H], imonoid : Monoid[I], jmonoid : Monoid[J], kmonoid : Monoid[K]) : Monoid[(A, B, C, D, E, F, G, H, I, J, K)] = { + new Tuple11Monoid[A, B, C, D, E, F, G, H, I, J, K]()(amonoid, bmonoid, cmonoid, dmonoid, emonoid, fmonoid, gmonoid, hmonoid, imonoid, jmonoid, kmonoid) + } + + implicit def monoid12[A, B, C, D, E, F, G, H, I, J, K, L](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F], gmonoid : Monoid[G], hmonoid : Monoid[H], imonoid : Monoid[I], jmonoid : Monoid[J], kmonoid : Monoid[K], lmonoid : Monoid[L]) : Monoid[(A, B, C, D, E, F, G, H, I, J, K, L)] = { + new Tuple12Monoid[A, B, C, D, E, F, G, H, I, J, K, L]()(amonoid, bmonoid, cmonoid, dmonoid, emonoid, fmonoid, gmonoid, hmonoid, imonoid, jmonoid, kmonoid, lmonoid) + } + + implicit def monoid13[A, B, C, D, E, F, G, H, I, J, K, L, M](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F], gmonoid : Monoid[G], hmonoid : Monoid[H], imonoid : Monoid[I], jmonoid : Monoid[J], kmonoid : Monoid[K], lmonoid : Monoid[L], mmonoid : Monoid[M]) : Monoid[(A, B, C, D, E, F, G, H, I, J, K, L, M)] = { + new Tuple13Monoid[A, B, C, D, E, F, G, H, I, J, K, L, M]()(amonoid, bmonoid, cmonoid, dmonoid, emonoid, fmonoid, gmonoid, hmonoid, imonoid, jmonoid, kmonoid, lmonoid, mmonoid) + } + + implicit def monoid14[A, B, C, D, E, F, G, H, I, J, K, L, M, N](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F], gmonoid : Monoid[G], hmonoid : Monoid[H], imonoid : Monoid[I], jmonoid : Monoid[J], kmonoid : Monoid[K], lmonoid : Monoid[L], mmonoid : Monoid[M], nmonoid : Monoid[N]) : Monoid[(A, B, C, D, E, F, G, H, I, J, K, L, M, N)] = { + new Tuple14Monoid[A, B, C, D, E, F, G, H, I, J, K, L, M, N]()(amonoid, bmonoid, cmonoid, dmonoid, emonoid, fmonoid, gmonoid, hmonoid, imonoid, jmonoid, kmonoid, lmonoid, mmonoid, nmonoid) + } + + implicit def monoid15[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F], gmonoid : Monoid[G], hmonoid : Monoid[H], imonoid : Monoid[I], jmonoid : Monoid[J], kmonoid : Monoid[K], lmonoid : Monoid[L], mmonoid : Monoid[M], nmonoid : Monoid[N], omonoid : Monoid[O]) : Monoid[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O)] = { + new Tuple15Monoid[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O]()(amonoid, bmonoid, cmonoid, dmonoid, emonoid, fmonoid, gmonoid, hmonoid, imonoid, jmonoid, kmonoid, lmonoid, mmonoid, nmonoid, omonoid) + } + + implicit def monoid16[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F], gmonoid : Monoid[G], hmonoid : Monoid[H], imonoid : Monoid[I], jmonoid : Monoid[J], kmonoid : Monoid[K], lmonoid : Monoid[L], mmonoid : Monoid[M], nmonoid : Monoid[N], omonoid : Monoid[O], pmonoid : Monoid[P]) : Monoid[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P)] = { + new Tuple16Monoid[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P]()(amonoid, bmonoid, cmonoid, dmonoid, emonoid, fmonoid, gmonoid, hmonoid, imonoid, jmonoid, kmonoid, lmonoid, mmonoid, nmonoid, omonoid, pmonoid) + } + + implicit def monoid17[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F], gmonoid : Monoid[G], hmonoid : Monoid[H], imonoid : Monoid[I], jmonoid : Monoid[J], kmonoid : Monoid[K], lmonoid : Monoid[L], mmonoid : Monoid[M], nmonoid : Monoid[N], omonoid : Monoid[O], pmonoid : Monoid[P], qmonoid : Monoid[Q]) : Monoid[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q)] = { + new Tuple17Monoid[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q]()(amonoid, bmonoid, cmonoid, dmonoid, emonoid, fmonoid, gmonoid, hmonoid, imonoid, jmonoid, kmonoid, lmonoid, mmonoid, nmonoid, omonoid, pmonoid, qmonoid) + } + + implicit def monoid18[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F], gmonoid : Monoid[G], hmonoid : Monoid[H], imonoid : Monoid[I], jmonoid : Monoid[J], kmonoid : Monoid[K], lmonoid : Monoid[L], mmonoid : Monoid[M], nmonoid : Monoid[N], omonoid : Monoid[O], pmonoid : Monoid[P], qmonoid : Monoid[Q], rmonoid : Monoid[R]) : Monoid[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R)] = { + new Tuple18Monoid[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R]()(amonoid, bmonoid, cmonoid, dmonoid, emonoid, fmonoid, gmonoid, hmonoid, imonoid, jmonoid, kmonoid, lmonoid, mmonoid, nmonoid, omonoid, pmonoid, qmonoid, rmonoid) + } + + implicit def monoid19[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F], gmonoid : Monoid[G], hmonoid : Monoid[H], imonoid : Monoid[I], jmonoid : Monoid[J], kmonoid : Monoid[K], lmonoid : Monoid[L], mmonoid : Monoid[M], nmonoid : Monoid[N], omonoid : Monoid[O], pmonoid : Monoid[P], qmonoid : Monoid[Q], rmonoid : Monoid[R], smonoid : Monoid[S]) : Monoid[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S)] = { + new Tuple19Monoid[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S]()(amonoid, bmonoid, cmonoid, dmonoid, emonoid, fmonoid, gmonoid, hmonoid, imonoid, jmonoid, kmonoid, lmonoid, mmonoid, nmonoid, omonoid, pmonoid, qmonoid, rmonoid, smonoid) + } + + implicit def monoid20[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F], gmonoid : Monoid[G], hmonoid : Monoid[H], imonoid : Monoid[I], jmonoid : Monoid[J], kmonoid : Monoid[K], lmonoid : Monoid[L], mmonoid : Monoid[M], nmonoid : Monoid[N], omonoid : Monoid[O], pmonoid : Monoid[P], qmonoid : Monoid[Q], rmonoid : Monoid[R], smonoid : Monoid[S], tmonoid : Monoid[T]) : Monoid[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T)] = { + new Tuple20Monoid[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T]()(amonoid, bmonoid, cmonoid, dmonoid, emonoid, fmonoid, gmonoid, hmonoid, imonoid, jmonoid, kmonoid, lmonoid, mmonoid, nmonoid, omonoid, pmonoid, qmonoid, rmonoid, smonoid, tmonoid) + } + + implicit def monoid21[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F], gmonoid : Monoid[G], hmonoid : Monoid[H], imonoid : Monoid[I], jmonoid : Monoid[J], kmonoid : Monoid[K], lmonoid : Monoid[L], mmonoid : Monoid[M], nmonoid : Monoid[N], omonoid : Monoid[O], pmonoid : Monoid[P], qmonoid : Monoid[Q], rmonoid : Monoid[R], smonoid : Monoid[S], tmonoid : Monoid[T], umonoid : Monoid[U]) : Monoid[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U)] = { + new Tuple21Monoid[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U]()(amonoid, bmonoid, cmonoid, dmonoid, emonoid, fmonoid, gmonoid, hmonoid, imonoid, jmonoid, kmonoid, lmonoid, mmonoid, nmonoid, omonoid, pmonoid, qmonoid, rmonoid, smonoid, tmonoid, umonoid) + } + + implicit def monoid22[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F], gmonoid : Monoid[G], hmonoid : Monoid[H], imonoid : Monoid[I], jmonoid : Monoid[J], kmonoid : Monoid[K], lmonoid : Monoid[L], mmonoid : Monoid[M], nmonoid : Monoid[N], omonoid : Monoid[O], pmonoid : Monoid[P], qmonoid : Monoid[Q], rmonoid : Monoid[R], smonoid : Monoid[S], tmonoid : Monoid[T], umonoid : Monoid[U], vmonoid : Monoid[V]) : Monoid[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V)] = { + new Tuple22Monoid[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V]()(amonoid, bmonoid, cmonoid, dmonoid, emonoid, fmonoid, gmonoid, hmonoid, imonoid, jmonoid, kmonoid, lmonoid, mmonoid, nmonoid, omonoid, pmonoid, qmonoid, rmonoid, smonoid, tmonoid, umonoid, vmonoid) + } +} +object Group { + implicit val boolGroup : Group[Boolean] = BooleanField + implicit val intGroup : Group[Int] = IntRing + implicit val longGroup : Group[Long] = LongRing + implicit val floatGroup : Group[Float] = FloatField + implicit val doubleGroup : Group[Double] = DoubleField + implicit def mapGroup[K,V](implicit group : Group[V]) = new MapGroup[K,V]()(group) + implicit def pairGroup[T,U](implicit tg : Group[T], ug : Group[U]) : Group[(T,U)] = { + new Tuple2Group[T,U]()(tg,ug) + } + implicit def group3[A, B, C](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C]) : Group[(A, B, C)] = { + new Tuple3Group[A, B, C]()(agroup, bgroup, cgroup) + } + + implicit def group4[A, B, C, D](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D]) : Group[(A, B, C, D)] = { + new Tuple4Group[A, B, C, D]()(agroup, bgroup, cgroup, dgroup) + } + + implicit def group5[A, B, C, D, E](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E]) : Group[(A, B, C, D, E)] = { + new Tuple5Group[A, B, C, D, E]()(agroup, bgroup, cgroup, dgroup, egroup) + } + + implicit def group6[A, B, C, D, E, F](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F]) : Group[(A, B, C, D, E, F)] = { + new Tuple6Group[A, B, C, D, E, F]()(agroup, bgroup, cgroup, dgroup, egroup, fgroup) + } + + implicit def group7[A, B, C, D, E, F, G](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F], ggroup : Group[G]) : Group[(A, B, C, D, E, F, G)] = { + new Tuple7Group[A, B, C, D, E, F, G]()(agroup, bgroup, cgroup, dgroup, egroup, fgroup, ggroup) + } + + implicit def group8[A, B, C, D, E, F, G, H](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F], ggroup : Group[G], hgroup : Group[H]) : Group[(A, B, C, D, E, F, G, H)] = { + new Tuple8Group[A, B, C, D, E, F, G, H]()(agroup, bgroup, cgroup, dgroup, egroup, fgroup, ggroup, hgroup) + } + + implicit def group9[A, B, C, D, E, F, G, H, I](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F], ggroup : Group[G], hgroup : Group[H], igroup : Group[I]) : Group[(A, B, C, D, E, F, G, H, I)] = { + new Tuple9Group[A, B, C, D, E, F, G, H, I]()(agroup, bgroup, cgroup, dgroup, egroup, fgroup, ggroup, hgroup, igroup) + } + + implicit def group10[A, B, C, D, E, F, G, H, I, J](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F], ggroup : Group[G], hgroup : Group[H], igroup : Group[I], jgroup : Group[J]) : Group[(A, B, C, D, E, F, G, H, I, J)] = { + new Tuple10Group[A, B, C, D, E, F, G, H, I, J]()(agroup, bgroup, cgroup, dgroup, egroup, fgroup, ggroup, hgroup, igroup, jgroup) + } + + implicit def group11[A, B, C, D, E, F, G, H, I, J, K](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F], ggroup : Group[G], hgroup : Group[H], igroup : Group[I], jgroup : Group[J], kgroup : Group[K]) : Group[(A, B, C, D, E, F, G, H, I, J, K)] = { + new Tuple11Group[A, B, C, D, E, F, G, H, I, J, K]()(agroup, bgroup, cgroup, dgroup, egroup, fgroup, ggroup, hgroup, igroup, jgroup, kgroup) + } + + implicit def group12[A, B, C, D, E, F, G, H, I, J, K, L](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F], ggroup : Group[G], hgroup : Group[H], igroup : Group[I], jgroup : Group[J], kgroup : Group[K], lgroup : Group[L]) : Group[(A, B, C, D, E, F, G, H, I, J, K, L)] = { + new Tuple12Group[A, B, C, D, E, F, G, H, I, J, K, L]()(agroup, bgroup, cgroup, dgroup, egroup, fgroup, ggroup, hgroup, igroup, jgroup, kgroup, lgroup) + } + + implicit def group13[A, B, C, D, E, F, G, H, I, J, K, L, M](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F], ggroup : Group[G], hgroup : Group[H], igroup : Group[I], jgroup : Group[J], kgroup : Group[K], lgroup : Group[L], mgroup : Group[M]) : Group[(A, B, C, D, E, F, G, H, I, J, K, L, M)] = { + new Tuple13Group[A, B, C, D, E, F, G, H, I, J, K, L, M]()(agroup, bgroup, cgroup, dgroup, egroup, fgroup, ggroup, hgroup, igroup, jgroup, kgroup, lgroup, mgroup) + } + + implicit def group14[A, B, C, D, E, F, G, H, I, J, K, L, M, N](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F], ggroup : Group[G], hgroup : Group[H], igroup : Group[I], jgroup : Group[J], kgroup : Group[K], lgroup : Group[L], mgroup : Group[M], ngroup : Group[N]) : Group[(A, B, C, D, E, F, G, H, I, J, K, L, M, N)] = { + new Tuple14Group[A, B, C, D, E, F, G, H, I, J, K, L, M, N]()(agroup, bgroup, cgroup, dgroup, egroup, fgroup, ggroup, hgroup, igroup, jgroup, kgroup, lgroup, mgroup, ngroup) + } + + implicit def group15[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F], ggroup : Group[G], hgroup : Group[H], igroup : Group[I], jgroup : Group[J], kgroup : Group[K], lgroup : Group[L], mgroup : Group[M], ngroup : Group[N], ogroup : Group[O]) : Group[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O)] = { + new Tuple15Group[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O]()(agroup, bgroup, cgroup, dgroup, egroup, fgroup, ggroup, hgroup, igroup, jgroup, kgroup, lgroup, mgroup, ngroup, ogroup) + } + + implicit def group16[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F], ggroup : Group[G], hgroup : Group[H], igroup : Group[I], jgroup : Group[J], kgroup : Group[K], lgroup : Group[L], mgroup : Group[M], ngroup : Group[N], ogroup : Group[O], pgroup : Group[P]) : Group[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P)] = { + new Tuple16Group[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P]()(agroup, bgroup, cgroup, dgroup, egroup, fgroup, ggroup, hgroup, igroup, jgroup, kgroup, lgroup, mgroup, ngroup, ogroup, pgroup) + } + + implicit def group17[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F], ggroup : Group[G], hgroup : Group[H], igroup : Group[I], jgroup : Group[J], kgroup : Group[K], lgroup : Group[L], mgroup : Group[M], ngroup : Group[N], ogroup : Group[O], pgroup : Group[P], qgroup : Group[Q]) : Group[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q)] = { + new Tuple17Group[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q]()(agroup, bgroup, cgroup, dgroup, egroup, fgroup, ggroup, hgroup, igroup, jgroup, kgroup, lgroup, mgroup, ngroup, ogroup, pgroup, qgroup) + } + + implicit def group18[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F], ggroup : Group[G], hgroup : Group[H], igroup : Group[I], jgroup : Group[J], kgroup : Group[K], lgroup : Group[L], mgroup : Group[M], ngroup : Group[N], ogroup : Group[O], pgroup : Group[P], qgroup : Group[Q], rgroup : Group[R]) : Group[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R)] = { + new Tuple18Group[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R]()(agroup, bgroup, cgroup, dgroup, egroup, fgroup, ggroup, hgroup, igroup, jgroup, kgroup, lgroup, mgroup, ngroup, ogroup, pgroup, qgroup, rgroup) + } + + implicit def group19[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F], ggroup : Group[G], hgroup : Group[H], igroup : Group[I], jgroup : Group[J], kgroup : Group[K], lgroup : Group[L], mgroup : Group[M], ngroup : Group[N], ogroup : Group[O], pgroup : Group[P], qgroup : Group[Q], rgroup : Group[R], sgroup : Group[S]) : Group[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S)] = { + new Tuple19Group[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S]()(agroup, bgroup, cgroup, dgroup, egroup, fgroup, ggroup, hgroup, igroup, jgroup, kgroup, lgroup, mgroup, ngroup, ogroup, pgroup, qgroup, rgroup, sgroup) + } + + implicit def group20[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F], ggroup : Group[G], hgroup : Group[H], igroup : Group[I], jgroup : Group[J], kgroup : Group[K], lgroup : Group[L], mgroup : Group[M], ngroup : Group[N], ogroup : Group[O], pgroup : Group[P], qgroup : Group[Q], rgroup : Group[R], sgroup : Group[S], tgroup : Group[T]) : Group[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T)] = { + new Tuple20Group[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T]()(agroup, bgroup, cgroup, dgroup, egroup, fgroup, ggroup, hgroup, igroup, jgroup, kgroup, lgroup, mgroup, ngroup, ogroup, pgroup, qgroup, rgroup, sgroup, tgroup) + } + + implicit def group21[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F], ggroup : Group[G], hgroup : Group[H], igroup : Group[I], jgroup : Group[J], kgroup : Group[K], lgroup : Group[L], mgroup : Group[M], ngroup : Group[N], ogroup : Group[O], pgroup : Group[P], qgroup : Group[Q], rgroup : Group[R], sgroup : Group[S], tgroup : Group[T], ugroup : Group[U]) : Group[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U)] = { + new Tuple21Group[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U]()(agroup, bgroup, cgroup, dgroup, egroup, fgroup, ggroup, hgroup, igroup, jgroup, kgroup, lgroup, mgroup, ngroup, ogroup, pgroup, qgroup, rgroup, sgroup, tgroup, ugroup) + } + + implicit def group22[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F], ggroup : Group[G], hgroup : Group[H], igroup : Group[I], jgroup : Group[J], kgroup : Group[K], lgroup : Group[L], mgroup : Group[M], ngroup : Group[N], ogroup : Group[O], pgroup : Group[P], qgroup : Group[Q], rgroup : Group[R], sgroup : Group[S], tgroup : Group[T], ugroup : Group[U], vgroup : Group[V]) : Group[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V)] = { + new Tuple22Group[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V]()(agroup, bgroup, cgroup, dgroup, egroup, fgroup, ggroup, hgroup, igroup, jgroup, kgroup, lgroup, mgroup, ngroup, ogroup, pgroup, qgroup, rgroup, sgroup, tgroup, ugroup, vgroup) + } +} +object Ring { + implicit val boolRing : Ring[Boolean] = BooleanField + implicit val intRing : Ring[Int] = IntRing + implicit val longRing : Ring[Long] = LongRing + implicit val floatRing : Ring[Float] = FloatField + implicit val doubleRing : Ring[Double] = DoubleField + implicit def mapRing[K,V](implicit ring : Ring[V]) = new MapRing[K,V]()(ring) + implicit def pairRing[T,U](implicit tr : Ring[T], ur : Ring[U]) : Ring[(T,U)] = { + new Tuple2Ring[T,U]()(tr,ur) + } + implicit def ring3[A, B, C](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C]) : Ring[(A, B, C)] = { + new Tuple3Ring[A, B, C]()(aring, bring, cring) + } + + implicit def ring4[A, B, C, D](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D]) : Ring[(A, B, C, D)] = { + new Tuple4Ring[A, B, C, D]()(aring, bring, cring, dring) + } + + implicit def ring5[A, B, C, D, E](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E]) : Ring[(A, B, C, D, E)] = { + new Tuple5Ring[A, B, C, D, E]()(aring, bring, cring, dring, ering) + } + + implicit def ring6[A, B, C, D, E, F](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E], fring : Ring[F]) : Ring[(A, B, C, D, E, F)] = { + new Tuple6Ring[A, B, C, D, E, F]()(aring, bring, cring, dring, ering, fring) + } + + implicit def ring7[A, B, C, D, E, F, G](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E], fring : Ring[F], gring : Ring[G]) : Ring[(A, B, C, D, E, F, G)] = { + new Tuple7Ring[A, B, C, D, E, F, G]()(aring, bring, cring, dring, ering, fring, gring) + } + + implicit def ring8[A, B, C, D, E, F, G, H](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E], fring : Ring[F], gring : Ring[G], hring : Ring[H]) : Ring[(A, B, C, D, E, F, G, H)] = { + new Tuple8Ring[A, B, C, D, E, F, G, H]()(aring, bring, cring, dring, ering, fring, gring, hring) + } + + implicit def ring9[A, B, C, D, E, F, G, H, I](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E], fring : Ring[F], gring : Ring[G], hring : Ring[H], iring : Ring[I]) : Ring[(A, B, C, D, E, F, G, H, I)] = { + new Tuple9Ring[A, B, C, D, E, F, G, H, I]()(aring, bring, cring, dring, ering, fring, gring, hring, iring) + } + + implicit def ring10[A, B, C, D, E, F, G, H, I, J](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E], fring : Ring[F], gring : Ring[G], hring : Ring[H], iring : Ring[I], jring : Ring[J]) : Ring[(A, B, C, D, E, F, G, H, I, J)] = { + new Tuple10Ring[A, B, C, D, E, F, G, H, I, J]()(aring, bring, cring, dring, ering, fring, gring, hring, iring, jring) + } + + implicit def ring11[A, B, C, D, E, F, G, H, I, J, K](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E], fring : Ring[F], gring : Ring[G], hring : Ring[H], iring : Ring[I], jring : Ring[J], kring : Ring[K]) : Ring[(A, B, C, D, E, F, G, H, I, J, K)] = { + new Tuple11Ring[A, B, C, D, E, F, G, H, I, J, K]()(aring, bring, cring, dring, ering, fring, gring, hring, iring, jring, kring) + } + + implicit def ring12[A, B, C, D, E, F, G, H, I, J, K, L](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E], fring : Ring[F], gring : Ring[G], hring : Ring[H], iring : Ring[I], jring : Ring[J], kring : Ring[K], lring : Ring[L]) : Ring[(A, B, C, D, E, F, G, H, I, J, K, L)] = { + new Tuple12Ring[A, B, C, D, E, F, G, H, I, J, K, L]()(aring, bring, cring, dring, ering, fring, gring, hring, iring, jring, kring, lring) + } + + implicit def ring13[A, B, C, D, E, F, G, H, I, J, K, L, M](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E], fring : Ring[F], gring : Ring[G], hring : Ring[H], iring : Ring[I], jring : Ring[J], kring : Ring[K], lring : Ring[L], mring : Ring[M]) : Ring[(A, B, C, D, E, F, G, H, I, J, K, L, M)] = { + new Tuple13Ring[A, B, C, D, E, F, G, H, I, J, K, L, M]()(aring, bring, cring, dring, ering, fring, gring, hring, iring, jring, kring, lring, mring) + } + + implicit def ring14[A, B, C, D, E, F, G, H, I, J, K, L, M, N](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E], fring : Ring[F], gring : Ring[G], hring : Ring[H], iring : Ring[I], jring : Ring[J], kring : Ring[K], lring : Ring[L], mring : Ring[M], nring : Ring[N]) : Ring[(A, B, C, D, E, F, G, H, I, J, K, L, M, N)] = { + new Tuple14Ring[A, B, C, D, E, F, G, H, I, J, K, L, M, N]()(aring, bring, cring, dring, ering, fring, gring, hring, iring, jring, kring, lring, mring, nring) + } + + implicit def ring15[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E], fring : Ring[F], gring : Ring[G], hring : Ring[H], iring : Ring[I], jring : Ring[J], kring : Ring[K], lring : Ring[L], mring : Ring[M], nring : Ring[N], oring : Ring[O]) : Ring[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O)] = { + new Tuple15Ring[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O]()(aring, bring, cring, dring, ering, fring, gring, hring, iring, jring, kring, lring, mring, nring, oring) + } + + implicit def ring16[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E], fring : Ring[F], gring : Ring[G], hring : Ring[H], iring : Ring[I], jring : Ring[J], kring : Ring[K], lring : Ring[L], mring : Ring[M], nring : Ring[N], oring : Ring[O], pring : Ring[P]) : Ring[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P)] = { + new Tuple16Ring[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P]()(aring, bring, cring, dring, ering, fring, gring, hring, iring, jring, kring, lring, mring, nring, oring, pring) + } + + implicit def ring17[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E], fring : Ring[F], gring : Ring[G], hring : Ring[H], iring : Ring[I], jring : Ring[J], kring : Ring[K], lring : Ring[L], mring : Ring[M], nring : Ring[N], oring : Ring[O], pring : Ring[P], qring : Ring[Q]) : Ring[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q)] = { + new Tuple17Ring[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q]()(aring, bring, cring, dring, ering, fring, gring, hring, iring, jring, kring, lring, mring, nring, oring, pring, qring) + } + + implicit def ring18[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E], fring : Ring[F], gring : Ring[G], hring : Ring[H], iring : Ring[I], jring : Ring[J], kring : Ring[K], lring : Ring[L], mring : Ring[M], nring : Ring[N], oring : Ring[O], pring : Ring[P], qring : Ring[Q], rring : Ring[R]) : Ring[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R)] = { + new Tuple18Ring[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R]()(aring, bring, cring, dring, ering, fring, gring, hring, iring, jring, kring, lring, mring, nring, oring, pring, qring, rring) + } + + implicit def ring19[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E], fring : Ring[F], gring : Ring[G], hring : Ring[H], iring : Ring[I], jring : Ring[J], kring : Ring[K], lring : Ring[L], mring : Ring[M], nring : Ring[N], oring : Ring[O], pring : Ring[P], qring : Ring[Q], rring : Ring[R], sring : Ring[S]) : Ring[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S)] = { + new Tuple19Ring[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S]()(aring, bring, cring, dring, ering, fring, gring, hring, iring, jring, kring, lring, mring, nring, oring, pring, qring, rring, sring) + } + + implicit def ring20[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E], fring : Ring[F], gring : Ring[G], hring : Ring[H], iring : Ring[I], jring : Ring[J], kring : Ring[K], lring : Ring[L], mring : Ring[M], nring : Ring[N], oring : Ring[O], pring : Ring[P], qring : Ring[Q], rring : Ring[R], sring : Ring[S], tring : Ring[T]) : Ring[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T)] = { + new Tuple20Ring[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T]()(aring, bring, cring, dring, ering, fring, gring, hring, iring, jring, kring, lring, mring, nring, oring, pring, qring, rring, sring, tring) + } + + implicit def ring21[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E], fring : Ring[F], gring : Ring[G], hring : Ring[H], iring : Ring[I], jring : Ring[J], kring : Ring[K], lring : Ring[L], mring : Ring[M], nring : Ring[N], oring : Ring[O], pring : Ring[P], qring : Ring[Q], rring : Ring[R], sring : Ring[S], tring : Ring[T], uring : Ring[U]) : Ring[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U)] = { + new Tuple21Ring[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U]()(aring, bring, cring, dring, ering, fring, gring, hring, iring, jring, kring, lring, mring, nring, oring, pring, qring, rring, sring, tring, uring) + } + + implicit def ring22[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C], dring : Ring[D], ering : Ring[E], fring : Ring[F], gring : Ring[G], hring : Ring[H], iring : Ring[I], jring : Ring[J], kring : Ring[K], lring : Ring[L], mring : Ring[M], nring : Ring[N], oring : Ring[O], pring : Ring[P], qring : Ring[Q], rring : Ring[R], sring : Ring[S], tring : Ring[T], uring : Ring[U], vring : Ring[V]) : Ring[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V)] = { + new Tuple22Ring[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V]()(aring, bring, cring, dring, ering, fring, gring, hring, iring, jring, kring, lring, mring, nring, oring, pring, qring, rring, sring, tring, uring, vring) + } + +} + +object Field { + implicit val boolField : Field[Boolean] = BooleanField + implicit val floatField : Field[Float] = FloatField + implicit val doubleField : Field[Double] = DoubleField +} diff --git a/src/test/scala/com/twitter/scalding/AlgebraicReductionsTest.scala b/src/test/scala/com/twitter/scalding/AlgebraicReductionsTest.scala new file mode 100644 index 0000000000..0d356e58e8 --- /dev/null +++ b/src/test/scala/com/twitter/scalding/AlgebraicReductionsTest.scala @@ -0,0 +1,33 @@ +package com.twitter.scalding + +import org.specs._ +/** + */ +class AlgebraJob(args : Args) extends Job(args) { + Tsv("input", ('x,'y,'z,'w)) + .map('w -> 'w) { w : Int => Set(w) } + .groupBy('x) { + _.plus[(Int,Int)](('y,'z) -> ('sy, 'sz)) + .plus[Set[Int]]('w -> 'setw) + .times[(Int,Int)](('y, 'z) -> ('py, 'pz)) + .dot[Int]('y,'z,'ydotz) + } + .write(Tsv("output")) +} + +class AlgebraJobTest extends Specification { + import RichPipe._ + val inputData = List((1,2,3,5),(1,4,5,7),(2,1,0,7)) + val correctOutput = List((1,6,8,Set(5,7), 8,15,(6 + 20)),(2,1,0,Set(7),1,0,0)) + "A AlgebraJob" should { + JobTest("com.twitter.scalding.AlgebraJob") + .source(Tsv("input",('x,'y,'z,'w)), inputData) + .sink[(Int, Int, Int, Set[Int], Int, Int, Int)](Tsv("output")) { buf => + "correctly do algebra" in { + buf.toList must be_==(correctOutput) + } + } + .run + .finish + } +} From 0122addcbf7857ea28e2e25e77a8b27036b39440 Mon Sep 17 00:00:00 2001 From: Sam Ritchie Date: Fri, 13 Apr 2012 15:30:39 -0700 Subject: [PATCH 15/25] Update to wip 271. Fixed issue w/ transient field in Maple. --- build.sbt | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/build.sbt b/build.sbt index 141a37e75b..ca8398447e 100644 --- a/build.sbt +++ b/build.sbt @@ -10,17 +10,17 @@ scalaVersion := "2.8.1" resolvers += "Concurrent Maven Repo" at "http://conjars.org/repo" -libraryDependencies += "cascading" % "cascading-core" % "2.0.0-wip-238" +libraryDependencies += "cascading" % "cascading-core" % "2.0.0-wip-271" -libraryDependencies += "cascading" % "cascading-local" % "2.0.0-wip-238" +libraryDependencies += "cascading" % "cascading-local" % "2.0.0-wip-271" -libraryDependencies += "cascading" % "cascading-hadoop" % "2.0.0-wip-238" +libraryDependencies += "cascading" % "cascading-hadoop" % "2.0.0-wip-271" libraryDependencies += "cascading.kryo" % "cascading.kryo" % "0.3.0" libraryDependencies += "com.twitter" % "meat-locker" % "0.2.0" -libraryDependencies += "com.twitter" % "maple" % "0.1.1" +libraryDependencies += "com.twitter" % "maple" % "0.1.2" libraryDependencies += "commons-lang" % "commons-lang" % "2.4" From 2c032583f43757969b34685b4f07b91e7f40bd69 Mon Sep 17 00:00:00 2001 From: Sam Ritchie Date: Fri, 13 Apr 2012 15:39:32 -0700 Subject: [PATCH 16/25] Change meatlocker to maple. --- src/main/scala/com/twitter/scalding/FileSource.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/com/twitter/scalding/FileSource.scala b/src/main/scala/com/twitter/scalding/FileSource.scala index 09664b1f3c..a9e64a3165 100644 --- a/src/main/scala/com/twitter/scalding/FileSource.scala +++ b/src/main/scala/com/twitter/scalding/FileSource.scala @@ -15,7 +15,7 @@ limitations under the License. */ package com.twitter.scalding -import com.twitter.meatlocker.tap.MemorySourceTap +import com.twitter.maple.tap.MemorySourceTap import java.io.File import java.util.TimeZone From 5ffbb7ec3dc1a56f2e205524ba626bc74b30a621 Mon Sep 17 00:00:00 2001 From: Sam Ritchie Date: Tue, 17 Apr 2012 15:32:06 -0700 Subject: [PATCH 17/25] Upgrade Scalding and its dependencies to Kryo 2.04. (This loses the kryo-serializers dependency.) --- build.sbt | 4 ++-- .../com/twitter/scalding/KryoHadoopSerialization.scala | 8 ++++---- src/test/scala/com/twitter/scalding/KryoTest.scala | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/build.sbt b/build.sbt index ca8398447e..66311e838c 100644 --- a/build.sbt +++ b/build.sbt @@ -16,9 +16,9 @@ libraryDependencies += "cascading" % "cascading-local" % "2.0.0-wip-271" libraryDependencies += "cascading" % "cascading-hadoop" % "2.0.0-wip-271" -libraryDependencies += "cascading.kryo" % "cascading.kryo" % "0.3.0" +libraryDependencies += "cascading.kryo" % "cascading.kryo" % "0.3.1" -libraryDependencies += "com.twitter" % "meat-locker" % "0.2.0" +libraryDependencies += "com.twitter" % "meat-locker" % "0.2.1" libraryDependencies += "com.twitter" % "maple" % "0.1.2" diff --git a/src/main/scala/com/twitter/scalding/KryoHadoopSerialization.scala b/src/main/scala/com/twitter/scalding/KryoHadoopSerialization.scala index d39641008e..a0ef4b0207 100644 --- a/src/main/scala/com/twitter/scalding/KryoHadoopSerialization.scala +++ b/src/main/scala/com/twitter/scalding/KryoHadoopSerialization.scala @@ -109,7 +109,7 @@ class KryoHadoopSerialization extends KryoSerialization { // It's important you actually do this, or Kryo will generate Nil != Nil, or None != None class SingletonSerializer[T](obj: T) extends KSerializer[T] { def write(kser: Kryo, out: Output, obj: T) {} - def read(kser: Kryo, in: Input, cls: Class[T]): T = obj + override def create(kser: Kryo, in: Input, cls: Class[T]): T = obj } // Lists cause stack overflows for Kryo because they are cons cells. @@ -130,7 +130,7 @@ class ListSerializer extends KSerializer[AnyRef] { list.foreach { t => kser.writeClassAndObject(out, t) } } -def read(kser: Kryo, in: Input, cls: Class[AnyRef]) : AnyRef = { + override def create(kser: Kryo, in: Input, cls: Class[AnyRef]) : AnyRef = { val size = in.readInt(true); //Produce the reversed list: @@ -158,7 +158,7 @@ class RichDateSerializer() extends KSerializer[RichDate] { out.writeLong(date.value.getTime, true); } - def read(kser: Kryo, in: Input, cls: Class[RichDate]): RichDate = { + override def create(kser: Kryo, in: Input, cls: Class[RichDate]): RichDate = { RichDate(in.readLong(true)) } } @@ -169,7 +169,7 @@ class DateRangeSerializer() extends KSerializer[DateRange] { out.writeLong(range.end.value.getTime, true); } - def read(kser: Kryo, in: Input, cls: Class[DateRange]): DateRange = { + override def create(kser: Kryo, in: Input, cls: Class[DateRange]): DateRange = { DateRange(RichDate(in.readLong(true)), RichDate(in.readLong(true))); } } diff --git a/src/test/scala/com/twitter/scalding/KryoTest.scala b/src/test/scala/com/twitter/scalding/KryoTest.scala index 198918c545..1afd0c9ed4 100644 --- a/src/test/scala/com/twitter/scalding/KryoTest.scala +++ b/src/test/scala/com/twitter/scalding/KryoTest.scala @@ -30,7 +30,7 @@ class KryoTest extends Specification { def deserObj[T <: AnyRef](cls : Class[_], input : Array[Byte]) : T = { val khs = new KryoHadoopSerialization khs.accept(cls) - val ks = khs.getDeserializer(cls.asInstanceOf[Class[T]]) + val ks = khs.getDeserializer(cls.asInstanceOf[Class[AnyRef]]) val in = new BIS(input) ks.open(in) val fakeInputHadoopNeeds = null From a19116978f8d2f319014077e04edacbd1bd7bc2b Mon Sep 17 00:00:00 2001 From: Argyris Zymnis Date: Tue, 17 Apr 2012 21:55:43 -0700 Subject: [PATCH 18/25] Break up autogenerated and manually written code in AbstractAlgebra. --- scripts/ntuple_generators.rb | 36 ++- .../mathematics/BaseAbstractAlgebra.scala | 289 +++++++++++++++++ ...a.scala => GeneratedAbstractAlgebra.scala} | 293 +----------------- .../scalding/AlgebraicReductionsTest.scala | 24 ++ 4 files changed, 348 insertions(+), 294 deletions(-) create mode 100644 src/main/scala/com/twitter/scalding/mathematics/BaseAbstractAlgebra.scala rename src/main/scala/com/twitter/scalding/mathematics/{AbstractAlgebra.scala => GeneratedAbstractAlgebra.scala} (93%) diff --git a/scripts/ntuple_generators.rb b/scripts/ntuple_generators.rb index 1ee4d1a3f4..ca7e08361e 100644 --- a/scripts/ntuple_generators.rb +++ b/scripts/ntuple_generators.rb @@ -1,6 +1,12 @@ # @author Edwin Chen (@echen) # Automatically write product monoid, product group, and product ring # classes for tuples up to size 22. +# +# Run it like this: +# +# ruby scripts/ntuple_generators.rb > src/main/scala/com/twitter/scalding/mathematics/GeneratedAbstractAlgebra.scala + +PACKAGE_NAME = "com.twitter.scalding.mathematics" # The tuple sizes we want. TUPLE_SIZES = (3..22).to_a @@ -9,6 +15,8 @@ # This provides an alphabet to draw types from. TYPE_SYMBOLS = ("A".."Z").to_a +INDENT = " " + # This returns the comment for each product monoid/group/ring definition. # n is the size of the product. # algebraic_structure is "monoid", "group", "ring", etc. @@ -70,7 +78,7 @@ def get_constant(n, algebraic_structure, constant) # Example return: # "override def negate(v : (T,U)) = (tgroup.negate(v._1), ugroup.negate(v._2))" def get_negate(n, algebraic_structure) - negates_commaed = TYPE_SYMBOLS.first(n).map.with_index{ |t, i| "#{t.downcase}#{algebraic_structure}.negate(v._#{i+1})" }.join(", ") + negates_commaed = TYPE_SYMBOLS.first(n).each_with_index.map{ |t, i| "#{t.downcase}#{algebraic_structure}.negate(v._#{i+1})" }.join(", ") "override def negate(v : (#{TYPE_SYMBOLS.first(n).join(", ")})) = (#{negates_commaed})" end @@ -90,7 +98,7 @@ def get_operation(n, algebraic_structure, operation) method_params = "l : #{individual_element_type}, r : #{individual_element_type}" # (1..n).to_a.map{ |i| "x#{i}" }.map{ |p| "#{p} : #{individual_element_type}" }.join(", ") # Example: "(tmonoid.plus(l._1,r._1), umonoid.plus(l._2, r._2))" - values_commaed = TYPE_SYMBOLS.first(n).map.with_index do |t, i| + values_commaed = TYPE_SYMBOLS.first(n).each_with_index.map do |t, i| "#{t.downcase}#{algebraic_structure}.#{operation}(l._#{i+1}, r._#{i+1})" end.join(", ") values_commaed = "(#{values_commaed})" @@ -111,11 +119,9 @@ def get_implicit_definition(n, algebraic_structure) # Example: "Monoid[(T,U)]" return_type = "#{algebraic_structure.capitalize}[(#{tuple_type_commaed})]" - ret = < + val leftV = left.getOrElse(newK, monoid.zero) + val rightV = right.getOrElse(newK, monoid.zero) + val newValue = monoid.plus(leftV, rightV) + if (monoid.isNonZero(newValue)) { + oldMap + (newK -> newValue) + } + else { + // Keep it sparse + oldMap - newK + } + } + } + // This is not part of the typeclass, but it is useful: + def sumValues(elem : Map[K,V]) : V = elem.values.reduceLeft { monoid.plus(_,_) } +} + +/** You can think of this as a Sparse vector group + */ +class MapGroup[K,V](implicit vgrp : Group[V]) extends MapMonoid[K,V]()(vgrp) with Group[Map[K,V]] { + override def negate(kv : Map[K,V]) = kv.mapValues { v => vgrp.negate(v) } +} + +/** You can think of this as a Sparse vector ring + */ +class MapRing[K,V](implicit ring : Ring[V]) extends MapGroup[K,V]()(ring) with Ring[Map[K,V]] { + // It is possible to implement this, but we need a special "identity map" which we + // deal with as if it were map with all possible keys (.get(x) == ring.one for all x). + // Then we have to manage the delta from this map as we add elements. That said, it + // is not actually needed in matrix multiplication, so we are punting on it for now. + override def one = error("multiplicative identity for Map unimplemented") + override def times(left : Map[K,V], right : Map[K,V]) : Map[K,V] = { + (left.keys.toSet & right.keys.toSet).foldLeft(Map[K,V]()) { (oldMap, newK) => + // The key is on both sides, so it is safe to get it out: + val leftV = left(newK) + val rightV = right(newK) + val newValue = ring.times(leftV, rightV) + if (ring.isNonZero(newValue)) { + oldMap + (newK -> newValue) + } + else { + // Keep it sparse + oldMap - newK + } + } + } + // This is not part of the typeclass, but it is useful: + def productValues(elem : Map[K,V]) : V = elem.values.reduceLeft { ring.times(_,_) } + def dot(left : Map[K,V], right : Map[K,V]) : V = sumValues(times(left, right)) +} + +object IntRing extends Ring[Int] { + override def zero = 0 + override def one = 1 + override def negate(v : Int) = -v + override def plus(l : Int, r : Int) = l + r + override def minus(l : Int, r : Int) = l - r + override def times(l : Int, r : Int) = l * r +} + +object LongRing extends Ring[Long] { + override def zero = 0L + override def one = 1L + override def negate(v : Long) = -v + override def plus(l : Long, r : Long) = l + r + override def minus(l : Long, r : Long) = l - r + override def times(l : Long, r : Long) = l * r +} + +object FloatField extends Field[Float] { + override def one = 1.0f + override def zero = 0.0f + override def negate(v : Float) = -v + override def plus(l : Float, r : Float) = l + r + override def minus(l : Float, r : Float) = l - r + override def times(l : Float, r : Float) = l * r + override def div(l : Float, r : Float) = { + assertNotZero(r) + l / r + } +} + +object DoubleField extends Field[Double] { + override def one = 1.0 + override def zero = 0.0 + override def negate(v : Double) = -v + override def plus(l : Double, r : Double) = l + r + override def minus(l : Double, r : Double) = l - r + override def times(l : Double, r : Double) = l * r + override def div(l : Double, r : Double) = { + assertNotZero(r) + l / r + } +} + +object BooleanField extends Field[Boolean] { + override def one = true + override def zero = false + override def negate(v : Boolean) = v + override def plus(l : Boolean, r : Boolean) = l ^ r + override def minus(l : Boolean, r : Boolean) = l ^ r + override def times(l : Boolean, r : Boolean) = l && r + override def inverse(l : Boolean) = { + assertNotZero(l) + true + } + override def div(l : Boolean, r : Boolean) = { + assertNotZero(r) + l + } +} + +/** +* Combine two monoids into a product monoid +*/ +class Tuple2Monoid[T,U](implicit tmonoid : Monoid[T], umonoid : Monoid[U]) extends Monoid[(T,U)] { + override def zero = (tmonoid.zero, umonoid.zero) + override def plus(l : (T,U), r : (T,U)) = (tmonoid.plus(l._1,r._1), umonoid.plus(l._2, r._2)) +} + +/** +* Combine two groups into a product group +*/ +class Tuple2Group[T,U](implicit tgroup : Group[T], ugroup : Group[U]) extends Group[(T,U)] { + override def zero = (tgroup.zero, ugroup.zero) + override def negate(v : (T,U)) = (tgroup.negate(v._1), ugroup.negate(v._2)) + override def plus(l : (T,U), r : (T,U)) = (tgroup.plus(l._1,r._1), ugroup.plus(l._2, r._2)) + override def minus(l : (T,U), r : (T,U)) = (tgroup.minus(l._1,r._1), ugroup.minus(l._2, r._2)) +} + +/** +* Combine two rings into a product ring +*/ +class Tuple2Ring[T,U](implicit tring : Ring[T], uring : Ring[U]) extends Ring[(T,U)] { + override def zero = (tring.zero, uring.zero) + override def one = (tring.one, uring.one) + override def negate(v : (T,U)) = (tring.negate(v._1), uring.negate(v._2)) + override def plus(l : (T,U), r : (T,U)) = (tring.plus(l._1,r._1), uring.plus(l._2, r._2)) + override def minus(l : (T,U), r : (T,U)) = (tring.minus(l._1,r._1), uring.minus(l._2, r._2)) + override def times(l : (T,U), r : (T,U)) = (tring.times(l._1,r._1), uring.times(l._2, r._2)) +} + +object Monoid extends GeneratedMonoidImplicits { + implicit val boolMonoid : Monoid[Boolean] = BooleanField + implicit val intMonoid : Monoid[Int] = IntRing + implicit val longMonoid : Monoid[Long] = LongRing + implicit val floatMonoid : Monoid[Float] = FloatField + implicit val doubleMonoid : Monoid[Double] = DoubleField + implicit def listMonoid[T] : Monoid[List[T]] = new ListMonoid[T] + implicit def setMonoid[T] : Monoid[Set[T]] = new SetMonoid[T] + implicit def mapMonoid[K,V](implicit monoid : Monoid[V]) = new MapMonoid[K,V]()(monoid) + implicit def pairMonoid[T,U](implicit tg : Monoid[T], ug : Monoid[U]) : Monoid[(T,U)] = { + new Tuple2Monoid[T,U]()(tg,ug) + } +} + +object Group extends GeneratedGroupImplicits { + implicit val boolGroup : Group[Boolean] = BooleanField + implicit val intGroup : Group[Int] = IntRing + implicit val longGroup : Group[Long] = LongRing + implicit val floatGroup : Group[Float] = FloatField + implicit val doubleGroup : Group[Double] = DoubleField + implicit def mapGroup[K,V](implicit group : Group[V]) = new MapGroup[K,V]()(group) + implicit def pairGroup[T,U](implicit tg : Group[T], ug : Group[U]) : Group[(T,U)] = { + new Tuple2Group[T,U]()(tg,ug) + } +} + +object Ring extends GeneratedRingImplicits { + implicit val boolRing : Ring[Boolean] = BooleanField + implicit val intRing : Ring[Int] = IntRing + implicit val longRing : Ring[Long] = LongRing + implicit val floatRing : Ring[Float] = FloatField + implicit val doubleRing : Ring[Double] = DoubleField + implicit def mapRing[K,V](implicit ring : Ring[V]) = new MapRing[K,V]()(ring) + implicit def pairRing[T,U](implicit tr : Ring[T], ur : Ring[U]) : Ring[(T,U)] = { + new Tuple2Ring[T,U]()(tr,ur) + } +} + +object Field { + implicit val boolField : Field[Boolean] = BooleanField + implicit val floatField : Field[Float] = FloatField + implicit val doubleField : Field[Double] = DoubleField +} diff --git a/src/main/scala/com/twitter/scalding/mathematics/AbstractAlgebra.scala b/src/main/scala/com/twitter/scalding/mathematics/GeneratedAbstractAlgebra.scala similarity index 93% rename from src/main/scala/com/twitter/scalding/mathematics/AbstractAlgebra.scala rename to src/main/scala/com/twitter/scalding/mathematics/GeneratedAbstractAlgebra.scala index a16341c861..4555047e2e 100644 --- a/src/main/scala/com/twitter/scalding/mathematics/AbstractAlgebra.scala +++ b/src/main/scala/com/twitter/scalding/mathematics/GeneratedAbstractAlgebra.scala @@ -1,249 +1,6 @@ -/* -Copyright 2012 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. -*/ +// following were autogenerated by scripts/ntuple_generators.rb at Tue Apr 17 21:48:42 -0700 2012 do not edit package com.twitter.scalding.mathematics -/** - * Monoid (take a deep breath, and relax about the weird name): - * This is a class that has an additive identify (called zero), and plus method that is - * associative: a+(b+c) = (a+b)+c and a+0=a, 0+a=a - * - * Group: this is a monoid that also has subtraction (and negation): - * So, you can do (a-b), or -a (which is equal to 0 - a). - * - * Ring: Group + multiplication (see: http://en.wikipedia.org/wiki/Ring_%28mathematics%29) - * and the three elements it defines: - * - additive identity aka zero - * - addition - * - multiplication - * - * The ring is to be passed as an argument to Matrix innerproduct and - * provides the definitions for addition and multiplication - * - * Field: Ring + division. It is a generalization of Ring and adds support for inversion and - * multiplicative identity. - */ - -trait Monoid[@specialized(Int,Long,Float,Double) T] extends java.io.Serializable { - def zero : T //additive identity - def assertNotZero(v : T) { - if(zero == v) { - throw new java.lang.IllegalArgumentException("argument should not be zero") - } - } - - def isNonZero(v : T) = (v != zero) - - def nonZeroOption(v : T): Option[T] = { - if (isNonZero(v)) { - Some(v) - } - else { - None - } - } - def plus(l : T, r : T) : T -} - -trait Group[@specialized(Int,Long,Float,Double) T] extends Monoid[T] { - // must override negate or minus (or both) - def negate(v : T) : T = minus(zero, v) - def minus(l : T, r : T) : T = plus(l, negate(r)) -} - -trait Ring[@specialized(Int,Long,Float,Double) T] extends Group[T] { - def one : T // Multiplicative identity - def times(l : T, r : T) : T -} - -trait Field[@specialized(Int,Long,Float,Double) T] extends Ring[T] { - // default implementation uses div YOU MUST OVERRIDE ONE OF THESE - def inverse(v : T) : T = { - assertNotZero(v) - div(one, v) - } - // default implementation uses inverse: - def div(l : T, r : T) : T = { - assertNotZero(r) - times(l, inverse(r)) - } -} - -/** List concatenation monoid. - * plus means concatenation, zero is empty list - */ -class ListMonoid[T] extends Monoid[List[T]] { - override def zero = List[T]() - override def plus(left : List[T], right : List[T]) = left ++ right -} - -/** Set union monoid. - * plus means union, zero is empty set - */ -class SetMonoid[T] extends Monoid[Set[T]] { - override def zero = Set[T]() - override def plus(left : Set[T], right : Set[T]) = left ++ right -} - -/** You can think of this as a Sparse vector monoid - */ -class MapMonoid[K,V](implicit monoid : Monoid[V]) extends Monoid[Map[K,V]] { - override def zero = Map[K,V]() - override def plus(left : Map[K,V], right : Map[K,V]) = { - (left.keys.toSet ++ right.keys.toSet).foldLeft(Map[K,V]()) { (oldMap, newK) => - val leftV = left.getOrElse(newK, monoid.zero) - val rightV = right.getOrElse(newK, monoid.zero) - val newValue = monoid.plus(leftV, rightV) - if (monoid.isNonZero(newValue)) { - oldMap + (newK -> newValue) - } - else { - // Keep it sparse - oldMap - newK - } - } - } - // This is not part of the typeclass, but it is useful: - def sumValues(elem : Map[K,V]) : V = elem.values.reduceLeft { monoid.plus(_,_) } -} - -/** You can think of this as a Sparse vector group - */ -class MapGroup[K,V](implicit vgrp : Group[V]) extends MapMonoid[K,V]()(vgrp) with Group[Map[K,V]] { - override def negate(kv : Map[K,V]) = kv.mapValues { v => vgrp.negate(v) } -} - -/** You can think of this as a Sparse vector ring - */ -class MapRing[K,V](implicit ring : Ring[V]) extends MapGroup[K,V]()(ring) with Ring[Map[K,V]] { - // It is possible to implement this, but we need a special "identity map" which we - // deal with as if it were map with all possible keys (.get(x) == ring.one for all x). - // Then we have to manage the delta from this map as we add elements. That said, it - // is not actually needed in matrix multiplication, so we are punting on it for now. - override def one = error("multiplicative identity for Map unimplemented") - override def times(left : Map[K,V], right : Map[K,V]) : Map[K,V] = { - (left.keys.toSet & right.keys.toSet).foldLeft(Map[K,V]()) { (oldMap, newK) => - // The key is on both sides, so it is safe to get it out: - val leftV = left(newK) - val rightV = right(newK) - val newValue = ring.times(leftV, rightV) - if (ring.isNonZero(newValue)) { - oldMap + (newK -> newValue) - } - else { - // Keep it sparse - oldMap - newK - } - } - } - // This is not part of the typeclass, but it is useful: - def productValues(elem : Map[K,V]) : V = elem.values.reduceLeft { ring.times(_,_) } - def dot(left : Map[K,V], right : Map[K,V]) : V = sumValues(times(left, right)) -} - -object IntRing extends Ring[Int] { - override def zero = 0 - override def one = 1 - override def negate(v : Int) = -v - override def plus(l : Int, r : Int) = l + r - override def minus(l : Int, r : Int) = l - r - override def times(l : Int, r : Int) = l * r -} - -object LongRing extends Ring[Long] { - override def zero = 0L - override def one = 1L - override def negate(v : Long) = -v - override def plus(l : Long, r : Long) = l + r - override def minus(l : Long, r : Long) = l - r - override def times(l : Long, r : Long) = l * r -} - -object FloatField extends Field[Float] { - override def one = 1.0f - override def zero = 0.0f - override def negate(v : Float) = -v - override def plus(l : Float, r : Float) = l + r - override def minus(l : Float, r : Float) = l - r - override def times(l : Float, r : Float) = l * r - override def div(l : Float, r : Float) = { - assertNotZero(r) - l / r - } -} - -object DoubleField extends Field[Double] { - override def one = 1.0 - override def zero = 0.0 - override def negate(v : Double) = -v - override def plus(l : Double, r : Double) = l + r - override def minus(l : Double, r : Double) = l - r - override def times(l : Double, r : Double) = l * r - override def div(l : Double, r : Double) = { - assertNotZero(r) - l / r - } -} - -object BooleanField extends Field[Boolean] { - override def one = true - override def zero = false - override def negate(v : Boolean) = v - override def plus(l : Boolean, r : Boolean) = l ^ r - override def minus(l : Boolean, r : Boolean) = l ^ r - override def times(l : Boolean, r : Boolean) = l && r - override def inverse(l : Boolean) = { - assertNotZero(l) - true - } - override def div(l : Boolean, r : Boolean) = { - assertNotZero(r) - l - } -} - -/** -* Combine two monoids into a product monoid -*/ -class Tuple2Monoid[T,U](implicit tmonoid : Monoid[T], umonoid : Monoid[U]) extends Monoid[(T,U)] { - override def zero = (tmonoid.zero, umonoid.zero) - override def plus(l : (T,U), r : (T,U)) = (tmonoid.plus(l._1,r._1), umonoid.plus(l._2, r._2)) -} - -/** -* Combine two groups into a product group -*/ -class Tuple2Group[T,U](implicit tgroup : Group[T], ugroup : Group[U]) extends Group[(T,U)] { - override def zero = (tgroup.zero, ugroup.zero) - override def negate(v : (T,U)) = (tgroup.negate(v._1), ugroup.negate(v._2)) - override def plus(l : (T,U), r : (T,U)) = (tgroup.plus(l._1,r._1), ugroup.plus(l._2, r._2)) - override def minus(l : (T,U), r : (T,U)) = (tgroup.minus(l._1,r._1), ugroup.minus(l._2, r._2)) -} - -/** -* Combine two rings into a product ring -*/ -class Tuple2Ring[T,U](implicit tring : Ring[T], uring : Ring[U]) extends Ring[(T,U)] { - override def zero = (tring.zero, uring.zero) - override def one = (tring.one, uring.one) - override def negate(v : (T,U)) = (tring.negate(v._1), uring.negate(v._2)) - override def plus(l : (T,U), r : (T,U)) = (tring.plus(l._1,r._1), uring.plus(l._2, r._2)) - override def minus(l : (T,U), r : (T,U)) = (tring.minus(l._1,r._1), uring.minus(l._2, r._2)) - override def times(l : (T,U), r : (T,U)) = (tring.times(l._1,r._1), uring.times(l._2, r._2)) -} - /** * Combine 3 monoids into a product monoid */ @@ -825,21 +582,7 @@ class Tuple22Ring[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, override def times(l : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V), r : (A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V)) = (aring.times(l._1, r._1), bring.times(l._2, r._2), cring.times(l._3, r._3), dring.times(l._4, r._4), ering.times(l._5, r._5), fring.times(l._6, r._6), gring.times(l._7, r._7), hring.times(l._8, r._8), iring.times(l._9, r._9), jring.times(l._10, r._10), kring.times(l._11, r._11), lring.times(l._12, r._12), mring.times(l._13, r._13), nring.times(l._14, r._14), oring.times(l._15, r._15), pring.times(l._16, r._16), qring.times(l._17, r._17), rring.times(l._18, r._18), sring.times(l._19, r._19), tring.times(l._20, r._20), uring.times(l._21, r._21), vring.times(l._22, r._22)) } -// Here we complete the typeclasses by added the implicits to the companions: - -object Monoid { - implicit val boolMonoid : Monoid[Boolean] = BooleanField - implicit val intMonoid : Monoid[Int] = IntRing - implicit val longMonoid : Monoid[Long] = LongRing - implicit val floatMonoid : Monoid[Float] = FloatField - implicit val doubleMonoid : Monoid[Double] = DoubleField - implicit def listMonoid[T] : Monoid[List[T]] = new ListMonoid[T] - implicit def setMonoid[T] : Monoid[Set[T]] = new SetMonoid[T] - implicit def mapMonoid[K,V](implicit monoid : Monoid[V]) = new MapMonoid[K,V]()(monoid) - - implicit def pairMonoid[T,U](implicit tg : Monoid[T], ug : Monoid[U]) : Monoid[(T,U)] = { - new Tuple2Monoid[T,U]()(tg,ug) - } +trait GeneratedMonoidImplicits { implicit def monoid3[A, B, C](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C]) : Monoid[(A, B, C)] = { new Tuple3Monoid[A, B, C]()(amonoid, bmonoid, cmonoid) } @@ -919,17 +662,10 @@ object Monoid { implicit def monoid22[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V](implicit amonoid : Monoid[A], bmonoid : Monoid[B], cmonoid : Monoid[C], dmonoid : Monoid[D], emonoid : Monoid[E], fmonoid : Monoid[F], gmonoid : Monoid[G], hmonoid : Monoid[H], imonoid : Monoid[I], jmonoid : Monoid[J], kmonoid : Monoid[K], lmonoid : Monoid[L], mmonoid : Monoid[M], nmonoid : Monoid[N], omonoid : Monoid[O], pmonoid : Monoid[P], qmonoid : Monoid[Q], rmonoid : Monoid[R], smonoid : Monoid[S], tmonoid : Monoid[T], umonoid : Monoid[U], vmonoid : Monoid[V]) : Monoid[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V)] = { new Tuple22Monoid[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V]()(amonoid, bmonoid, cmonoid, dmonoid, emonoid, fmonoid, gmonoid, hmonoid, imonoid, jmonoid, kmonoid, lmonoid, mmonoid, nmonoid, omonoid, pmonoid, qmonoid, rmonoid, smonoid, tmonoid, umonoid, vmonoid) } + } -object Group { - implicit val boolGroup : Group[Boolean] = BooleanField - implicit val intGroup : Group[Int] = IntRing - implicit val longGroup : Group[Long] = LongRing - implicit val floatGroup : Group[Float] = FloatField - implicit val doubleGroup : Group[Double] = DoubleField - implicit def mapGroup[K,V](implicit group : Group[V]) = new MapGroup[K,V]()(group) - implicit def pairGroup[T,U](implicit tg : Group[T], ug : Group[U]) : Group[(T,U)] = { - new Tuple2Group[T,U]()(tg,ug) - } + +trait GeneratedGroupImplicits { implicit def group3[A, B, C](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C]) : Group[(A, B, C)] = { new Tuple3Group[A, B, C]()(agroup, bgroup, cgroup) } @@ -1009,17 +745,10 @@ object Group { implicit def group22[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V](implicit agroup : Group[A], bgroup : Group[B], cgroup : Group[C], dgroup : Group[D], egroup : Group[E], fgroup : Group[F], ggroup : Group[G], hgroup : Group[H], igroup : Group[I], jgroup : Group[J], kgroup : Group[K], lgroup : Group[L], mgroup : Group[M], ngroup : Group[N], ogroup : Group[O], pgroup : Group[P], qgroup : Group[Q], rgroup : Group[R], sgroup : Group[S], tgroup : Group[T], ugroup : Group[U], vgroup : Group[V]) : Group[(A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V)] = { new Tuple22Group[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U, V]()(agroup, bgroup, cgroup, dgroup, egroup, fgroup, ggroup, hgroup, igroup, jgroup, kgroup, lgroup, mgroup, ngroup, ogroup, pgroup, qgroup, rgroup, sgroup, tgroup, ugroup, vgroup) } + } -object Ring { - implicit val boolRing : Ring[Boolean] = BooleanField - implicit val intRing : Ring[Int] = IntRing - implicit val longRing : Ring[Long] = LongRing - implicit val floatRing : Ring[Float] = FloatField - implicit val doubleRing : Ring[Double] = DoubleField - implicit def mapRing[K,V](implicit ring : Ring[V]) = new MapRing[K,V]()(ring) - implicit def pairRing[T,U](implicit tr : Ring[T], ur : Ring[U]) : Ring[(T,U)] = { - new Tuple2Ring[T,U]()(tr,ur) - } + +trait GeneratedRingImplicits { implicit def ring3[A, B, C](implicit aring : Ring[A], bring : Ring[B], cring : Ring[C]) : Ring[(A, B, C)] = { new Tuple3Ring[A, B, C]()(aring, bring, cring) } @@ -1101,9 +830,3 @@ object Ring { } } - -object Field { - implicit val boolField : Field[Boolean] = BooleanField - implicit val floatField : Field[Float] = FloatField - implicit val doubleField : Field[Double] = DoubleField -} diff --git a/src/test/scala/com/twitter/scalding/AlgebraicReductionsTest.scala b/src/test/scala/com/twitter/scalding/AlgebraicReductionsTest.scala index 0d356e58e8..d7d72a3596 100644 --- a/src/test/scala/com/twitter/scalding/AlgebraicReductionsTest.scala +++ b/src/test/scala/com/twitter/scalding/AlgebraicReductionsTest.scala @@ -15,7 +15,17 @@ class AlgebraJob(args : Args) extends Job(args) { .write(Tsv("output")) } +class ComplicatedAlgebraJob(args : Args) extends Job(args) { + Tsv("input", ('x,'y,'z,'w,'v)) + .map('w -> 'w) { w : Int => Set(w) } + .groupBy('x) { + _.plus[(Int,Int,Set[Int],Double)](('y,'z,'w,'v) -> ('sy,'sz,'sw,'sv)) + } + .write(Tsv("output")) +} + class AlgebraJobTest extends Specification { + noDetailedDiffs() import RichPipe._ val inputData = List((1,2,3,5),(1,4,5,7),(2,1,0,7)) val correctOutput = List((1,6,8,Set(5,7), 8,15,(6 + 20)),(2,1,0,Set(7),1,0,0)) @@ -30,4 +40,18 @@ class AlgebraJobTest extends Specification { .run .finish } + + val inputData2 = List((1,2,3,5,1.2),(1,4,5,7,0.1),(2,1,0,7,3.2)) + val correctOutput2 = List((1,6,8,Set(5,7),1.3),(2,1,0,Set(7),3.2)) + "A ComplicatedAlgebraJob" should { + JobTest("com.twitter.scalding.ComplicatedAlgebraJob") + .source(Tsv("input",('x,'y,'z,'w,'v)), inputData2) + .sink[(Int, Int, Int, Set[Int], Double)](Tsv("output")) { buf => + "correctly do complex algebra" in { + buf.toList must be_==(correctOutput2) + } + } + .run + .finish + } } From 2500cfc57b6065fdbecc40b25d52aa0283230efc Mon Sep 17 00:00:00 2001 From: Argyris Zymnis Date: Wed, 18 Apr 2012 08:32:01 -0700 Subject: [PATCH 19/25] Fix serialization test. --- src/test/scala/com/twitter/scalding/KryoTest.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/scala/com/twitter/scalding/KryoTest.scala b/src/test/scala/com/twitter/scalding/KryoTest.scala index 1afd0c9ed4..2491b3e5a3 100644 --- a/src/test/scala/com/twitter/scalding/KryoTest.scala +++ b/src/test/scala/com/twitter/scalding/KryoTest.scala @@ -19,7 +19,7 @@ class KryoTest extends Specification { def serObj[T <: AnyRef](in : T) = { val khs = new KryoHadoopSerialization khs.accept(in.getClass) - val ks = khs.getSerializer(in.getClass.asInstanceOf[Class[T]]) + val ks = khs.getSerializer(in.getClass.asInstanceOf[Class[AnyRef]]) val out = new BOS ks.open(out) ks.serialize(in) From 62ff22953e76e6196b04d8d271a07e6f76398e20 Mon Sep 17 00:00:00 2001 From: Argyris Zymnis Date: Fri, 20 Apr 2012 13:36:48 -0700 Subject: [PATCH 20/25] Bump cascading to wip 278. --- build.sbt | 6 +++--- src/main/scala/com/twitter/scalding/FileSource.scala | 6 +++--- src/main/scala/com/twitter/scalding/IterableSource.scala | 6 ++++-- src/main/scala/com/twitter/scalding/MemoryTap.scala | 2 +- src/main/scala/com/twitter/scalding/Source.scala | 2 +- 5 files changed, 12 insertions(+), 10 deletions(-) diff --git a/build.sbt b/build.sbt index 66311e838c..d21cf6e3ae 100644 --- a/build.sbt +++ b/build.sbt @@ -10,11 +10,11 @@ scalaVersion := "2.8.1" resolvers += "Concurrent Maven Repo" at "http://conjars.org/repo" -libraryDependencies += "cascading" % "cascading-core" % "2.0.0-wip-271" +libraryDependencies += "cascading" % "cascading-core" % "2.0.0-wip-278" -libraryDependencies += "cascading" % "cascading-local" % "2.0.0-wip-271" +libraryDependencies += "cascading" % "cascading-local" % "2.0.0-wip-278" -libraryDependencies += "cascading" % "cascading-hadoop" % "2.0.0-wip-271" +libraryDependencies += "cascading" % "cascading-hadoop" % "2.0.0-wip-278" libraryDependencies += "cascading.kryo" % "cascading.kryo" % "0.3.1" diff --git a/src/main/scala/com/twitter/scalding/FileSource.scala b/src/main/scala/com/twitter/scalding/FileSource.scala index a9e64a3165..1f8ba23f81 100644 --- a/src/main/scala/com/twitter/scalding/FileSource.scala +++ b/src/main/scala/com/twitter/scalding/FileSource.scala @@ -142,7 +142,7 @@ abstract class FileSource extends Source { */ trait TextLineScheme extends Mappable[String] { override def localScheme = new CLTextLine() - override def hdfsScheme = new CHTextLine().asInstanceOf[Scheme[_ <: FlowProcess[_],_,_,_,_,_]] + override def hdfsScheme = new CHTextLine().asInstanceOf[Scheme[_ <: FlowProcess[JobConf],JobConf,_,_,_,_]] //In textline, 0 is the byte position, the actual text string is in column 1 override val columnNums = Seq(1) } @@ -160,7 +160,7 @@ trait DelimitedScheme extends Source { //These should not be changed: override def localScheme = new CLTextDelimited(fields, separator, types) override def hdfsScheme = { - new CHTextDelimited(fields, separator, types).asInstanceOf[Scheme[_ <: FlowProcess[_],_,_,_,_,_]] + new CHTextDelimited(fields, separator, types).asInstanceOf[Scheme[_ <: FlowProcess[JobConf],JobConf,_,_,_,_]] } } @@ -169,7 +169,7 @@ trait SequenceFileScheme extends Source { val fields = Fields.ALL // TODO Cascading doesn't support local mode yet override def hdfsScheme = { - new CHSequenceFile(fields).asInstanceOf[Scheme[_ <: FlowProcess[_],_,_,_,_,_]] + new CHSequenceFile(fields).asInstanceOf[Scheme[_ <: FlowProcess[JobConf],JobConf,_,_,_,_]] } } diff --git a/src/main/scala/com/twitter/scalding/IterableSource.scala b/src/main/scala/com/twitter/scalding/IterableSource.scala index dc6478e68f..8201d742a9 100644 --- a/src/main/scala/com/twitter/scalding/IterableSource.scala +++ b/src/main/scala/com/twitter/scalding/IterableSource.scala @@ -23,6 +23,8 @@ import cascading.scheme.Scheme import cascading.tuple.Tuple import cascading.tuple.Fields +import org.apache.hadoop.mapred.JobConf + import scala.collection.mutable.Buffer import scala.collection.JavaConverters._ @@ -54,8 +56,8 @@ case class IterableSource[T](@transient iter: Iterable[T], inFields : Fields = F new CLTextDelimited(fields, "\t", null : Array[Class[_]]) } - override def hdfsScheme : Scheme[_ <: FlowProcess[_],_,_,_,_,_] = { - hdfsTap.getScheme.asInstanceOf[Scheme[_ <: FlowProcess[_],_,_,_,_,_]] + override def hdfsScheme : Scheme[_ <: FlowProcess[JobConf],JobConf,_,_,_,_] = { + hdfsTap.getScheme.asInstanceOf[Scheme[_ <: FlowProcess[JobConf],JobConf,_,_,_,_]] } private lazy val hdfsTap = new MemorySourceTap(asBuffer.asJava, fields) diff --git a/src/main/scala/com/twitter/scalding/MemoryTap.scala b/src/main/scala/com/twitter/scalding/MemoryTap.scala index a03d232f9c..ce99698166 100644 --- a/src/main/scala/com/twitter/scalding/MemoryTap.scala +++ b/src/main/scala/com/twitter/scalding/MemoryTap.scala @@ -25,7 +25,7 @@ import cascading.scheme.Scheme import cascading.flow.FlowProcess import collection.mutable.{Buffer, MutableList} -class MemoryTap(val scheme : Scheme[_ <: FlowProcess[_],_,_,_,_,_], val tupleBuffer : Buffer[Tuple]) +class MemoryTap[C](val scheme : Scheme[_ <: FlowProcess[C],C,_,_,_,_], val tupleBuffer : Buffer[Tuple]) extends Tap[LocalFlowProcess, Properties, Any, Any](scheme) { override def createResource(conf : Properties) = true diff --git a/src/main/scala/com/twitter/scalding/Source.scala b/src/main/scala/com/twitter/scalding/Source.scala index 7491f8be32..a8586f3548 100644 --- a/src/main/scala/com/twitter/scalding/Source.scala +++ b/src/main/scala/com/twitter/scalding/Source.scala @@ -75,7 +75,7 @@ abstract class Source extends java.io.Serializable { def localScheme : LocalScheme = { error("Cascading local mode not supported for: " + toString) } - def hdfsScheme : Scheme[_ <: FlowProcess[_],_,_,_,_,_] = { + def hdfsScheme : Scheme[_ <: FlowProcess[JobConf],JobConf,_,_,_,_] = { error("Cascading Hadoop mode not supported for: " + toString) } From c3f95eea1fb12b6b324156eefec190206c6db581 Mon Sep 17 00:00:00 2001 From: Argyris Zymnis Date: Fri, 20 Apr 2012 14:03:57 -0700 Subject: [PATCH 21/25] Also upgrade maple. --- build.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index d21cf6e3ae..46751d3fa5 100644 --- a/build.sbt +++ b/build.sbt @@ -20,7 +20,7 @@ libraryDependencies += "cascading.kryo" % "cascading.kryo" % "0.3.1" libraryDependencies += "com.twitter" % "meat-locker" % "0.2.1" -libraryDependencies += "com.twitter" % "maple" % "0.1.2" +libraryDependencies += "com.twitter" % "maple" % "0.1.3" libraryDependencies += "commons-lang" % "commons-lang" % "2.4" From 3137c599b5466d59195ec1727026aec0157b37c4 Mon Sep 17 00:00:00 2001 From: Mike Jahr Date: Tue, 24 Apr 2012 16:14:21 -0700 Subject: [PATCH 22/25] Fixed the error message thrown by FileSource.validateTaps when strict mode is enabled. Previously it said "no data found" when it should say "some data not found". --- src/main/scala/com/twitter/scalding/FileSource.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/main/scala/com/twitter/scalding/FileSource.scala b/src/main/scala/com/twitter/scalding/FileSource.scala index 1f8ba23f81..1b12e281bb 100644 --- a/src/main/scala/com/twitter/scalding/FileSource.scala +++ b/src/main/scala/com/twitter/scalding/FileSource.scala @@ -95,11 +95,14 @@ abstract class FileSource extends Source { mode match { case Hdfs(strict, conf) => { if (strict && (!hdfsReadPathsAreGood(conf))) { - throw new InvalidSourceException("[" + this.toString + "] No good paths in: " + hdfsPaths.toString) + throw new InvalidSourceException( + "[" + this.toString + "] Data is missing from one or more paths in: " + + hdfsPaths.toString) } else if (!hdfsPaths.exists { pathIsGood(_, conf) }) { //Check that there is at least one good path: - throw new InvalidSourceException("[" + this.toString + "] No good paths in: " + hdfsPaths.toString) + throw new InvalidSourceException( + "[" + this.toString + "] No good paths in: " + hdfsPaths.toString) } } case _ => () From 1ebf4005722e17e71d88d38e0612093656769e1c Mon Sep 17 00:00:00 2001 From: Mike Jahr Date: Tue, 24 Apr 2012 16:01:57 -0700 Subject: [PATCH 23/25] Added a defaultTimeZone member to DefaultDateRangeJob, and added UtcDateRangeJob which uses UTC as the default time zone. --- src/main/scala/com/twitter/scalding/Job.scala | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/src/main/scala/com/twitter/scalding/Job.scala b/src/main/scala/com/twitter/scalding/Job.scala index 15be6640f4..2d22db46bd 100644 --- a/src/main/scala/com/twitter/scalding/Job.scala +++ b/src/main/scala/com/twitter/scalding/Job.scala @@ -151,10 +151,12 @@ trait DefaultDateRangeJob extends Job { //Get date implicits and PACIFIC and UTC vals. import DateOps._ - //optionally take --tz argument, or use Pacific time + // Optionally take --tz argument, or use Pacific time. Derived classes may + // override defaultTimeZone to change the default. + def defaultTimeZone = PACIFIC implicit val tz = args.optional("tz") match { case Some(tzn) => java.util.TimeZone.getTimeZone(tzn) - case None => PACIFIC + case None => defaultTimeZone } val (start, end) = args.list("date") match { @@ -167,6 +169,11 @@ trait DefaultDateRangeJob extends Job { implicit val dateRange = DateRange(start, end) } +// DefaultDateRangeJob with default time zone as UTC instead of Pacific. +trait UtcDateRangeJob extends DefaultDateRangeJob { + override def defaultTimeZone = DateOps.UTC +} + /* * Run a list of shell commands through bash in the given order. Return success * when all commands succeed. Excution stops after the first failure. The From f9ccf9ac69e1732b1c7b2b875d1c1ad625cbb92a Mon Sep 17 00:00:00 2001 From: Argyris Zymnis Date: Tue, 24 Apr 2012 18:35:09 -0700 Subject: [PATCH 24/25] Upgrade cascading to wip-281. --- build.sbt | 8 ++++---- src/main/scala/com/twitter/scalding/FileSource.scala | 10 +++++----- .../scala/com/twitter/scalding/IterableSource.scala | 6 ++++-- .../scala/com/twitter/scalding/JoinAlgorithms.scala | 6 +++--- .../com/twitter/scalding/KryoHadoopSerialization.scala | 8 ++++---- src/main/scala/com/twitter/scalding/Source.scala | 2 +- 6 files changed, 21 insertions(+), 19 deletions(-) diff --git a/build.sbt b/build.sbt index 46751d3fa5..bac55dfa1e 100644 --- a/build.sbt +++ b/build.sbt @@ -10,17 +10,17 @@ scalaVersion := "2.8.1" resolvers += "Concurrent Maven Repo" at "http://conjars.org/repo" -libraryDependencies += "cascading" % "cascading-core" % "2.0.0-wip-278" +libraryDependencies += "cascading" % "cascading-core" % "2.0.0-wip-281" -libraryDependencies += "cascading" % "cascading-local" % "2.0.0-wip-278" +libraryDependencies += "cascading" % "cascading-local" % "2.0.0-wip-281" -libraryDependencies += "cascading" % "cascading-hadoop" % "2.0.0-wip-278" +libraryDependencies += "cascading" % "cascading-hadoop" % "2.0.0-wip-281" libraryDependencies += "cascading.kryo" % "cascading.kryo" % "0.3.1" libraryDependencies += "com.twitter" % "meat-locker" % "0.2.1" -libraryDependencies += "com.twitter" % "maple" % "0.1.3" +libraryDependencies += "com.twitter" % "maple" % "0.1.4" libraryDependencies += "commons-lang" % "commons-lang" % "2.4" diff --git a/src/main/scala/com/twitter/scalding/FileSource.scala b/src/main/scala/com/twitter/scalding/FileSource.scala index 1b12e281bb..6b30298452 100644 --- a/src/main/scala/com/twitter/scalding/FileSource.scala +++ b/src/main/scala/com/twitter/scalding/FileSource.scala @@ -39,8 +39,8 @@ import cascading.tuple.{Tuple, TupleEntryIterator, Fields} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.mapred.JobConf -import org.apache.hadoop.mapred.OutputCollector; -import org.apache.hadoop.mapred.RecordReader; +import org.apache.hadoop.mapred.OutputCollector +import org.apache.hadoop.mapred.RecordReader import collection.mutable.{Buffer, MutableList} import scala.collection.JavaConverters._ @@ -145,7 +145,7 @@ abstract class FileSource extends Source { */ trait TextLineScheme extends Mappable[String] { override def localScheme = new CLTextLine() - override def hdfsScheme = new CHTextLine().asInstanceOf[Scheme[_ <: FlowProcess[JobConf],JobConf,_,_,_,_]] + override def hdfsScheme = new CHTextLine().asInstanceOf[Scheme[FlowProcess[JobConf],JobConf,RecordReader[_,_],OutputCollector[_,_],_,_]] //In textline, 0 is the byte position, the actual text string is in column 1 override val columnNums = Seq(1) } @@ -163,7 +163,7 @@ trait DelimitedScheme extends Source { //These should not be changed: override def localScheme = new CLTextDelimited(fields, separator, types) override def hdfsScheme = { - new CHTextDelimited(fields, separator, types).asInstanceOf[Scheme[_ <: FlowProcess[JobConf],JobConf,_,_,_,_]] + new CHTextDelimited(fields, separator, types).asInstanceOf[Scheme[FlowProcess[JobConf],JobConf,RecordReader[_,_],OutputCollector[_,_],_,_]] } } @@ -172,7 +172,7 @@ trait SequenceFileScheme extends Source { val fields = Fields.ALL // TODO Cascading doesn't support local mode yet override def hdfsScheme = { - new CHSequenceFile(fields).asInstanceOf[Scheme[_ <: FlowProcess[JobConf],JobConf,_,_,_,_]] + new CHSequenceFile(fields).asInstanceOf[Scheme[FlowProcess[JobConf],JobConf,RecordReader[_,_],OutputCollector[_,_],_,_]] } } diff --git a/src/main/scala/com/twitter/scalding/IterableSource.scala b/src/main/scala/com/twitter/scalding/IterableSource.scala index 8201d742a9..d76dd0b79d 100644 --- a/src/main/scala/com/twitter/scalding/IterableSource.scala +++ b/src/main/scala/com/twitter/scalding/IterableSource.scala @@ -24,6 +24,8 @@ import cascading.tuple.Tuple import cascading.tuple.Fields import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.mapred.OutputCollector +import org.apache.hadoop.mapred.RecordReader import scala.collection.mutable.Buffer import scala.collection.JavaConverters._ @@ -56,8 +58,8 @@ case class IterableSource[T](@transient iter: Iterable[T], inFields : Fields = F new CLTextDelimited(fields, "\t", null : Array[Class[_]]) } - override def hdfsScheme : Scheme[_ <: FlowProcess[JobConf],JobConf,_,_,_,_] = { - hdfsTap.getScheme.asInstanceOf[Scheme[_ <: FlowProcess[JobConf],JobConf,_,_,_,_]] + override def hdfsScheme : Scheme[FlowProcess[JobConf],JobConf,RecordReader[_,_],OutputCollector[_,_],_,_] = { + hdfsTap.getScheme.asInstanceOf[Scheme[FlowProcess[JobConf],JobConf,RecordReader[_,_],OutputCollector[_,_],_,_]] } private lazy val hdfsTap = new MemorySourceTap(asBuffer.asJava, fields) diff --git a/src/main/scala/com/twitter/scalding/JoinAlgorithms.scala b/src/main/scala/com/twitter/scalding/JoinAlgorithms.scala index bf6f284153..d0ceebc4ad 100644 --- a/src/main/scala/com/twitter/scalding/JoinAlgorithms.scala +++ b/src/main/scala/com/twitter/scalding/JoinAlgorithms.scala @@ -172,18 +172,18 @@ trait JoinAlgorithms { def joinWithTiny(fs :(Fields,Fields), that : Pipe) = { val intersection = asSet(fs._1).intersect(asSet(fs._2)) if (intersection.size == 0) { - new Join(assignName(pipe), fs._1, assignName(that), fs._2, new InnerJoin) + new HashJoin(assignName(pipe), fs._1, assignName(that), fs._2, new InnerJoin) } else { val (renamedThat, newJoinFields, temp) = renameCollidingFields(that, fs._2, intersection) - (new Join(assignName(pipe), fs._1, assignName(renamedThat), newJoinFields, new InnerJoin)) + (new HashJoin(assignName(pipe), fs._1, assignName(renamedThat), newJoinFields, new InnerJoin)) .discard(temp) } } def leftJoinWithTiny(fs :(Fields,Fields), that : Pipe) = { //Rename these pipes to avoid cascading name conflicts - new Join(assignName(pipe), fs._1, assignName(that), fs._2, new LeftJoin) + new HashJoin(assignName(pipe), fs._1, assignName(that), fs._2, new LeftJoin) } /* diff --git a/src/main/scala/com/twitter/scalding/KryoHadoopSerialization.scala b/src/main/scala/com/twitter/scalding/KryoHadoopSerialization.scala index a0ef4b0207..345dcfbfe3 100644 --- a/src/main/scala/com/twitter/scalding/KryoHadoopSerialization.scala +++ b/src/main/scala/com/twitter/scalding/KryoHadoopSerialization.scala @@ -27,8 +27,8 @@ import com.esotericsoftware.kryo.{Serializer => KSerializer} import com.esotericsoftware.kryo.io.{Input, Output} import cascading.kryo.KryoSerialization; -import cascading.tuple.hadoop.BufferedInputStream import cascading.tuple.hadoop.TupleSerialization +import cascading.tuple.hadoop.io.BufferedInputStream import scala.annotation.tailrec @@ -113,7 +113,7 @@ class SingletonSerializer[T](obj: T) extends KSerializer[T] { } // Lists cause stack overflows for Kryo because they are cons cells. -class ListSerializer extends KSerializer[AnyRef] { +class ListSerializer extends KSerializer[AnyRef] { def write(kser: Kryo, out: Output, obj: AnyRef) { val list = obj.asInstanceOf[List[AnyRef]] //Write the size: @@ -132,7 +132,7 @@ class ListSerializer extends KSerializer[AnyRef] { override def create(kser: Kryo, in: Input, cls: Class[AnyRef]) : AnyRef = { val size = in.readInt(true); - + //Produce the reversed list: if (size == 0) { /* @@ -168,7 +168,7 @@ class DateRangeSerializer() extends KSerializer[DateRange] { out.writeLong(range.start.value.getTime, true); out.writeLong(range.end.value.getTime, true); } - + override def create(kser: Kryo, in: Input, cls: Class[DateRange]): DateRange = { DateRange(RichDate(in.readLong(true)), RichDate(in.readLong(true))); } diff --git a/src/main/scala/com/twitter/scalding/Source.scala b/src/main/scala/com/twitter/scalding/Source.scala index a8586f3548..7212420d5e 100644 --- a/src/main/scala/com/twitter/scalding/Source.scala +++ b/src/main/scala/com/twitter/scalding/Source.scala @@ -75,7 +75,7 @@ abstract class Source extends java.io.Serializable { def localScheme : LocalScheme = { error("Cascading local mode not supported for: " + toString) } - def hdfsScheme : Scheme[_ <: FlowProcess[JobConf],JobConf,_,_,_,_] = { + def hdfsScheme : Scheme[FlowProcess[JobConf],JobConf,RecordReader[_,_],OutputCollector[_,_],_,_] = { error("Cascading Hadoop mode not supported for: " + toString) } From 4ec6a75fd48d8ecf03bab7a00ea0a1253c09b9b2 Mon Sep 17 00:00:00 2001 From: Argyris Zymnis Date: Tue, 24 Apr 2012 20:08:56 -0700 Subject: [PATCH 25/25] Bump version to 0.5.0. --- CHANGES.md | 15 ++++++++++++--- README.md | 8 ++++---- build.sbt | 2 +- scripts/scald.rb | 2 +- 4 files changed, 18 insertions(+), 9 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index 6004618efb..4df66efd40 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,9 +1,18 @@ # Scalding # +### Versions 0.5.0 ### +* ISSUE 67: Upgrade cascading to wip-281. +* ISSUE 66: Allow default time zone in DefaultDateRangeJob. +* ISSUE 65: Fixed the error message thrown by FileSource.validateTaps. +* ISSUE 62: Kryo Upgrade to 2.04 +* ISSUE 60: Feature/abstract algebra +* ISSUE 52: Feature/cogroup builder +* ISSUE 51: Feature/headfix + ### Version 0.4.1 ### -* ISSUE 42 Feature/iterable source -* ISSUE 41 Adds blockJoinWithSmaller to JoinAlgorithms. -* ISSUE 39 Adding default value to pivot +* ISSUE 42: Feature/iterable source +* ISSUE 41: Adds blockJoinWithSmaller to JoinAlgorithms. +* ISSUE 39: Adding default value to pivot ### Version 0.4.0 ### * ISSUE 38: Fix bug with hash code collisions of Source objects diff --git a/README.md b/README.md index dbd8ce6bfc..281e582cff 100644 --- a/README.md +++ b/README.md @@ -4,7 +4,7 @@ Scalding is a Scala library that makes it easy to write MapReduce jobs in Hadoop Scalding is built on top of [Cascading](http://www.cascading.org/), a Java library that abstracts away much of the complexity of Hadoop. -Current version: 0.4.1 +Current version: 0.5.0 ## Word Count @@ -20,12 +20,12 @@ class WordCountJob(args : Args) extends Job(args) { .flatMap('line -> 'word) { line : String => tokenize(line) } .groupBy('word) { _.size } .write( Tsv( args("output") ) ) - + // Split a piece of text into individual words. def tokenize(text : String) : Array[String] = { // Lowercase each word and remove punctuation. text.toLowerCase.replaceAll("[^a-zA-Z0-9\\s]", "").split("\\s+") - } + } } ``` @@ -49,7 +49,7 @@ You can find more example code under [examples/](https://github.com/twitter/scal We use [Travis CI](http://travis-ci.org/) to verify the build: [![Build Status](https://secure.travis-ci.org/twitter/scalding.png)](http://travis-ci.org/twitter/scalding) -The current version is 0.4.1 and is available from maven central: org="com.twitter", +The current version is 0.5.0 and is available from maven central: org="com.twitter", artifact="scalding_2.8.1" or artifact="scalding_2.9.1". ## Contact diff --git a/build.sbt b/build.sbt index bac55dfa1e..f25c4e3d2a 100644 --- a/build.sbt +++ b/build.sbt @@ -2,7 +2,7 @@ import AssemblyKeys._ name := "scalding" -version := "0.4.1" +version := "0.5.0" organization := "com.twitter" diff --git a/scripts/scald.rb b/scripts/scald.rb index 0bbbe50cdf..8566fb5a6b 100755 --- a/scripts/scald.rb +++ b/scripts/scald.rb @@ -2,7 +2,7 @@ require 'fileutils' require 'thread' -SCALDING_VERSION="0.4.1" +SCALDING_VERSION="0.5.0" #Usage : scald.rb [--hdfs|--local|--print] job # --hdfs: if job ends in ".scala" or ".java" and the file exists, link it against JARFILE (below) and then run it on HOST.