From c58b09ac83adb40fb3a19b9b7da51c0fd30b8bf2 Mon Sep 17 00:00:00 2001 From: Ruban Monu Date: Wed, 4 Feb 2015 14:14:17 -0800 Subject: [PATCH 001/177] Execution.unit to object --- .../src/main/scala/com/twitter/scalding/Execution.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala b/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala index a4f3aba941..2f248728ae 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala @@ -423,6 +423,9 @@ object Execution { */ def fromFuture[T](fn: ConcurrentExecutionContext => Future[T]): Execution[T] = FutureConst(fn) + /** This creates a constant Execution[Unit] */ + def unit: Execution[Unit] = from(()) + private[scalding] def factory[T](fn: (Config, Mode) => Execution[T]): Execution[T] = FactoryExecution(fn) From 7481bd1833fc67a48e01c99879bf26ee662e82fb Mon Sep 17 00:00:00 2001 From: Ruban Monu Date: Wed, 4 Feb 2015 14:58:24 -0800 Subject: [PATCH 002/177] change to val --- .../src/main/scala/com/twitter/scalding/Execution.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala b/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala index 2f248728ae..44261cdb75 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala @@ -423,8 +423,8 @@ object Execution { */ def fromFuture[T](fn: ConcurrentExecutionContext => Future[T]): Execution[T] = FutureConst(fn) - /** This creates a constant Execution[Unit] */ - def unit: Execution[Unit] = from(()) + /** Returns a constant Execution[Unit] */ + val unit: Execution[Unit] = from(()) private[scalding] def factory[T](fn: (Config, Mode) => Execution[T]): Execution[T] = FactoryExecution(fn) From 500ae654c83f60b23bb5b37b94f388b442dd7169 Mon Sep 17 00:00:00 2001 From: Stephanie Bian Date: Thu, 5 Feb 2015 10:12:40 -0800 Subject: [PATCH 003/177] Override hashCode for Args Currently, Args overrides equals but does not override hashCode. This leads to unexpected behavior when hashing and checking for equality on objects that depend on Args. --- scalding-args/src/main/scala/com/twitter/scalding/Args.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/scalding-args/src/main/scala/com/twitter/scalding/Args.scala b/scalding-args/src/main/scala/com/twitter/scalding/Args.scala index 98f09c941b..6d71596b2a 100644 --- a/scalding-args/src/main/scala/com/twitter/scalding/Args.scala +++ b/scalding-args/src/main/scala/com/twitter/scalding/Args.scala @@ -112,6 +112,8 @@ class Args(val m: Map[String, List[String]]) extends java.io.Serializable { } } + override def hashCode(): Int = m.hashCode() + /** * Equivalent to .optional(key).getOrElse(default) */ From e6a374bf09c786ba10621c83100d30d6b1ac3c05 Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Fri, 6 Feb 2015 14:54:27 -1000 Subject: [PATCH 004/177] Put a value in a exception message --- .../src/main/scala/com/twitter/scalding/RichPipe.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/RichPipe.scala b/scalding-core/src/main/scala/com/twitter/scalding/RichPipe.scala index a867cad1e2..4827478d65 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/RichPipe.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/RichPipe.scala @@ -53,7 +53,7 @@ object RichPipe extends java.io.Serializable { p.getStepConfigDef() .setProperty(REDUCER_KEY, reducers.toString) } else if (reducers != -1) { - throw new IllegalArgumentException("Number of reducers must be non-negative") + throw new IllegalArgumentException(s"Number of reducers must be non-negative. Got: ${reducers}") } p } From 4557075308c006e6a7efb98e316d3606313b72b6 Mon Sep 17 00:00:00 2001 From: Ian O Connell Date: Tue, 10 Feb 2015 11:06:20 -0800 Subject: [PATCH 005/177] Move main simple example to be the typed API, and put the .'s at the start of lines like we usually do --- .../com/twitter/scalding/examples/WordCountJob.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/examples/WordCountJob.scala b/scalding-core/src/main/scala/com/twitter/scalding/examples/WordCountJob.scala index 8801642b38..f928d4c841 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/examples/WordCountJob.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/examples/WordCountJob.scala @@ -3,8 +3,9 @@ package com.twitter.scalding.examples import com.twitter.scalding._ class WordCountJob(args: Args) extends Job(args) { - TextLine(args("input")).read. - flatMap('line -> 'word) { line: String => line.split("\\s+") }. - groupBy('word) { _.size }. - write(Tsv(args("output"))) + TypedPipe.from(TextLine(args("input"))) + .flatMap { line: String => line.split("\\s+") } + .map (_ -> 1L) + .sumByKey + .write(TypedTsv[(String, Long)](args("output"))) } From ff15e90061f3a237329cf3e713c5b662e15f11d4 Mon Sep 17 00:00:00 2001 From: Ian O Connell Date: Tue, 10 Feb 2015 11:21:53 -0800 Subject: [PATCH 006/177] Review comments --- .../scala/com/twitter/scalding/examples/WordCountJob.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/examples/WordCountJob.scala b/scalding-core/src/main/scala/com/twitter/scalding/examples/WordCountJob.scala index f928d4c841..0c7f513e7f 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/examples/WordCountJob.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/examples/WordCountJob.scala @@ -4,8 +4,9 @@ import com.twitter.scalding._ class WordCountJob(args: Args) extends Job(args) { TypedPipe.from(TextLine(args("input"))) - .flatMap { line: String => line.split("\\s+") } - .map (_ -> 1L) + .flatMap { line => line.split("\\s+") } + .map { word => (word, 1L) } .sumByKey + // The compiler will enforce the type coming out of the sumByKey is the same as the type we have for our sink .write(TypedTsv[(String, Long)](args("output"))) } From e082878bfffc2ad21238cac63b6a8a14dd49aafe Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Tue, 10 Feb 2015 11:38:50 -1000 Subject: [PATCH 007/177] Add an exclusiveUpper method to DateRange --- .../main/scala/com/twitter/scalding/DateRange.scala | 7 +++++++ .../scala/com/twitter/scalding/DateProperties.scala | 11 +++++++++++ 2 files changed, 18 insertions(+) diff --git a/scalding-date/src/main/scala/com/twitter/scalding/DateRange.scala b/scalding-date/src/main/scala/com/twitter/scalding/DateRange.scala index d8b0abf633..44d235e935 100644 --- a/scalding-date/src/main/scala/com/twitter/scalding/DateRange.scala +++ b/scalding-date/src/main/scala/com/twitter/scalding/DateRange.scala @@ -56,6 +56,13 @@ object DateRange extends java.io.Serializable { case Seq(o) => parse(o) case x => sys.error("--date must have exactly one or two date[time]s. Got: " + x.toString) } + + /** + * DateRanges are inclusive. Use this to create a DateRange that excludes + * the last millisecond from the second argument. + */ + def exclusiveUpper(include: RichDate, exclude: RichDate): DateRange = + DateRange(include, exclude - Millisecs(1)) } /** diff --git a/scalding-date/src/test/scala/com/twitter/scalding/DateProperties.scala b/scalding-date/src/test/scala/com/twitter/scalding/DateProperties.scala index 1e504e567c..3d3340efd6 100644 --- a/scalding-date/src/test/scala/com/twitter/scalding/DateProperties.scala +++ b/scalding-date/src/test/scala/com/twitter/scalding/DateProperties.scala @@ -113,6 +113,17 @@ object DateProperties extends Properties("Date Properties") { dr.start + dr.length - AbsoluteDuration.fromMillisecs(1L) == dr.end } + property("DateRange.exclusiveUpper works") = forAll { (a: RichDate, b: RichDate) => + val lower = Ordering[RichDate].min(a, b) + val upper = Ordering[RichDate].max(a, b) + val ex = DateRange.exclusiveUpper(lower, upper) + val in = DateRange(lower, upper) + val upperPred = upper - Millisecs(1) + + (false == ex.contains(upper)) && + (ex.contains(upperPred) || (lower == upper)) + } + def toRegex(glob: String) = (glob.flatMap { c => if (c == '*') ".*" else c.toString }).r def matches(l: List[String], arg: String): Int = l From 43266133b8de065c314cf42ed8210d4971ba809e Mon Sep 17 00:00:00 2001 From: Ian O Connell Date: Wed, 11 Feb 2015 12:10:54 -0800 Subject: [PATCH 008/177] Fixes test --- .../scala/com/twitter/scalding/examples/WordCountJob.scala | 4 ++-- .../src/test/scala/com/twitter/scalding/WordCountTest.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/examples/WordCountJob.scala b/scalding-core/src/main/scala/com/twitter/scalding/examples/WordCountJob.scala index 0c7f513e7f..83a8dd0175 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/examples/WordCountJob.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/examples/WordCountJob.scala @@ -4,9 +4,9 @@ import com.twitter.scalding._ class WordCountJob(args: Args) extends Job(args) { TypedPipe.from(TextLine(args("input"))) - .flatMap { line => line.split("\\s+") } + .flatMap { line => line.split("\\s+") } .map { word => (word, 1L) } .sumByKey - // The compiler will enforce the type coming out of the sumByKey is the same as the type we have for our sink + // The compiler will enforce the type coming out of the sumByKey is the same as the type we have for our sink .write(TypedTsv[(String, Long)](args("output"))) } diff --git a/scalding-core/src/test/scala/com/twitter/scalding/WordCountTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/WordCountTest.scala index 8f0f907c23..fec67ae0cf 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/WordCountTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/WordCountTest.scala @@ -23,7 +23,7 @@ class WordCountTest extends WordSpec with Matchers { .arg("input", "inputFile") .arg("output", "outputFile") .source(TextLine("inputFile"), List((0, "hack hack hack and hack"))) - .sink[(String, Int)](Tsv("outputFile")){ outputBuffer => + .sink[(String, Int)](TypedTsv[(String, Long)]("outputFile")){ outputBuffer => val outMap = outputBuffer.toMap "count words correctly" in { outMap("hack") shouldBe 4 From fbc7467da75915e5795ae8e337c293eccf4a925b Mon Sep 17 00:00:00 2001 From: Kevin Lin Date: Wed, 11 Feb 2015 23:03:46 -0800 Subject: [PATCH 009/177] Add tutorial example of calling Execution from stand-alone ExecutionApp. --- README.md | 7 +- project/Build.scala | 24 ++++++- .../ExecutionTutorial.scala | 64 +++++++++++++++++++ 3 files changed, 92 insertions(+), 3 deletions(-) create mode 100644 tutorial/execution-tutorial/ExecutionTutorial.scala diff --git a/README.md b/README.md index 7f86144067..33e4938c7a 100644 --- a/README.md +++ b/README.md @@ -37,16 +37,17 @@ You can find more example code under [examples/](https://github.com/twitter/scal ## Documentation and Getting Started * [**Getting Started**](https://github.com/twitter/scalding/wiki/Getting-Started) page on the [Scalding Wiki](https://github.com/twitter/scalding/wiki) +* [Scalding Scaladocs](http://twitter.github.com/scalding) provide details beyond the API References. Prefer using this as it's always up to date. * [**REPL in Wonderland**](https://gist.github.com/johnynek/a47699caa62f4f38a3e2) a hands-on tour of the scalding REPL requiring only git and java installed. * [**Runnable tutorials**](https://github.com/twitter/scalding/tree/master/tutorial) in the source. * The API Reference, including many example Scalding snippets: * [Type-safe API Reference](https://github.com/twitter/scalding/wiki/Type-safe-api-reference) * [Fields-based API Reference](https://github.com/twitter/scalding/wiki/Fields-based-API-Reference) -* [Scalding Scaladocs](http://twitter.github.com/scalding) provide details beyond the API References * The Matrix Library provides a way of working with key-attribute-value scalding pipes: * The [Introduction to Matrix Library](https://github.com/twitter/scalding/wiki/Introduction-to-Matrix-Library) contains an overview and a "getting started" example * The [Matrix API Reference](https://github.com/twitter/scalding/wiki/Matrix-API-Reference) contains the Matrix Library API reference with examples +* [**Introduction to Scalding Execution**](https://github.com/twitter/scalding/wiki/Calling-Scalding-from-inside-your-application) contains general rules and examples of calling Scalding from inside another application. Please feel free to use the beautiful [Scalding logo](https://drive.google.com/folderview?id=0B3i3pDi3yVgNbm9pMUdDcHFKVEk&usp=sharing) artwork anywhere. @@ -124,6 +125,9 @@ Thanks for assistance and contributions: * Sam Ritchie * Aaron Siegel: +* Ian O'Connell +* Alex Levenson +* Kevin Lin * Brad Greenlee: * Edwin Chen * Arkajit Dey: @@ -133,7 +137,6 @@ Thanks for assistance and contributions: * Ning Liang * Dmitriy Ryaboy * Dong Wang -* Kevin Lin * Josh Attenberg * Juliet Hougland diff --git a/project/Build.scala b/project/Build.scala index c349eafed7..6b30e295be 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -201,7 +201,8 @@ object ScaldingBuild extends Build { scaldingJdbc, scaldingHadoopTest, scaldingMacros, - maple + maple, + executionTutorial ) lazy val formattingPreferences = { @@ -430,4 +431,25 @@ object ScaldingBuild extends Build { ) } ) + + lazy val executionTutorial = Project( + id = "execution-tutorial", + base = file("tutorial/execution-tutorial"), + settings = sharedSettings + ).settings( + name := "execution-tutorial", + previousArtifact := None, + crossPaths := false, + autoScalaLibrary := false, + libraryDependencies <++= (scalaVersion) { scalaVersion => Seq( + "org.scala-lang" % "scala-library" % scalaVersion, + "org.scala-lang" % "scala-reflect" % scalaVersion, + "org.apache.hadoop" % "hadoop-core" % hadoopVersion % "provided", + "org.apache.hadoop" % "hadoop-minicluster" % hadoopVersion, + "org.slf4j" % "slf4j-api" % slf4jVersion, + "org.slf4j" % "slf4j-log4j12" % slf4jVersion, + "cascading" % "cascading-hadoop" % cascadingVersion + ) + } + ).dependsOn(scaldingCore) } diff --git a/tutorial/execution-tutorial/ExecutionTutorial.scala b/tutorial/execution-tutorial/ExecutionTutorial.scala new file mode 100644 index 0000000000..d1567b1574 --- /dev/null +++ b/tutorial/execution-tutorial/ExecutionTutorial.scala @@ -0,0 +1,64 @@ +/* +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.tutorial + +import java.io._ +import scala.util.{Failure, Success} + +import com.twitter.scalding._ +import com.twitter.algebird.monad.Reader + +/** +Tutorial of using Execution + +This tutorial gives an example of use Execution to do MapReduce word count. +Instead of writing the results in reducers, it writes the data at submitter node. + +To test it, first build the assembly jar from root directory: + ./sbt execution-tutorial/assembly + +Run: + scala -classpath tutorial/execution-tutorial/target/execution-tutorial-assembly-0.13.1.jar \ + com.twitter.scalding.tutorial.MyExecJob --local \ + --input tutorial/data/hello.txt \ + --output tutorial/data/execution_output.txt +**/ + +object MyExecJob extends ExecutionApp { + + override def job = Execution.getConfig.flatMap { config => + val args = config.getArgs + val file = new PrintWriter(new File(args("output"))) + + TypedPipe.from(TextLine(args("input"))) + .flatMap(_.split("\\s+")) + .map((_, 1L)) + .sumByKey + .toIterableExecution + .onComplete { t => t match { + case Success(iter) => + iter.foreach { case (k, v) => + file.write(s"$k\t$v\n") + } + file.close + case Failure(e) => println("Error: " + e.toString) + } + } + .unit + } +} + + From 4d21fd14046a987b99c22d5bcd6d6e0b18b2c794 Mon Sep 17 00:00:00 2001 From: Kevin Lin Date: Wed, 11 Feb 2015 23:23:43 -0800 Subject: [PATCH 010/177] Add jonathan. --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index 33e4938c7a..54a9e0c948 100644 --- a/README.md +++ b/README.md @@ -127,6 +127,7 @@ Thanks for assistance and contributions: * Aaron Siegel: * Ian O'Connell * Alex Levenson +* Jonathan Coveney * Kevin Lin * Brad Greenlee: * Edwin Chen From 38c348e87d883ca783b34d6eeb00dc2e914f2966 Mon Sep 17 00:00:00 2001 From: Ian O Connell Date: Thu, 12 Feb 2015 14:59:33 -0800 Subject: [PATCH 011/177] Config/Cascading updater --- .../scalding/CascadingTokenUpdater.scala | 61 +++++++++++++++++++ .../scala/com/twitter/scalding/Config.scala | 33 +++++++++- 2 files changed, 93 insertions(+), 1 deletion(-) create mode 100644 scalding-core/src/main/scala/com/twitter/scalding/CascadingTokenUpdater.scala diff --git a/scalding-core/src/main/scala/com/twitter/scalding/CascadingTokenUpdater.scala b/scalding-core/src/main/scala/com/twitter/scalding/CascadingTokenUpdater.scala new file mode 100644 index 0000000000..57d2907b46 --- /dev/null +++ b/scalding-core/src/main/scala/com/twitter/scalding/CascadingTokenUpdater.scala @@ -0,0 +1,61 @@ +/* +Copyright 2014 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 + +object CascadingTokenUpdater { + private final val lowestAllowed = 128 // cascading rules + + // Take a cascading string of tokens and turns it into a map + // from token index to class + private def parseTokens(tokClass: String): Map[Int, String] = + if (tokClass == null || tokClass.isEmpty) + Map[Int, String]() + else + tokClass + .split(",") + .view + .map { _.split("=") } + .map { ary => (ary(0).toInt, ary(1)) } + .toMap + + // does the inverse of the previous function, given a Map of index to class + // return the cascading token format for it + private def toksToString(m: Map[Int, String]): String = + m.map { case (tok, clazz) => tok.toString + "=" + clazz }.mkString(",") + + // Given the map of already assigned tokens, what is the next available one + private def firstAvailableToken(m: Map[Int, String]): Int = + if (m.isEmpty) lowestAllowed + else scala.math.max(m.keys.toList.max + 1, lowestAllowed) + + // Given the first free token spot + // assign each of the class names given to al the subsequent + // positions + private def assignTokens(first: Int, names: Iterable[String]): Map[Int, String] = + names.foldLeft((first, Map[Int, String]())) { (idMap, clz) => + val (id, m) = idMap + (id + 1, m + (id -> clz)) + }._2 + + def update(config: Config, clazzes: Set[Class[_]]): Config = { + val toks: Map[Int, String] = parseTokens(config.get(Config.CascadingSerializationTokens).map(_.toString).orNull) + // We don't want to assign tokens to classes already in the map + val newClasses: Iterable[String] = clazzes.map { _.getName } -- toks.values + + config + (Config.CascadingSerializationTokens -> toksToString(toks ++ assignTokens(firstAvailableToken(toks), newClasses))) + } + +} diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala index 1a286d5085..75b02aaa0b 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala @@ -110,6 +110,29 @@ trait Config { def setMapSideAggregationThreshold(count: Int): Config = this + (AggregateBy.AGGREGATE_BY_THRESHOLD -> count.toString) + /** + * This function gets the set of classes that have been registered to Kryo. + * They may or may not be used in this job, but Cascading might want to be made aware + * that these classes exist + */ + def getKryoRegisteredClasses: Set[Class[_]] = { + // Get an instance of the Kryo serializer (which is populated with registrations) + getKryo.map { kryo => + val cr = kryo.newKryo.getClassResolver + + @annotation.tailrec + def kryoClasses(idx: Int, acc: Set[Class[_]]): Set[Class[_]] = + Option(cr.getRegistration(idx)) match { + case present @ Some(reg) => kryoClasses(idx + 1, acc ++ present.map { _.getType } + .filterNot { _.isPrimitive } // Cascading handles primitives and arrays + .filterNot { _.isArray }) + case None => acc // The first null is the end of the line + } + + kryoClasses(0, Set[Class[_]]()) + }.getOrElse(Set()) + } + /* * Hadoop and Cascading serialization needs to be first, and the Kryo serialization * needs to be last and this method handles this for you: @@ -142,7 +165,11 @@ trait Config { case Left((bootstrap, inst)) => ConfiguredInstantiator.setSerialized(chillConf, bootstrap, inst) case Right(refl) => ConfiguredInstantiator.setReflect(chillConf, refl) } - Config(chillConf.toMap + hadoopKV) + val withKryo = Config(chillConf.toMap + hadoopKV) + + val kryoClasses = withKryo.getKryoRegisteredClasses + + addCascadingClassSerializationTokens(kryoClasses) } /* @@ -172,6 +199,9 @@ trait Config { def getUniqueId: Option[UniqueID] = get(UniqueID.UNIQUE_JOB_ID).map(UniqueID(_)) + def addCascadingClassSerializationTokens(clazzes: Set[Class[_]]): Config = + CascadingTokenUpdater.update(this, clazzes) + /* * This is *required* if you are using counters. You must use * the same UniqueID as you used when defining your jobs. @@ -252,6 +282,7 @@ trait Config { object Config { val CascadingAppName: String = "cascading.app.name" val CascadingAppId: String = "cascading.app.id" + val CascadingSerializationTokens = "cascading.serialization.tokens" val IoSerializationsKey: String = "io.serializations" val ScaldingFlowClassName: String = "scalding.flow.class.name" val ScaldingFlowClassSignature: String = "scalding.flow.class.signature" From d213ce1d718153db3b0d7db080b6a425b3dd778c Mon Sep 17 00:00:00 2001 From: Ian O Connell Date: Thu, 12 Feb 2015 15:36:17 -0800 Subject: [PATCH 012/177] Review comments, use string interpolation --- .../com/twitter/scalding/CascadingTokenUpdater.scala | 9 ++++++--- .../src/main/scala/com/twitter/scalding/Config.scala | 2 +- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/CascadingTokenUpdater.scala b/scalding-core/src/main/scala/com/twitter/scalding/CascadingTokenUpdater.scala index 57d2907b46..3b1aacb44c 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/CascadingTokenUpdater.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/CascadingTokenUpdater.scala @@ -26,15 +26,18 @@ object CascadingTokenUpdater { else tokClass .split(",") - .view - .map { _.split("=") } + .map(_.trim) + .filter(_.size > 1) + .toIterator + .map(_.split("=")) + .filter(_.size == 2) .map { ary => (ary(0).toInt, ary(1)) } .toMap // does the inverse of the previous function, given a Map of index to class // return the cascading token format for it private def toksToString(m: Map[Int, String]): String = - m.map { case (tok, clazz) => tok.toString + "=" + clazz }.mkString(",") + m.map { case (tok, clazz) => s"$tok=$clazz" }.mkString(",") // Given the map of already assigned tokens, what is the next available one private def firstAvailableToken(m: Map[Int, String]): Int = diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala index 75b02aaa0b..7ac9dbe6ad 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala @@ -169,7 +169,7 @@ trait Config { val kryoClasses = withKryo.getKryoRegisteredClasses - addCascadingClassSerializationTokens(kryoClasses) + withKryo.addCascadingClassSerializationTokens(kryoClasses) } /* From f54305f1ec24274cc16ace2bd443fe0f5c51f1c6 Mon Sep 17 00:00:00 2001 From: Kevin Lin Date: Thu, 12 Feb 2015 15:37:26 -0800 Subject: [PATCH 013/177] Add execution_tutorial to travis test and update comments. --- .travis.yml | 6 ++++++ scripts/test_execution_tutorial.sh | 19 +++++++++++++++++++ .../ExecutionTutorial.scala | 3 ++- 3 files changed, 27 insertions(+), 1 deletion(-) create mode 100755 scripts/test_execution_tutorial.sh diff --git a/.travis.yml b/.travis.yml index 2fb94e5371..2c4c41e79e 100644 --- a/.travis.yml +++ b/.travis.yml @@ -97,5 +97,11 @@ matrix: - "scripts/build_assembly_no_test.sh scalding-core" - "scripts/test_typed_tutorials.sh" + - scala: 2.10.4 + env: BUILD="test execution tutorials" + script: + - "scripts/build_assembly_no_test.sh execution-tutorial" + - "scripts/test_execution_tutorial.sh" + notifications: irc: "chat.freenode.net#scalding" diff --git a/scripts/test_execution_tutorial.sh b/scripts/test_execution_tutorial.sh new file mode 100755 index 0000000000..256d6bb0bf --- /dev/null +++ b/scripts/test_execution_tutorial.sh @@ -0,0 +1,19 @@ +set -e # first error should stop execution of this script + +# Identify the bin dir in the distribution, and source the common include script +BASE_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )"/.. && pwd )" +source ${BASE_DIR}/scripts/common.sh + +# also trap errors, to reenable terminal settings +trap onExit ERR + +time scala -classpath tutorial/execution-tutorial/target/execution-tutorial-assembly-0.13.1.jar \ + com.twitter.scalding.tutorial.MyExecJob --local \ + --input tutorial/data/hello.txt \ + --output tutorial/data/execution_output.txt + +# restore stty +SCALA_EXIT_STATUS=0 +onExit + + diff --git a/tutorial/execution-tutorial/ExecutionTutorial.scala b/tutorial/execution-tutorial/ExecutionTutorial.scala index d1567b1574..1e4ec80364 100644 --- a/tutorial/execution-tutorial/ExecutionTutorial.scala +++ b/tutorial/execution-tutorial/ExecutionTutorial.scala @@ -19,7 +19,6 @@ import java.io._ import scala.util.{Failure, Success} import com.twitter.scalding._ -import com.twitter.algebird.monad.Reader /** Tutorial of using Execution @@ -48,6 +47,8 @@ object MyExecJob extends ExecutionApp { .map((_, 1L)) .sumByKey .toIterableExecution + // toIterableExecution will materialize the outputs to submitter node when finish. + // We can also write the outputs on HDFS via .writeExecution(TypedTsv(args("output"))) .onComplete { t => t match { case Success(iter) => iter.foreach { case (k, v) => From ebc64bbff4fe04909875a8ed799366bcb6bd4bfb Mon Sep 17 00:00:00 2001 From: Kevin Lin Date: Thu, 12 Feb 2015 23:38:12 -0800 Subject: [PATCH 014/177] Use java in travis test. --- scripts/test_execution_tutorial.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scripts/test_execution_tutorial.sh b/scripts/test_execution_tutorial.sh index 256d6bb0bf..aade8e957f 100755 --- a/scripts/test_execution_tutorial.sh +++ b/scripts/test_execution_tutorial.sh @@ -7,7 +7,7 @@ source ${BASE_DIR}/scripts/common.sh # also trap errors, to reenable terminal settings trap onExit ERR -time scala -classpath tutorial/execution-tutorial/target/execution-tutorial-assembly-0.13.1.jar \ +time java-classpath tutorial/execution-tutorial/target/execution-tutorial-assembly-0.13.1.jar \ com.twitter.scalding.tutorial.MyExecJob --local \ --input tutorial/data/hello.txt \ --output tutorial/data/execution_output.txt From f6bfd1aa4cee903cd994105b1a1134fd6b16c198 Mon Sep 17 00:00:00 2001 From: Kevin Lin Date: Fri, 13 Feb 2015 11:53:25 -0800 Subject: [PATCH 015/177] Make travis test work. --- scripts/test_execution_tutorial.sh | 11 ++++++----- tutorial/execution-tutorial/ExecutionTutorial.scala | 2 +- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/scripts/test_execution_tutorial.sh b/scripts/test_execution_tutorial.sh index aade8e957f..54aa653ecb 100755 --- a/scripts/test_execution_tutorial.sh +++ b/scripts/test_execution_tutorial.sh @@ -6,11 +6,12 @@ source ${BASE_DIR}/scripts/common.sh # also trap errors, to reenable terminal settings trap onExit ERR - -time java-classpath tutorial/execution-tutorial/target/execution-tutorial-assembly-0.13.1.jar \ - com.twitter.scalding.tutorial.MyExecJob --local \ - --input tutorial/data/hello.txt \ - --output tutorial/data/execution_output.txt +export CLASSPATH=tutorial/execution-tutorial/target/execution-tutorial-assembly-0.13.1.jar +time $SCALD \ + --tool com.twitter.scalding.tutorial.MyExecJob \ + NOARG \ + --input tutorial/data/hello.txt \ + --output tutorial/data/execution_output.txt # restore stty SCALA_EXIT_STATUS=0 diff --git a/tutorial/execution-tutorial/ExecutionTutorial.scala b/tutorial/execution-tutorial/ExecutionTutorial.scala index 1e4ec80364..09ec0557fd 100644 --- a/tutorial/execution-tutorial/ExecutionTutorial.scala +++ b/tutorial/execution-tutorial/ExecutionTutorial.scala @@ -40,7 +40,6 @@ object MyExecJob extends ExecutionApp { override def job = Execution.getConfig.flatMap { config => val args = config.getArgs - val file = new PrintWriter(new File(args("output"))) TypedPipe.from(TextLine(args("input"))) .flatMap(_.split("\\s+")) @@ -51,6 +50,7 @@ object MyExecJob extends ExecutionApp { // We can also write the outputs on HDFS via .writeExecution(TypedTsv(args("output"))) .onComplete { t => t match { case Success(iter) => + val file = new PrintWriter(new File(args("output"))) iter.foreach { case (k, v) => file.write(s"$k\t$v\n") } From b579e31929a4d3fa90dd74d85d50a46388c79221 Mon Sep 17 00:00:00 2001 From: Kevin Lin Date: Fri, 13 Feb 2015 13:49:10 -0800 Subject: [PATCH 016/177] Also build scalding-core. --- .travis.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.travis.yml b/.travis.yml index 2c4c41e79e..5ec69944df 100644 --- a/.travis.yml +++ b/.travis.yml @@ -100,6 +100,7 @@ matrix: - scala: 2.10.4 env: BUILD="test execution tutorials" script: + - "scripts/build_assembly_no_test.sh scalding-core" - "scripts/build_assembly_no_test.sh execution-tutorial" - "scripts/test_execution_tutorial.sh" From 51a2a4181b677a15c8f400d2968a3ec240f05901 Mon Sep 17 00:00:00 2001 From: Kevin Lin Date: Fri, 13 Feb 2015 17:22:57 -0800 Subject: [PATCH 017/177] Partial of Oscar and Alex's comments. --- project/Build.scala | 6 +----- scripts/test_execution_tutorial.sh | 9 ++++----- tutorial/execution-tutorial/ExecutionTutorial.scala | 1 + 3 files changed, 6 insertions(+), 10 deletions(-) diff --git a/project/Build.scala b/project/Build.scala index 6b30e295be..f3125f01d6 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -438,14 +438,10 @@ object ScaldingBuild extends Build { settings = sharedSettings ).settings( name := "execution-tutorial", - previousArtifact := None, - crossPaths := false, - autoScalaLibrary := false, libraryDependencies <++= (scalaVersion) { scalaVersion => Seq( "org.scala-lang" % "scala-library" % scalaVersion, "org.scala-lang" % "scala-reflect" % scalaVersion, - "org.apache.hadoop" % "hadoop-core" % hadoopVersion % "provided", - "org.apache.hadoop" % "hadoop-minicluster" % hadoopVersion, + "org.apache.hadoop" % "hadoop-core" % hadoopVersion, "org.slf4j" % "slf4j-api" % slf4jVersion, "org.slf4j" % "slf4j-log4j12" % slf4jVersion, "cascading" % "cascading-hadoop" % cascadingVersion diff --git a/scripts/test_execution_tutorial.sh b/scripts/test_execution_tutorial.sh index 54aa653ecb..334b40c43f 100755 --- a/scripts/test_execution_tutorial.sh +++ b/scripts/test_execution_tutorial.sh @@ -7,11 +7,10 @@ source ${BASE_DIR}/scripts/common.sh # also trap errors, to reenable terminal settings trap onExit ERR export CLASSPATH=tutorial/execution-tutorial/target/execution-tutorial-assembly-0.13.1.jar -time $SCALD \ - --tool com.twitter.scalding.tutorial.MyExecJob \ - NOARG \ - --input tutorial/data/hello.txt \ - --output tutorial/data/execution_output.txt +time java -jar tutorial/execution-tutorial/target/execution-tutorial-assembly-0.13.1.jar \ + com.twitter.scalding.tutorial.MyExecJob --local \ + --input tutorial/data/hello.txt \ + --output tutorial/data/execution_output.txt # restore stty SCALA_EXIT_STATUS=0 diff --git a/tutorial/execution-tutorial/ExecutionTutorial.scala b/tutorial/execution-tutorial/ExecutionTutorial.scala index 09ec0557fd..f27ee2da8e 100644 --- a/tutorial/execution-tutorial/ExecutionTutorial.scala +++ b/tutorial/execution-tutorial/ExecutionTutorial.scala @@ -58,6 +58,7 @@ object MyExecJob extends ExecutionApp { case Failure(e) => println("Error: " + e.toString) } } + // use the result and map it to a Unit. Otherwise the onComplete call won't happen .unit } } From 0744cf0343931a1bfe9e1ed024d0da03946f8ac1 Mon Sep 17 00:00:00 2001 From: adam Date: Sun, 15 Feb 2015 11:02:45 +0100 Subject: [PATCH 018/177] add LookupJoin and LookupJoinTest changes from summingbird --- .../twitter/scalding/typed/LookupJoin.scala | 145 ++++++++--- .../com/twitter/scalding/LookupJoinTest.scala | 231 +++++++++++++++--- 2 files changed, 303 insertions(+), 73 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/LookupJoin.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/LookupJoin.scala index 1ada8894e8..1bf2cd2b74 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/LookupJoin.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/LookupJoin.scala @@ -18,6 +18,26 @@ package com.twitter.scalding.typed import java.io.Serializable +import com.twitter.algebird.Semigroup + +/* + Copyright 2013 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. + */ + + + /** * lookupJoin simulates the behavior of a realtime system attempting * to leftJoin (K, V) pairs against some other value type (JoinedV) @@ -30,7 +50,7 @@ import java.io.Serializable * * To simulate this behavior, lookupJoin accepts pipes of key-value * pairs with an explicit time value T attached. T must have some - * sensible ordering. The semantics are, if one were to hit the + * sensical ordering. The semantics are, if one were to hit the * right pipe's simulated realtime service at any time between * T(tuple) T(tuple + 1), one would receive Some((K, * JoinedV)(tuple)). @@ -38,7 +58,7 @@ import java.io.Serializable * The entries in the left pipe's tuples have the following * meaning: * - * T: The time at which the (K, W) lookup occurred. + * T: The the time at which the (K, W) lookup occurred. * K: the join key. * W: the current value for the join key. * @@ -53,16 +73,56 @@ import java.io.Serializable * right side will return None only if the key is absent, * else, the service will return Some(joinedV). */ + + object LookupJoin extends Serializable { + + /** + * This is the "infinite history" join and always joins regardless of how + * much time is between the left and the right + */ + def apply[T: Ordering, K: Ordering, V, JoinedV]( left: TypedPipe[(T, (K, V))], right: TypedPipe[(T, (K, JoinedV))], - reducers: Option[Int] = None): TypedPipe[(T, (K, (V, Option[JoinedV])))] = { + reducers: Option[Int] = None): TypedPipe[(T, (K, (V, Option[JoinedV])))] = + + withWindow(left, right, reducers)((_, _) => true) + /** + * In this case, the right pipe is fed through a scanLeft doing a Semigroup.plus + * before joined to the left + */ + def rightSumming[T: Ordering, K: Ordering, V, JoinedV: Semigroup](left: TypedPipe[(T, (K, V))], + right: TypedPipe[(T, (K, JoinedV))], + reducers: Option[Int] = None): TypedPipe[(T, (K, (V, Option[JoinedV])))] = + withWindowRightSumming(left, right, reducers)((_, _) => true) + + /** + * This ensures that gate(Tleft, Tright) == true, else the None is emitted + * as the joined value. + * Useful for bounding the time of the join to a recent window + */ + def withWindow[T: Ordering, K: Ordering, V, JoinedV](left: TypedPipe[(T, (K, V))], + right: TypedPipe[(T, (K, JoinedV))], + reducers: Option[Int] = None)(gate: (T, T) => Boolean): TypedPipe[(T, (K, (V, Option[JoinedV])))] = { + + implicit val keepNew: Semigroup[JoinedV] = Semigroup.from { (older, newer) => newer } + withWindowRightSumming(left, right, reducers)(gate) + } + + /** + * This ensures that gate(Tleft, Tright) == true, else the None is emitted + * as the joined value, and sums are only done as long as they they come + * within the gate interval as well + */ + def withWindowRightSumming[T: Ordering, K: Ordering, V, JoinedV: Semigroup](left: TypedPipe[(T, (K, V))], + right: TypedPipe[(T, (K, JoinedV))], + reducers: Option[Int] = None)(gate: (T, T) => Boolean): TypedPipe[(T, (K, (V, Option[JoinedV])))] = { /** * Implicit ordering on an either that doesn't care about the - * actual container values, puts the lookups before the service - * writes Since we assume it takes non-zero time to do a lookup. + * actual container values, puts the lookups before the service writes + * Since we assume it takes non-zero time to do a lookup. */ implicit def eitherOrd[T, U]: Ordering[Either[T, U]] = new Ordering[Either[T, U]] { @@ -75,12 +135,15 @@ object LookupJoin extends Serializable { } } - val joined: TypedPipe[(K, (Option[JoinedV], Option[(T, V, Option[JoinedV])]))] = + val joined: TypedPipe[(K, (Option[(T, JoinedV)], Option[(T, V, Option[JoinedV])]))] = left.map { case (t, (k, v)) => (k, (t, Left(v): Either[V, JoinedV])) } - .++(right.map { case (t, (k, joinedV)) => (k, (t, Right(joinedV): Either[V, JoinedV])) }) + .++(right.map { + case (t, (k, joinedV)) => + (k, (t, Right(joinedV): Either[V, JoinedV])) + }) .group .withReducers(reducers.getOrElse(-1)) // -1 means default in scalding - .sortBy(identity) // time then left before right + .sortBy(identity) /** * Grouping by K leaves values of (T, Either[V, JoinedV]). Sort * by time and scanLeft. The iterator will now represent pairs of @@ -99,30 +162,46 @@ object LookupJoin extends Serializable { * JoinedV is updated and Some(newValue) when a (K, V) * shows up and a new join occurs. */ - (None: Option[JoinedV], None: Option[(T, V, Option[JoinedV])])) { - case ((lastJoined, _), (thisTime, leftOrRight)) => - leftOrRight match { - // Left(v) means that we have a new value from the left - // pipe that we need to join against the current - // "lastJoined" value sitting in scanLeft's state. This - // is equivalent to a lookup on the data in the right - // pipe at time "thisTime". - case Left(v) => (lastJoined, Some((thisTime, v, lastJoined))) - - // Right(joinedV) means that we've received a new value - // to use in the simulated realtime service described in - // the comments above - case Right(joined) => (Some(joined), None) - } - }.toTypedPipe - - for { - // Now, get rid of residual state from the scanLeft above: - (k, (_, optV)) <- joined - - // filter out every event that produced a Right(delta) above, - // leaving only the leftJoin events that occurred above: - (t, v, optJoined) <- optV - } yield (t, (k, (v, optJoined))) + (Option.empty[(T, JoinedV)], Option.empty[(T, V, Option[JoinedV])]) + ) { + case ((None, result), (time, Left(v))) => { + // The was no value previously + (None, Some((time, v, None))) + } + + case ((prev @ Some((oldt, jv)), result), (time, Left(v))) => { + // Left(v) means that we have a new value from the left + // pipe that we need to join against the current + // "lastJoined" value sitting in scanLeft's state. This + // is equivalent to a lookup on the data in the right + // pipe at time "thisTime". + val filteredJoined = if (gate(time, oldt)) Some(jv) else None + (prev, Some((time, v, filteredJoined))) + } + + case ((None, result), (time, Right(joined))) => { + // There was no value before, so we just update to joined + (Some((time, joined)), None) + } + + case ((Some((oldt, oldJ)), result), (time, Right(joined))) => { + // Right(joinedV) means that we've received a new value + // to use in the simulated realtime service + // described in the comments above + // did it fall out of cache? + val nextJoined = if (gate(time, oldt)) Semigroup.plus(oldJ, joined) else joined + (Some((time, nextJoined)), None) + } + }.toTypedPipe + + // Now, get rid of residual state from the scanLeft above: + joined.flatMap { + case (k, (_, optV)) => + // filter out every event that produced a Right(delta) above, + // leaving only the leftJoin events that occurred above: + optV.map { + case (t, v, optJoined) => (t, (k, (v, optJoined))) + } + } } } diff --git a/scalding-core/src/test/scala/com/twitter/scalding/LookupJoinTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/LookupJoinTest.scala index bb616a9fc0..abfe81dca4 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/LookupJoinTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/LookupJoinTest.scala @@ -18,55 +18,206 @@ package com.twitter.scalding import com.twitter.scalding.typed.LookupJoin import org.scalatest.{ Matchers, WordSpec } -class LookupJoinerJob(args: Args) extends Job(args) { - import TDsl._ +import com.twitter.algebird.{ Monoid, Semigroup, Group } - val in0 = TypedTsv[(Int, Int, Int)]("input0") - val in1 = TypedTsv[(Int, Int, Int)]("input1") +object LookupJoinedTest { - LookupJoin(TypedPipe.from(in0).map { case (t, k, v) => (t, (k, v)) }, - TypedPipe.from(in1).map { case (t, k, v) => (t, (k, v)) }) - .map { + // Not defined if there is a collision in K and T, so make those unique: + def genList(maxTime: Int, maxKey: Int, sz: Int): List[(Int, Int, Int)] = { + val rng = new java.util.Random + (0 until sz).view.map { _ => + (rng.nextInt(maxTime), rng.nextInt(maxKey), rng.nextInt) + } + .groupBy { case (t, k, v) => (t, k)} + .mapValues(_.headOption.toList) + .values + .flatten + .toList + } +} + + + class LookupJoinerJob(args: Args) extends Job(args) { + + import TDsl._ + + val in0 = TypedTsv[(Int, Int, Int)]("input0") + val in1 = TypedTsv[(Int, Int, Int)]("input1") + + LookupJoin(TypedPipe.from(in0).map { case (t, k, v) => (t, (k, v))}, + TypedPipe.from(in1).map { case (t, k, v) => (t, (k, v))}) + .map { case (t, (k, (v, opt))) => (t.toString, k.toString, v.toString, opt.toString) } - .write(TypedTsv[(String, String, String, String)]("output")) -} + .write(TypedTsv[(String, String, String, String)]("output")) -class LookupJoinedTest extends WordSpec with Matchers { - import Dsl._ - def lookupJoin[T: Ordering, K, V, W](in0: Iterable[(T, K, V)], in1: Iterable[(T, K, W)]) = { - // super inefficient, but easy to verify: - def lookup(t: T, k: K): Option[W] = { - implicit val ord = Ordering.by { tkw: (T, K, W) => tkw._1 } - in1.filter { case (t1, k1, _) => (k1 == k) && Ordering[T].lt(t1, t) } - .reduceOption(Ordering[(T, K, W)].max(_, _)) - .map { _._3 } + LookupJoin.rightSumming(TypedPipe.from(in0).map { case (t, k, v) => (t, (k, v))}, + TypedPipe.from(in1).map { case (t, k, v) => (t, (k, v))}) + .map { + case (t, (k, (v, opt))) => + (t.toString, k.toString, v.toString, opt.toString) } - in0.map { case (t, k, v) => (t.toString, k.toString, v.toString, lookup(t, k).toString) } + .write(TypedTsv[(String, String, String, String)]("output2")) } - "A LookupJoinerJob" should { - "correctly lookup" in { - val rng = new java.util.Random - val MAX_KEY = 10 - def genList(sz: Int): List[(Int, Int, Int)] = { - (0 until sz).map { _ => - (rng.nextInt, rng.nextInt(MAX_KEY), rng.nextInt) - }.toList + + class LookupJoinedTest extends WordSpec with Matchers { + + import Dsl._ + import LookupJoinedTest.genList + + def lookupJoin[T: Ordering, K, V, W](in0: Iterable[(T, K, V)], in1: Iterable[(T, K, W)]) = { + val serv = in1.groupBy(_._2) + def lookup(t: T, k: K): Option[W] = { + val ord = Ordering.by { tkw: (T, K, W) => tkw._1} + serv.get(k).flatMap { in1s => + in1s.filter { case (t1, _, _) => Ordering[T].lt(t1, t)} + .reduceOption(ord.max(_, _)) + .map { + _._3 + } + } } - val in0 = genList(1000) - val in1 = genList(1000) - JobTest(new LookupJoinerJob(_)) - .source(TypedTsv[(Int, Int, Int)]("input0"), in0) - .source(TypedTsv[(Int, Int, Int)]("input1"), in1) - .sink[(String, String, String, String)]( - TypedTsv[(String, String, String, String)]("output")) { outBuf => - outBuf.toSet shouldBe (lookupJoin(in0, in1).toSet) - in0 should have size (outBuf.size) + in0.map { case (t, k, v) => (t.toString, k.toString, v.toString, lookup(t, k).toString)} + } + + def lookupSumJoin[T: Ordering, K, V, W: Semigroup](in0: Iterable[(T, K, V)], in1: Iterable[(T, K, W)]) = { + implicit val ord: Ordering[(T, K, W)] = Ordering.by { + _._1 + } + val serv = in1.groupBy(_._2).mapValues { + _.toList + .sorted + .scanLeft(None: Option[(T, K, W)]) { (old, newer) => + old.map { case (_, _, w) => (newer._1, newer._2, Semigroup.plus(w, newer._3))} + .orElse(Some(newer)) + } + .filter { + _.isDefined + } + .map { + _.get + } + }.toMap // Force the map + + def lookup(t: T, k: K): Option[W] = { + val ord = Ordering.by { tkw: (T, K, W) => tkw._1} + serv.get(k).flatMap { in1s => + in1s.filter { case (t1, _, _) => Ordering[T].lt(t1, t)} + .reduceOption(ord.max(_, _)) + .map { + _._3 } - .run - .runHadoop - .finish + } + } + in0.map { case (t, k, v) => (t.toString, k.toString, v.toString, lookup(t, k).toString)} + } + + "A LookupJoinerJob" should { + "correctly lookup" in { + val MAX_KEY = 100 + val VAL_COUNT = 10000 + val in0 = genList(Int.MaxValue, MAX_KEY, VAL_COUNT) + val in1 = genList(Int.MaxValue, MAX_KEY, VAL_COUNT) + JobTest(new LookupJoinerJob(_)) + .source(TypedTsv[(Int, Int, Int)]("input0"), in0) + .source(TypedTsv[(Int, Int, Int)]("input1"), in1) + .sink[(String, String, String, String)]( + TypedTsv[(String, String, String, String)]("output")) { outBuf => + outBuf.toSet should equal (lookupJoin(in0, in1).toSet) + in0.size should equal (outBuf.size) + } + .sink[(String, String, String, String)]( + TypedTsv[(String, String, String, String)]("output2")) { outBuf => + outBuf.toSet should equal(lookupSumJoin(in0, in1).toSet) + in0.size should equal(outBuf.size) + } + .run + //.runHadoop + .finish + } } } -} + + class WindowLookupJoinerJob(args: Args) extends Job(args) { + + import TDsl._ + + val in0 = TypedTsv[(Int, Int, Int)]("input0") + val in1 = TypedTsv[(Int, Int, Int)]("input1") + val window = args("window").toInt + + def gate(left: Int, right: Int) = + (left.toLong - right.toLong) < window + + LookupJoin.withWindow(TypedPipe.from(in0).map { case (t, k, v) => (t, (k, v))}, + TypedPipe.from(in1).map { case (t, k, v) => (t, (k, v))})(gate _) + .map { + case (t, (k, (v, opt))) => + (t.toString, k.toString, v.toString, opt.toString) + } + .write(TypedTsv[(String, String, String, String)]("output")) + } + + class WindowLookupJoinedTest extends WordSpec with Matchers { + + import Dsl._ + import LookupJoinedTest.genList + + def windowLookupJoin[K, V, W](in0: Iterable[(Int, K, V)], in1: Iterable[(Int, K, W)], win: Int) = { + val serv = in1.groupBy(_._2) + // super inefficient, but easy to verify: + def lookup(t: Int, k: K): Option[W] = { + val ord = Ordering.by { tkw: (Int, K, W) => tkw._1} + serv.get(k).flatMap { in1s => + in1s.filter { + case (t1, _, _) => + (t1 < t) && ((t.toLong - t1.toLong) < win) + } + .reduceOption(ord.max(_, _)) + .map { + _._3 + } + } + } + in0.map { case (t, k, v) => (t.toString, k.toString, v.toString, lookup(t, k).toString)} + } + + "A WindowLookupJoinerJob" should { + //Set up the job: + "correctly lookup" in { + val MAX_KEY = 10 + val MAX_TIME = 10000 + val sz:Int = 10000; + val in0 = genList(MAX_TIME, MAX_KEY, 10000) + val in1 = genList(MAX_TIME, MAX_KEY, 10000) + JobTest(new WindowLookupJoinerJob(_)) + .arg("window", "100") + .source(TypedTsv[(Int, Int, Int)]("input0"), in0) + .source(TypedTsv[(Int, Int, Int)]("input1"), in1) + .sink[(String, String, String, String)]( + TypedTsv[(String, String, String, String)]("output")) { outBuf => + val results = outBuf.toList.sorted + val correct = windowLookupJoin(in0, in1, 100).toList.sorted + def some(it: List[(String, String, String, String)]) = + it.filter(_._4.startsWith("Some")) + + def none(it: List[(String, String, String, String)]) = + it.filter(_._4.startsWith("None")) + + some(results) shouldBe (some(correct)) + none(results) shouldBe (none(correct)) + in0.size should equal (outBuf.size) + } + .run + //.runHadoop + .finish + } + } + } + + + + + + From 53f790aaa0965a752d9054ea850e984ddefbbb17 Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Mon, 16 Feb 2015 15:49:08 -1000 Subject: [PATCH 019/177] Remove Travis IRC notifications --- .travis.yml | 3 --- 1 file changed, 3 deletions(-) diff --git a/.travis.yml b/.travis.yml index 2fb94e5371..560a43f05d 100644 --- a/.travis.yml +++ b/.travis.yml @@ -96,6 +96,3 @@ matrix: script: - "scripts/build_assembly_no_test.sh scalding-core" - "scripts/test_typed_tutorials.sh" - -notifications: - irc: "chat.freenode.net#scalding" From 0c28598c2ae8691bfbc0a329ec58ceac732c5915 Mon Sep 17 00:00:00 2001 From: Kevin Lin Date: Mon, 16 Feb 2015 21:31:14 -0800 Subject: [PATCH 020/177] Correct jar path. --- scripts/test_execution_tutorial.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/scripts/test_execution_tutorial.sh b/scripts/test_execution_tutorial.sh index 334b40c43f..cb31ba234d 100755 --- a/scripts/test_execution_tutorial.sh +++ b/scripts/test_execution_tutorial.sh @@ -6,8 +6,8 @@ source ${BASE_DIR}/scripts/common.sh # also trap errors, to reenable terminal settings trap onExit ERR -export CLASSPATH=tutorial/execution-tutorial/target/execution-tutorial-assembly-0.13.1.jar -time java -jar tutorial/execution-tutorial/target/execution-tutorial-assembly-0.13.1.jar \ +export CLASSPATH=tutorial/execution-tutorial/target/scala-#{SHORT_SCALA_VERSION}/execution-tutorial-assembly-#{SCALDING_VERSION}.jar +time java -jar tutorial/execution-tutorial/target/scala-2.10/execution-tutorial-assembly-0.13.1.jar \ com.twitter.scalding.tutorial.MyExecJob --local \ --input tutorial/data/hello.txt \ --output tutorial/data/execution_output.txt From ea3a7716f1456029df55653fed34105ac35eaf0b Mon Sep 17 00:00:00 2001 From: Kevin Lin Date: Mon, 16 Feb 2015 23:34:28 -0800 Subject: [PATCH 021/177] Use version variable. --- scripts/test_execution_tutorial.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scripts/test_execution_tutorial.sh b/scripts/test_execution_tutorial.sh index cb31ba234d..c086410cab 100755 --- a/scripts/test_execution_tutorial.sh +++ b/scripts/test_execution_tutorial.sh @@ -7,7 +7,7 @@ source ${BASE_DIR}/scripts/common.sh # also trap errors, to reenable terminal settings trap onExit ERR export CLASSPATH=tutorial/execution-tutorial/target/scala-#{SHORT_SCALA_VERSION}/execution-tutorial-assembly-#{SCALDING_VERSION}.jar -time java -jar tutorial/execution-tutorial/target/scala-2.10/execution-tutorial-assembly-0.13.1.jar \ +time java -jar tutorial/execution-tutorial/target/scala-#{SHORT_SCALA_VERSION}/execution-tutorial-assembly-#{SCALDING_VERSION}.jar \ com.twitter.scalding.tutorial.MyExecJob --local \ --input tutorial/data/hello.txt \ --output tutorial/data/execution_output.txt From 278eb71d216b2e52bd1123c93e6c4a7df66c8593 Mon Sep 17 00:00:00 2001 From: Adam Date: Tue, 17 Feb 2015 17:31:30 +0100 Subject: [PATCH 022/177] fix --- .../main/scala/com/twitter/scalding/typed/LookupJoin.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/LookupJoin.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/LookupJoin.scala index 1bf2cd2b74..32c9c26aa1 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/LookupJoin.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/LookupJoin.scala @@ -50,7 +50,7 @@ import com.twitter.algebird.Semigroup * * To simulate this behavior, lookupJoin accepts pipes of key-value * pairs with an explicit time value T attached. T must have some - * sensical ordering. The semantics are, if one were to hit the + * sensible ordering. The semantics are, if one were to hit the * right pipe's simulated realtime service at any time between * T(tuple) T(tuple + 1), one would receive Some((K, * JoinedV)(tuple)). @@ -58,7 +58,7 @@ import com.twitter.algebird.Semigroup * The entries in the left pipe's tuples have the following * meaning: * - * T: The the time at which the (K, W) lookup occurred. + * T: The time at which the (K, W) lookup occurred. * K: the join key. * W: the current value for the join key. * @@ -143,7 +143,7 @@ object LookupJoin extends Serializable { }) .group .withReducers(reducers.getOrElse(-1)) // -1 means default in scalding - .sortBy(identity) + .sorted /** * Grouping by K leaves values of (T, Either[V, JoinedV]). Sort * by time and scanLeft. The iterator will now represent pairs of From 350288697fb928ca74b9c5c81a34591569d1cfb3 Mon Sep 17 00:00:00 2001 From: Kevin Lin Date: Tue, 17 Feb 2015 10:43:01 -0800 Subject: [PATCH 023/177] Only use TRAVIS_SCALA_VERSION. --- scripts/test_execution_tutorial.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/scripts/test_execution_tutorial.sh b/scripts/test_execution_tutorial.sh index c086410cab..a085bb6231 100755 --- a/scripts/test_execution_tutorial.sh +++ b/scripts/test_execution_tutorial.sh @@ -6,8 +6,8 @@ source ${BASE_DIR}/scripts/common.sh # also trap errors, to reenable terminal settings trap onExit ERR -export CLASSPATH=tutorial/execution-tutorial/target/scala-#{SHORT_SCALA_VERSION}/execution-tutorial-assembly-#{SCALDING_VERSION}.jar -time java -jar tutorial/execution-tutorial/target/scala-#{SHORT_SCALA_VERSION}/execution-tutorial-assembly-#{SCALDING_VERSION}.jar \ +export CLASSPATH=tutorial/execution-tutorial/target/scala-${TRAVIS_SCALA_VERSION}/execution-tutorial-assembly-0.13.1.jar +time java -jar tutorial/execution-tutorial/target/scala-${TRAVIS_SCALA_VERSION}/execution-tutorial-assembly-0.13.1.jar \ com.twitter.scalding.tutorial.MyExecJob --local \ --input tutorial/data/hello.txt \ --output tutorial/data/execution_output.txt From aa1ddaa4fe35f8cff0ec0e9eada64fa9861fd06a Mon Sep 17 00:00:00 2001 From: Kevin Lin Date: Tue, 17 Feb 2015 13:32:40 -0800 Subject: [PATCH 024/177] Use shell to read scala and scalding versions. --- scripts/test_execution_tutorial.sh | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/scripts/test_execution_tutorial.sh b/scripts/test_execution_tutorial.sh index a085bb6231..a2178482e2 100755 --- a/scripts/test_execution_tutorial.sh +++ b/scripts/test_execution_tutorial.sh @@ -3,11 +3,16 @@ set -e # first error should stop execution of this script # Identify the bin dir in the distribution, and source the common include script BASE_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )"/.. && pwd )" source ${BASE_DIR}/scripts/common.sh +SHORT_SCALA_VERSION=${TRAVIS_SCALA_VERSION%.*} +SCALDING_VERSION=`cat ${BASE_DIR}/version.sbt` +SCALDING_VERSION=${SCALDING_VERSION#*\"} +SCALDING_VERSION=${SCALDING_VERSION%\"} + # also trap errors, to reenable terminal settings trap onExit ERR -export CLASSPATH=tutorial/execution-tutorial/target/scala-${TRAVIS_SCALA_VERSION}/execution-tutorial-assembly-0.13.1.jar -time java -jar tutorial/execution-tutorial/target/scala-${TRAVIS_SCALA_VERSION}/execution-tutorial-assembly-0.13.1.jar \ +export CLASSPATH=tutorial/execution-tutorial/target/scala-${SHORT_SCALA_VERSION}/execution-tutorial-assembly-${SCALDING_VERSION}.jar +time java -jar tutorial/execution-tutorial/target/scala-${SHORT_SCALA_VERSION}/execution-tutorial-assembly-${SCALDING_VERSION}.jar \ com.twitter.scalding.tutorial.MyExecJob --local \ --input tutorial/data/hello.txt \ --output tutorial/data/execution_output.txt From e6aad516305b4ddf0c8223f761c5644ce77f051f Mon Sep 17 00:00:00 2001 From: Kevin Lin Date: Tue, 17 Feb 2015 19:51:39 -0800 Subject: [PATCH 025/177] Remove scalind-core from travis build. --- .travis.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index 4a024468e7..365b2e9cd4 100644 --- a/.travis.yml +++ b/.travis.yml @@ -100,6 +100,5 @@ matrix: - scala: 2.10.4 env: BUILD="test execution tutorials" script: - - "scripts/build_assembly_no_test.sh scalding-core" - "scripts/build_assembly_no_test.sh execution-tutorial" - "scripts/test_execution_tutorial.sh" From 988b9f21ebea1154ca1875b71ae27e323ae3154e Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Thu, 19 Feb 2015 14:03:06 -1000 Subject: [PATCH 026/177] Add tests, minor refactor --- .../scalding/CascadingTokenUpdater.scala | 4 ++-- .../scala/com/twitter/scalding/Config.scala | 11 ++++++--- .../com/twitter/scalding/ConfigTest.scala | 24 +++++++++++++++++++ 3 files changed, 34 insertions(+), 5 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/CascadingTokenUpdater.scala b/scalding-core/src/main/scala/com/twitter/scalding/CascadingTokenUpdater.scala index 3b1aacb44c..7d035ba42a 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/CascadingTokenUpdater.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/CascadingTokenUpdater.scala @@ -20,7 +20,7 @@ object CascadingTokenUpdater { // Take a cascading string of tokens and turns it into a map // from token index to class - private def parseTokens(tokClass: String): Map[Int, String] = + def parseTokens(tokClass: String): Map[Int, String] = if (tokClass == null || tokClass.isEmpty) Map[Int, String]() else @@ -54,7 +54,7 @@ object CascadingTokenUpdater { }._2 def update(config: Config, clazzes: Set[Class[_]]): Config = { - val toks: Map[Int, String] = parseTokens(config.get(Config.CascadingSerializationTokens).map(_.toString).orNull) + val toks = config.getCascadingSerializationTokens // We don't want to assign tokens to classes already in the map val newClasses: Iterable[String] = clazzes.map { _.getName } -- toks.values diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala index 7ac9dbe6ad..f8e6423677 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala @@ -110,6 +110,11 @@ trait Config { def setMapSideAggregationThreshold(count: Int): Config = this + (AggregateBy.AGGREGATE_BY_THRESHOLD -> count.toString) + def getCascadingSerializationTokens: Map[Int, String] = + get(Config.CascadingSerializationTokens) + .map(CascadingTokenUpdater.parseTokens) + .getOrElse(Map.empty[Int, String]) + /** * This function gets the set of classes that have been registered to Kryo. * They may or may not be used in this job, but Cascading might want to be made aware @@ -123,9 +128,7 @@ trait Config { @annotation.tailrec def kryoClasses(idx: Int, acc: Set[Class[_]]): Set[Class[_]] = Option(cr.getRegistration(idx)) match { - case present @ Some(reg) => kryoClasses(idx + 1, acc ++ present.map { _.getType } - .filterNot { _.isPrimitive } // Cascading handles primitives and arrays - .filterNot { _.isArray }) + case Some(reg) => kryoClasses(idx + 1, acc + reg.getType) case None => acc // The first null is the end of the line } @@ -168,6 +171,8 @@ trait Config { val withKryo = Config(chillConf.toMap + hadoopKV) val kryoClasses = withKryo.getKryoRegisteredClasses + .filterNot(_.isPrimitive) // Cascading handles primitives and arrays + .filterNot(_.isArray) withKryo.addCascadingClassSerializationTokens(kryoClasses) } diff --git a/scalding-core/src/test/scala/com/twitter/scalding/ConfigTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/ConfigTest.scala index 9cc897317e..ed7fa01708 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/ConfigTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/ConfigTest.scala @@ -47,6 +47,30 @@ class ConfigTest extends WordSpec with Matchers { stillOld should contain (date) new2 shouldBe newConf } + "Default serialization should have tokens" in { + Config.default.getCascadingSerializationTokens should not be empty + Config.default.getCascadingSerializationTokens + .values + .map(Class.forName) + .filter(c => c.isPrimitive || c.isArray) shouldBe empty + + Config.empty.getCascadingSerializationTokens shouldBe empty + + // tokenClasses are a subset that don't include primites or arrays. + val tokenClasses = Config.default.getCascadingSerializationTokens.values.toSet + val kryoClasses = Config.default.getKryoRegisteredClasses.map(_.getName) + // Tokens are a subset of Kryo registered classes + (kryoClasses & tokenClasses) shouldBe tokenClasses + // the only Kryo classes we don't assign tokens for are the primitives + array + (kryoClasses -- tokenClasses).forall { c => + // primitives cannot be forName'd + val prim = Set(classOf[Boolean], classOf[Byte], classOf[Short], + classOf[Int], classOf[Long], classOf[Float], classOf[Double], classOf[Char]) + .map(_.getName) + + prim(c) || Class.forName(c).isArray + } shouldBe true + } } } From 35b23a1dc582512993482e6624da6d98835888c4 Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Thu, 19 Feb 2015 14:07:41 -1000 Subject: [PATCH 027/177] address some review comments --- .../scala/com/twitter/scalding/CascadingTokenUpdater.scala | 4 ++-- .../src/main/scala/com/twitter/scalding/Config.scala | 6 ++++++ 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/CascadingTokenUpdater.scala b/scalding-core/src/main/scala/com/twitter/scalding/CascadingTokenUpdater.scala index 7d035ba42a..f8f0a5da82 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/CascadingTokenUpdater.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/CascadingTokenUpdater.scala @@ -1,5 +1,5 @@ /* -Copyright 2014 Twitter, Inc. +Copyright 2015 Twitter, Inc. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. @@ -42,7 +42,7 @@ object CascadingTokenUpdater { // Given the map of already assigned tokens, what is the next available one private def firstAvailableToken(m: Map[Int, String]): Int = if (m.isEmpty) lowestAllowed - else scala.math.max(m.keys.toList.max + 1, lowestAllowed) + else scala.math.max(m.keys.max + 1, lowestAllowed) // Given the first free token spot // assign each of the class names given to al the subsequent diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala index f8e6423677..02ac5063b4 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala @@ -204,6 +204,12 @@ trait Config { def getUniqueId: Option[UniqueID] = get(UniqueID.UNIQUE_JOB_ID).map(UniqueID(_)) + /** + * The serialization of your data will be smaller if any classes passed between tasks in your job + * are listed here. Without this, strings are used to write the types IN EACH RECORD, which + * compression probably takes care of, but compression acts AFTER the data is serialized into + * buffers and spilling has been triggered. + */ def addCascadingClassSerializationTokens(clazzes: Set[Class[_]]): Config = CascadingTokenUpdater.update(this, clazzes) From 37908a6b5480a2a2adaaebcc87430487c0d41b6b Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Thu, 19 Feb 2015 14:17:18 -1000 Subject: [PATCH 028/177] Add Execution.withArgs --- .../com/twitter/scalding/Execution.scala | 7 ++ .../twitter/scalding/typed/LookupJoin.scala | 82 +++++++++---------- 2 files changed, 46 insertions(+), 43 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala b/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala index 44261cdb75..5715e03a0c 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala @@ -450,6 +450,13 @@ object Execution { /** Use this to get the config and mode. */ def getConfigMode: Execution[(Config, Mode)] = factory { case (conf, mode) => from((conf, mode)) } + /** + * This is convenience method only here to make it slightly cleaner + * to get Args, which are in the Config + */ + def withArgs[T](fn: Args => Execution[T]): Execution[T] = + getConfig.flatMap { conf => fn(conf.getArgs) } + /** * Use this to use counters/stats with Execution. You do this: * Execution.withId { implicit uid => diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/LookupJoin.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/LookupJoin.scala index 32c9c26aa1..4df34601a0 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/LookupJoin.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/LookupJoin.scala @@ -36,8 +36,6 @@ import com.twitter.algebird.Semigroup limitations under the License. */ - - /** * lookupJoin simulates the behavior of a realtime system attempting * to leftJoin (K, V) pairs against some other value type (JoinedV) @@ -74,7 +72,6 @@ import com.twitter.algebird.Semigroup * else, the service will return Some(joinedV). */ - object LookupJoin extends Serializable { /** @@ -94,8 +91,8 @@ object LookupJoin extends Serializable { * before joined to the left */ def rightSumming[T: Ordering, K: Ordering, V, JoinedV: Semigroup](left: TypedPipe[(T, (K, V))], - right: TypedPipe[(T, (K, JoinedV))], - reducers: Option[Int] = None): TypedPipe[(T, (K, (V, Option[JoinedV])))] = + right: TypedPipe[(T, (K, JoinedV))], + reducers: Option[Int] = None): TypedPipe[(T, (K, (V, Option[JoinedV])))] = withWindowRightSumming(left, right, reducers)((_, _) => true) /** @@ -104,8 +101,8 @@ object LookupJoin extends Serializable { * Useful for bounding the time of the join to a recent window */ def withWindow[T: Ordering, K: Ordering, V, JoinedV](left: TypedPipe[(T, (K, V))], - right: TypedPipe[(T, (K, JoinedV))], - reducers: Option[Int] = None)(gate: (T, T) => Boolean): TypedPipe[(T, (K, (V, Option[JoinedV])))] = { + right: TypedPipe[(T, (K, JoinedV))], + reducers: Option[Int] = None)(gate: (T, T) => Boolean): TypedPipe[(T, (K, (V, Option[JoinedV])))] = { implicit val keepNew: Semigroup[JoinedV] = Semigroup.from { (older, newer) => newer } withWindowRightSumming(left, right, reducers)(gate) @@ -117,8 +114,8 @@ object LookupJoin extends Serializable { * within the gate interval as well */ def withWindowRightSumming[T: Ordering, K: Ordering, V, JoinedV: Semigroup](left: TypedPipe[(T, (K, V))], - right: TypedPipe[(T, (K, JoinedV))], - reducers: Option[Int] = None)(gate: (T, T) => Boolean): TypedPipe[(T, (K, (V, Option[JoinedV])))] = { + right: TypedPipe[(T, (K, JoinedV))], + reducers: Option[Int] = None)(gate: (T, T) => Boolean): TypedPipe[(T, (K, (V, Option[JoinedV])))] = { /** * Implicit ordering on an either that doesn't care about the * actual container values, puts the lookups before the service writes @@ -138,9 +135,9 @@ object LookupJoin extends Serializable { val joined: TypedPipe[(K, (Option[(T, JoinedV)], Option[(T, V, Option[JoinedV])]))] = left.map { case (t, (k, v)) => (k, (t, Left(v): Either[V, JoinedV])) } .++(right.map { - case (t, (k, joinedV)) => - (k, (t, Right(joinedV): Either[V, JoinedV])) - }) + case (t, (k, joinedV)) => + (k, (t, Right(joinedV): Either[V, JoinedV])) + }) .group .withReducers(reducers.getOrElse(-1)) // -1 means default in scalding .sorted @@ -162,37 +159,36 @@ object LookupJoin extends Serializable { * JoinedV is updated and Some(newValue) when a (K, V) * shows up and a new join occurs. */ - (Option.empty[(T, JoinedV)], Option.empty[(T, V, Option[JoinedV])]) - ) { - case ((None, result), (time, Left(v))) => { - // The was no value previously - (None, Some((time, v, None))) - } - - case ((prev @ Some((oldt, jv)), result), (time, Left(v))) => { - // Left(v) means that we have a new value from the left - // pipe that we need to join against the current - // "lastJoined" value sitting in scanLeft's state. This - // is equivalent to a lookup on the data in the right - // pipe at time "thisTime". - val filteredJoined = if (gate(time, oldt)) Some(jv) else None - (prev, Some((time, v, filteredJoined))) - } - - case ((None, result), (time, Right(joined))) => { - // There was no value before, so we just update to joined - (Some((time, joined)), None) - } - - case ((Some((oldt, oldJ)), result), (time, Right(joined))) => { - // Right(joinedV) means that we've received a new value - // to use in the simulated realtime service - // described in the comments above - // did it fall out of cache? - val nextJoined = if (gate(time, oldt)) Semigroup.plus(oldJ, joined) else joined - (Some((time, nextJoined)), None) - } - }.toTypedPipe + (Option.empty[(T, JoinedV)], Option.empty[(T, V, Option[JoinedV])])) { + case ((None, result), (time, Left(v))) => { + // The was no value previously + (None, Some((time, v, None))) + } + + case ((prev @ Some((oldt, jv)), result), (time, Left(v))) => { + // Left(v) means that we have a new value from the left + // pipe that we need to join against the current + // "lastJoined" value sitting in scanLeft's state. This + // is equivalent to a lookup on the data in the right + // pipe at time "thisTime". + val filteredJoined = if (gate(time, oldt)) Some(jv) else None + (prev, Some((time, v, filteredJoined))) + } + + case ((None, result), (time, Right(joined))) => { + // There was no value before, so we just update to joined + (Some((time, joined)), None) + } + + case ((Some((oldt, oldJ)), result), (time, Right(joined))) => { + // Right(joinedV) means that we've received a new value + // to use in the simulated realtime service + // described in the comments above + // did it fall out of cache? + val nextJoined = if (gate(time, oldt)) Semigroup.plus(oldJ, joined) else joined + (Some((time, nextJoined)), None) + } + }.toTypedPipe // Now, get rid of residual state from the scanLeft above: joined.flatMap { From 7f10ec9c6cf511cc99285db90f2a72722facaced Mon Sep 17 00:00:00 2001 From: "adam.kozuch@gmail.com" Date: Tue, 24 Feb 2015 23:05:51 +0100 Subject: [PATCH 029/177] Remove algebird serializers --- project/Build.scala | 1 + .../com/twitter/scalding/serialization/KryoHadoop.scala | 7 +++++-- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/project/Build.scala b/project/Build.scala index f3125f01d6..9e9b7fffa4 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -250,6 +250,7 @@ object ScaldingBuild extends Build { "cascading" % "cascading-hadoop" % cascadingVersion, "com.twitter" %% "chill" % chillVersion, "com.twitter" % "chill-hadoop" % chillVersion, + "com.twitter" %% "chill-algebird" % chillVersion, "com.twitter" % "chill-java" % chillVersion, "com.twitter" %% "bijection-core" % bijectionVersion, "com.twitter" %% "algebird-core" % algebirdVersion, diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala index 97fc2055d3..6234ec0806 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala @@ -20,6 +20,8 @@ import java.io.OutputStream import java.io.Serializable import java.nio.ByteBuffer + +import com.twitter.chill.algebird._ import org.apache.hadoop.io.serializer.{ Serialization, Deserializer, Serializer, WritableSerialization } import com.esotericsoftware.kryo.Kryo @@ -38,8 +40,9 @@ import com.twitter.scalding.DateRange import com.twitter.scalding.RichDate import com.twitter.scalding.Args -import com.twitter.chill._ + import com.twitter.chill.config.Config +import com.twitter.chill.{SingletonSerializer, ScalaKryoInstantiator, KryoInstantiator, algebird} class KryoHadoop(config: Config) extends KryoInstantiator { @@ -61,7 +64,7 @@ class KryoHadoop(config: Config) extends KryoInstantiator { newK.register(classOf[DateRange], new DateRangeSerializer()) newK.register(classOf[Args], new ArgsSerializer) // Some of the monoids from Algebird that we use: - newK.register(classOf[com.twitter.algebird.AveragedValue], new AveragedValueSerializer) + newK.register(classOf[com.twitter.algebird.AveragedValue], new AveragedValueSerializer) newK.register(classOf[com.twitter.algebird.DecayedValue], new DecayedValueSerializer) newK.register(classOf[com.twitter.algebird.HyperLogLogMonoid], new HLLMonoidSerializer) newK.register(classOf[com.twitter.algebird.Moments], new MomentsSerializer) From 433dc35f01430213dfc715db18b913ff31ef73a2 Mon Sep 17 00:00:00 2001 From: "adam.kozuch@gmail.com" Date: Tue, 24 Feb 2015 23:13:14 +0100 Subject: [PATCH 030/177] fix --- .../com/twitter/scalding/serialization/KryoHadoop.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala index 6234ec0806..c9665784fc 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala @@ -21,7 +21,7 @@ import java.io.Serializable import java.nio.ByteBuffer -import com.twitter.chill.algebird._ + import org.apache.hadoop.io.serializer.{ Serialization, Deserializer, Serializer, WritableSerialization } import com.esotericsoftware.kryo.Kryo @@ -40,9 +40,9 @@ import com.twitter.scalding.DateRange import com.twitter.scalding.RichDate import com.twitter.scalding.Args - +import com.twitter.chill.algebird._ import com.twitter.chill.config.Config -import com.twitter.chill.{SingletonSerializer, ScalaKryoInstantiator, KryoInstantiator, algebird} +import com.twitter.chill.{SingletonSerializer, ScalaKryoInstantiator, KryoInstantiator} class KryoHadoop(config: Config) extends KryoInstantiator { From b29ae505c62cb0f8a35d72701f19176aa18160b9 Mon Sep 17 00:00:00 2001 From: Adam Date: Tue, 24 Feb 2015 23:16:04 +0100 Subject: [PATCH 031/177] fix --- .../serialization/AlgebirdSerializers.scala | 93 ------------------- 1 file changed, 93 deletions(-) delete mode 100644 scalding-core/src/main/scala/com/twitter/scalding/serialization/AlgebirdSerializers.scala diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/AlgebirdSerializers.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/AlgebirdSerializers.scala deleted file mode 100644 index 8b749123e2..0000000000 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/AlgebirdSerializers.scala +++ /dev/null @@ -1,93 +0,0 @@ -/* -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.serialization - -import com.esotericsoftware.kryo.Kryo -import com.esotericsoftware.kryo.{ Serializer => KSerializer } -import com.esotericsoftware.kryo.io.{ Input, Output } - -import com.twitter.algebird.{ - AveragedValue, - DecayedValue, - HLL, - HyperLogLog, - HyperLogLogMonoid, - Moments -} - -import scala.collection.mutable.{ Map => MMap } - -class AveragedValueSerializer extends KSerializer[AveragedValue] { - setImmutable(true) - def write(kser: Kryo, out: Output, s: AveragedValue) { - out.writeLong(s.count, true) - out.writeDouble(s.value) - } - def read(kser: Kryo, in: Input, cls: Class[AveragedValue]): AveragedValue = - AveragedValue(in.readLong(true), in.readDouble) -} - -class MomentsSerializer extends KSerializer[Moments] { - setImmutable(true) - def write(kser: Kryo, out: Output, s: Moments) { - out.writeLong(s.m0, true) - out.writeDouble(s.m1) - out.writeDouble(s.m2) - out.writeDouble(s.m3) - out.writeDouble(s.m4) - } - def read(kser: Kryo, in: Input, cls: Class[Moments]): Moments = { - Moments(in.readLong(true), - in.readDouble, - in.readDouble, - in.readDouble, - in.readDouble) - } -} - -class DecayedValueSerializer extends KSerializer[DecayedValue] { - setImmutable(true) - def write(kser: Kryo, out: Output, s: DecayedValue) { - out.writeDouble(s.value) - out.writeDouble(s.scaledTime) - } - def read(kser: Kryo, in: Input, cls: Class[DecayedValue]): DecayedValue = - DecayedValue(in.readDouble, in.readDouble) -} - -class HLLSerializer extends KSerializer[HLL] { - setImmutable(true) - def write(kser: Kryo, out: Output, s: HLL) { - val bytes = HyperLogLog.toBytes(s) - out.writeInt(bytes.size, true) - out.writeBytes(bytes) - } - def read(kser: Kryo, in: Input, cls: Class[HLL]): HLL = { - HyperLogLog.fromBytes(in.readBytes(in.readInt(true))) - } -} - -class HLLMonoidSerializer extends KSerializer[HyperLogLogMonoid] { - setImmutable(true) - val hllMonoids = MMap[Int, HyperLogLogMonoid]() - def write(kser: Kryo, out: Output, mon: HyperLogLogMonoid) { - out.writeInt(mon.bits, true) - } - def read(kser: Kryo, in: Input, cls: Class[HyperLogLogMonoid]): HyperLogLogMonoid = { - val bits = in.readInt(true) - hllMonoids.getOrElseUpdate(bits, new HyperLogLogMonoid(bits)) - } -} From 3042d6b5be76cb76f04b8e8cfdc8358bcdcdb5f2 Mon Sep 17 00:00:00 2001 From: Adam Date: Wed, 25 Feb 2015 06:21:34 +0100 Subject: [PATCH 032/177] fix --- .../scala/com/twitter/scalding/serialization/KryoHadoop.scala | 3 --- 1 file changed, 3 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala index c9665784fc..b3a86c6f58 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala @@ -20,8 +20,6 @@ import java.io.OutputStream import java.io.Serializable import java.nio.ByteBuffer - - import org.apache.hadoop.io.serializer.{ Serialization, Deserializer, Serializer, WritableSerialization } import com.esotericsoftware.kryo.Kryo @@ -45,7 +43,6 @@ import com.twitter.chill.config.Config import com.twitter.chill.{SingletonSerializer, ScalaKryoInstantiator, KryoInstantiator} class KryoHadoop(config: Config) extends KryoInstantiator { - /** * TODO!!! * Deal with this issue. The problem is grouping by Kryo serialized From a0d7fa6b07936ad44e486cbb9ee34345180aed6a Mon Sep 17 00:00:00 2001 From: Adam Date: Wed, 25 Feb 2015 14:29:17 +0100 Subject: [PATCH 033/177] fix --- .../scala/com/twitter/scalding/serialization/KryoHadoop.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala index b3a86c6f58..5dbe4241ab 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala @@ -66,7 +66,6 @@ class KryoHadoop(config: Config) extends KryoInstantiator { newK.register(classOf[com.twitter.algebird.HyperLogLogMonoid], new HLLMonoidSerializer) newK.register(classOf[com.twitter.algebird.Moments], new MomentsSerializer) newK.addDefaultSerializer(classOf[com.twitter.algebird.HLL], new HLLSerializer) - /** * AdaptiveVector is IndexedSeq, which picks up the chill IndexedSeq serializer * (which is its own bug), force using the fields serializer here From f39a5d2e4b27a4f558e11f282da363aa4e3b299b Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Wed, 25 Feb 2015 14:21:22 -1000 Subject: [PATCH 034/177] Commit non-macro ordered-serialization --- .travis.yml | 34 +- project/Build.scala | 7 +- .../scala/com/twitter/scalding/Config.scala | 16 +- .../twitter/scalding/ExecutionContext.scala | 6 + .../twitter/scalding/FieldConversions.scala | 6 + .../com/twitter/scalding/Operations.scala | 3 +- .../scalding/serialization/Boxed.scala | 470 ++++++++++++++++++ .../CascadingBinaryComparator.scala | 76 +++ .../scalding/serialization/Hasher.scala | 84 ++++ .../serialization/JavaStreamEnrichments.scala | 228 +++++++++ .../scalding/serialization/KryoHadoop.scala | 2 - .../twitter/scalding/serialization/Laws.scala | 31 ++ .../serialization/MurmerHashUtils.scala | 114 +++++ .../serialization/OrderedSerialization.scala | 147 ++++++ .../serialization/PositionInputStream.scala | 78 +++ .../scalding/serialization/Reader.scala | 130 +++++ .../serialization/Serialization.scala | 149 ++++++ .../serialization/Serialization2.scala | 64 +++ .../StringOrderedSerialization.scala | 116 +++++ .../serialization/UnsignedComparisons.scala | 49 ++ .../serialization/WrappedSerialization.scala | 121 +++++ .../scalding/serialization/Writer.scala | 128 +++++ .../twitter/scalding/typed/CoGrouped.scala | 139 +++--- .../com/twitter/scalding/typed/Grouped.scala | 109 +++- .../twitter/scalding/typed/HashJoinable.scala | 4 +- .../scalding/mathematics/SizeHintTest.scala | 2 +- .../JavaStreamEnrichmentsProperties.scala | 99 ++++ .../SerializationProperties.scala | 127 +++++ .../UnsignedComparisonLaws.scala | 41 ++ .../WriterReaderProperties.scala | 102 ++++ .../RequireOrderedSerializationTest.scala | 94 ++++ .../platform/HadoopPlatformJobTest.scala | 1 + .../platform/HadoopSharedPlatformTest.scala | 49 ++ .../scalding/platform/PlatformTest.scala | 50 +- 34 files changed, 2779 insertions(+), 97 deletions(-) create mode 100644 scalding-core/src/main/scala/com/twitter/scalding/serialization/Boxed.scala create mode 100644 scalding-core/src/main/scala/com/twitter/scalding/serialization/CascadingBinaryComparator.scala create mode 100644 scalding-core/src/main/scala/com/twitter/scalding/serialization/Hasher.scala create mode 100644 scalding-core/src/main/scala/com/twitter/scalding/serialization/JavaStreamEnrichments.scala create mode 100644 scalding-core/src/main/scala/com/twitter/scalding/serialization/Laws.scala create mode 100644 scalding-core/src/main/scala/com/twitter/scalding/serialization/MurmerHashUtils.scala create mode 100644 scalding-core/src/main/scala/com/twitter/scalding/serialization/OrderedSerialization.scala create mode 100644 scalding-core/src/main/scala/com/twitter/scalding/serialization/PositionInputStream.scala create mode 100644 scalding-core/src/main/scala/com/twitter/scalding/serialization/Reader.scala create mode 100644 scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization.scala create mode 100644 scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization2.scala create mode 100644 scalding-core/src/main/scala/com/twitter/scalding/serialization/StringOrderedSerialization.scala create mode 100644 scalding-core/src/main/scala/com/twitter/scalding/serialization/UnsignedComparisons.scala create mode 100644 scalding-core/src/main/scala/com/twitter/scalding/serialization/WrappedSerialization.scala create mode 100644 scalding-core/src/main/scala/com/twitter/scalding/serialization/Writer.scala create mode 100644 scalding-core/src/test/scala/com/twitter/scalding/serialization/JavaStreamEnrichmentsProperties.scala create mode 100644 scalding-core/src/test/scala/com/twitter/scalding/serialization/SerializationProperties.scala create mode 100644 scalding-core/src/test/scala/com/twitter/scalding/serialization/UnsignedComparisonLaws.scala create mode 100644 scalding-core/src/test/scala/com/twitter/scalding/serialization/WriterReaderProperties.scala create mode 100644 scalding-core/src/test/scala/com/twitter/scalding/typed/RequireOrderedSerializationTest.scala create mode 100644 scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/HadoopSharedPlatformTest.scala diff --git a/.travis.yml b/.travis.yml index 365b2e9cd4..4e239a7252 100644 --- a/.travis.yml +++ b/.travis.yml @@ -11,7 +11,7 @@ matrix: env: BUILD="base" TEST_TARGET="scalding-args scalding-date" script: "scripts/run_test.sh" - - scala: 2.11.4 + - scala: 2.11.5 env: BUILD="base" TEST_TARGET="scalding-args scalding-date" script: "scripts/run_test.sh" @@ -19,7 +19,7 @@ matrix: env: BUILD="base" TEST_TARGET="scalding-avro scalding-hraven scalding-commons" script: "scripts/run_test.sh" - - scala: 2.11.4 + - scala: 2.11.5 env: BUILD="base" TEST_TARGET="scalding-avro scalding-hraven scalding-commons" script: "scripts/run_test.sh" @@ -27,7 +27,7 @@ matrix: env: BUILD="base" TEST_TARGET="scalding-core" script: "scripts/run_test.sh" - - scala: 2.11.4 + - scala: 2.11.5 env: BUILD="base" TEST_TARGET="scalding-core" script: "scripts/run_test.sh" @@ -35,7 +35,7 @@ matrix: env: BUILD="base" TEST_TARGET="scalding-hadoop-test" script: "scripts/run_test.sh" - - scala: 2.11.4 + - scala: 2.11.5 env: BUILD="base" TEST_TARGET="scalding-hadoop-test" script: "scripts/run_test.sh" @@ -43,15 +43,31 @@ matrix: env: BUILD="base" TEST_TARGET="scalding-jdbc scalding-json" script: "scripts/run_test.sh" - - scala: 2.11.4 + - scala: 2.11.5 env: BUILD="base" TEST_TARGET="scalding-jdbc scalding-json" script: "scripts/run_test.sh" + - scala: 2.10.4 + env: BUILD="base" TEST_TARGET="scalding-macros" + script: "scripts/run_test.sh" + + - scala: 2.11.5 + env: BUILD="base" TEST_TARGET="scalding-macros" + script: "scripts/run_test.sh" + + - scala: 2.10.4 + env: BUILD="base" TEST_TARGET="scalding-commons-macros" + script: "scripts/run_test.sh" + + - scala: 2.11.5 + env: BUILD="base" TEST_TARGET="scalding-commons-macros" + script: "scripts/run_test.sh" + - scala: 2.10.4 env: BUILD="base" TEST_TARGET="scalding-parquet scalding-parquet-scrooge" script: "scripts/run_test.sh" - - scala: 2.11.4 + - scala: 2.11.5 env: BUILD="base" TEST_TARGET="scalding-parquet scalding-parquet-scrooge" script: "scripts/run_test.sh" @@ -65,7 +81,7 @@ matrix: - "scripts/build_assembly_no_test.sh scalding" - "scripts/test_tutorials.sh" - - scala: 2.11.4 + - scala: 2.11.5 env: BUILD="test tutorials" script: - "scripts/build_assembly_no_test.sh scalding" @@ -77,7 +93,7 @@ matrix: - "scripts/build_assembly_no_test.sh scalding" - "scripts/test_matrix_tutorials.sh" - - scala: 2.11.4 + - scala: 2.11.5 env: BUILD="test matrix tutorials" script: - "scripts/build_assembly_no_test.sh scalding" @@ -91,7 +107,7 @@ matrix: - "scripts/build_assembly_no_test.sh scalding-core" - "scripts/test_typed_tutorials.sh" - - scala: 2.11.4 + - scala: 2.11.5 env: BUILD="test typed tutorials" script: - "scripts/build_assembly_no_test.sh scalding-core" diff --git a/project/Build.scala b/project/Build.scala index 9e9b7fffa4..cd5c003c82 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -21,7 +21,7 @@ object ScaldingBuild extends Build { def isScala210x(scalaVersion: String) = scalaBinaryVersion(scalaVersion) == "2.10" val scalaTestVersion = "2.2.2" - val scalaCheckVersion = "1.11.5" + val scalaCheckVersion = "1.12.1" val hadoopVersion = "1.2.1" val algebirdVersion = "0.9.0" val bijectionVersion = "0.7.2" @@ -39,6 +39,7 @@ object ScaldingBuild extends Build { val cascadingAvroVersion = "2.1.2" val avroVersion = "1.7.4" val json4SVersion = "3.2.11" + val scroogeVersion = "3.17.0" val printDependencyClasspath = taskKey[Unit]("Prints location of the dependencies") @@ -275,10 +276,12 @@ object ScaldingBuild extends Build { "com.hadoop.gplcompression" % "hadoop-lzo" % hadoopLzoVersion, // TODO: split this out into scalding-thrift "org.apache.thrift" % "libthrift" % thriftVersion, + // TODO: split this out into a scalding-scrooge + "com.twitter" %% "scrooge-serializer" % scroogeVersion % "provided", "org.slf4j" % "slf4j-api" % slf4jVersion, "org.slf4j" % "slf4j-log4j12" % slf4jVersion % "provided" ) - ).dependsOn(scaldingArgs, scaldingDate, scaldingCore) + ).dependsOn(scaldingArgs, scaldingDate, scaldingCore, scaldingHadoopTest % "test") lazy val scaldingAvro = module("avro").settings( libraryDependencies ++= Seq( diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala index 02ac5063b4..dd6777269b 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala @@ -110,6 +110,18 @@ trait Config { def setMapSideAggregationThreshold(count: Int): Config = this + (AggregateBy.AGGREGATE_BY_THRESHOLD -> count.toString) + /** + * Set this configuration option to require all grouping/cogrouping + * to use OrderedSerialization + */ + def setRequireOrderedSerialization(b: Boolean): Config = + this + (ScaldingRequireOrderedSerialization -> (b.toString)) + + def getRequireOrderedSerialization: Boolean = + get(ScaldingRequireOrderedSerialization) + .map(java.lang.Boolean.parseBoolean) + .getOrElse(false) + def getCascadingSerializationTokens: Map[Int, String] = get(Config.CascadingSerializationTokens) .map(CascadingTokenUpdater.parseTokens) @@ -153,7 +165,8 @@ trait Config { // Hadoop and Cascading should come first val first: Seq[Class[_ <: HSerialization[_]]] = Seq(classOf[org.apache.hadoop.io.serializer.WritableSerialization], - classOf[cascading.tuple.hadoop.TupleSerialization]) + classOf[cascading.tuple.hadoop.TupleSerialization], + classOf[serialization.WrappedSerialization[_]]) // this must come last val last: Seq[Class[_ <: HSerialization[_]]] = Seq(classOf[com.twitter.chill.hadoop.KryoSerialization]) val required = (first ++ last).toSet[AnyRef] // Class is invariant, but we use it as a function @@ -301,6 +314,7 @@ object Config { val ScaldingJobArgs: String = "scalding.job.args" val ScaldingVersion: String = "scalding.version" val HRavenHistoryUserName: String = "hraven.history.user.name" + val ScaldingRequireOrderedSerialization: String = "scalding.require.orderedserialization" /** * Parameter that actually controls the number of reduce tasks. diff --git a/scalding-core/src/main/scala/com/twitter/scalding/ExecutionContext.scala b/scalding-core/src/main/scala/com/twitter/scalding/ExecutionContext.scala index bf6619d6ce..5080256923 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/ExecutionContext.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/ExecutionContext.scala @@ -17,6 +17,7 @@ package com.twitter.scalding import cascading.flow.{ FlowDef, Flow } import com.twitter.scalding.reducer_estimation.ReducerEstimatorStepStrategy +import com.twitter.scalding.serialization.CascadingBinaryComparator import scala.concurrent.Future import scala.util.{ Failure, Success, Try } @@ -44,6 +45,11 @@ trait ExecutionContext { try { // identify the flowDef val withId = config.addUniqueId(UniqueID.getIDFor(flowDef)) + if (config.getRequireOrderedSerialization) { + // This will throw, but be caught by the outer try if + // we have groupby/cogroupby not using OrderedSerializations + CascadingBinaryComparator.checkForOrderedSerialization(flowDef).get + } val flow = mode.newFlowConnector(withId).connect(flowDef) // if any reducer estimators have been set, register the step strategy diff --git a/scalding-core/src/main/scala/com/twitter/scalding/FieldConversions.scala b/scalding-core/src/main/scala/com/twitter/scalding/FieldConversions.scala index 078cb7e803..343ac0c9c2 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/FieldConversions.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/FieldConversions.scala @@ -268,4 +268,10 @@ object Field { def apply[T](index: Int)(implicit ord: Ordering[T], mf: Manifest[T]) = IntField[T](index)(ord, Some(mf)) def apply[T](name: String)(implicit ord: Ordering[T], mf: Manifest[T]) = StringField[T](name)(ord, Some(mf)) def apply[T](symbol: Symbol)(implicit ord: Ordering[T], mf: Manifest[T]) = StringField[T](symbol.name)(ord, Some(mf)) + + def singleOrdered[T](name: String)(implicit ord: Ordering[T]): Fields = { + val f = new Fields(name) + f.setComparator(name, ord) + f + } } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Operations.scala b/scalding-core/src/main/scala/com/twitter/scalding/Operations.scala index 69917de209..8225324a91 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Operations.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Operations.scala @@ -483,6 +483,7 @@ package com.twitter.scalding { /** In the typed API every reduce operation is handled by this Buffer */ class TypedBufferOp[K, V, U]( + conv: TupleConverter[K], @transient reduceFn: (K, Iterator[V]) => Iterator[U], valueField: Fields) extends BaseOperation[Any](valueField) with Buffer[Any] with ScaldingPrepare[Any] { @@ -490,7 +491,7 @@ package com.twitter.scalding { def operate(flowProcess: FlowProcess[_], call: BufferCall[Any]) { val oc = call.getOutputCollector - val key = call.getGroup.getObject(0).asInstanceOf[K] + val key = conv(call.getGroup) val values = call.getArgumentsIterator .asScala .map(_.getObject(0).asInstanceOf[V]) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Boxed.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Boxed.scala new file mode 100644 index 0000000000..ff7d15e1d3 --- /dev/null +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Boxed.scala @@ -0,0 +1,470 @@ +/* +Copyright 2015 Twitter, Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + +http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ +package com.twitter.scalding.serialization + +import java.util.concurrent.atomic.AtomicReference +import java.io.{ InputStream, OutputStream } + +import com.esotericsoftware.kryo.{ Serializer => KSerializer, DefaultSerializer, Kryo } +import com.esotericsoftware.kryo.io.{ Input, Output } + +/** + * This interface is a way of wrapping a value in a marker class + * whose class identity is used to control which serialization we + * use. This is an internal implementation detail about how we + * interact with cascading and hadoop. Users should never care. + */ +trait Boxed[+K] { + def get: K +} + +class BoxedDefaultSerialization extends KSerializer[Boxed[_]] { + def write(kryo: Kryo, output: Output, t: Boxed[_]) { + sys.error(s"Kryo should never be used to serialize a boxed instance: $t") + } + def read(kryo: Kryo, input: Input, t: Class[Boxed[_]]): Boxed[_] = { + sys.error("Kryo should never be used to serialize a boxed instance, class: $t") + } +} + +// TODO: Make more of these with a script + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed0[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed1[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed2[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed3[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed4[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed5[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed6[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed7[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed8[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed9[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed10[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed11[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed12[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed13[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed14[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed15[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed16[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed17[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed18[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed19[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed20[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed21[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed22[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed23[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed24[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed25[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed26[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed27[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed28[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed29[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed30[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed31[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed32[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed33[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed34[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed35[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed36[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed37[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed38[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed39[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed40[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed41[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed42[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed43[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed44[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed45[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed46[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed47[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed48[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed49[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed50[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed51[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed52[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed53[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed54[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed55[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed56[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed57[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed58[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed59[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed60[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed61[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed62[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed63[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed64[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed65[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed66[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed67[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed68[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed69[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed70[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed71[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed72[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed73[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed74[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed75[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed76[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed77[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed78[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed79[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed80[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed81[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed82[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed83[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed84[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed85[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed86[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed87[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed88[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed89[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed90[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed91[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed92[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed93[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed94[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed95[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed96[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed97[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed98[K](override val get: K) extends Boxed[K] + +@DefaultSerializer(classOf[BoxedDefaultSerialization]) +class Boxed99[K](override val get: K) extends Boxed[K] + +// TODO this could be any general bijection +case class BoxedOrderedSerialization[K](box: K => Boxed[K], + ord: OrderedSerialization[K]) extends OrderedSerialization[Boxed[K]] { + + override def compare(a: Boxed[K], b: Boxed[K]) = ord.compare(a.get, b.get) + override def hash(k: Boxed[K]) = ord.hash(k.get) + override def compareBinary(a: InputStream, b: InputStream) = ord.compareBinary(a, b) + override def read(from: InputStream) = ord.read(from).map(box) + override def write(into: OutputStream, bk: Boxed[K]) = ord.write(into, bk.get) +} + +object Boxed { + private[this] val allBoxes = List( + ({ t: Any => new Boxed0(t) }, classOf[Boxed0[Any]]), + ({ t: Any => new Boxed1(t) }, classOf[Boxed1[Any]]), + ({ t: Any => new Boxed2(t) }, classOf[Boxed2[Any]]), + ({ t: Any => new Boxed3(t) }, classOf[Boxed3[Any]]), + ({ t: Any => new Boxed4(t) }, classOf[Boxed4[Any]]), + ({ t: Any => new Boxed5(t) }, classOf[Boxed5[Any]]), + ({ t: Any => new Boxed6(t) }, classOf[Boxed6[Any]]), + ({ t: Any => new Boxed7(t) }, classOf[Boxed7[Any]]), + ({ t: Any => new Boxed8(t) }, classOf[Boxed8[Any]]), + ({ t: Any => new Boxed9(t) }, classOf[Boxed9[Any]]), + ({ t: Any => new Boxed10(t) }, classOf[Boxed10[Any]]), + ({ t: Any => new Boxed11(t) }, classOf[Boxed11[Any]]), + ({ t: Any => new Boxed12(t) }, classOf[Boxed12[Any]]), + ({ t: Any => new Boxed13(t) }, classOf[Boxed13[Any]]), + ({ t: Any => new Boxed14(t) }, classOf[Boxed14[Any]]), + ({ t: Any => new Boxed15(t) }, classOf[Boxed15[Any]]), + ({ t: Any => new Boxed16(t) }, classOf[Boxed16[Any]]), + ({ t: Any => new Boxed17(t) }, classOf[Boxed17[Any]]), + ({ t: Any => new Boxed18(t) }, classOf[Boxed18[Any]]), + ({ t: Any => new Boxed19(t) }, classOf[Boxed19[Any]]), + ({ t: Any => new Boxed20(t) }, classOf[Boxed20[Any]]), + ({ t: Any => new Boxed21(t) }, classOf[Boxed21[Any]]), + ({ t: Any => new Boxed22(t) }, classOf[Boxed22[Any]]), + ({ t: Any => new Boxed23(t) }, classOf[Boxed23[Any]]), + ({ t: Any => new Boxed24(t) }, classOf[Boxed24[Any]]), + ({ t: Any => new Boxed25(t) }, classOf[Boxed25[Any]]), + ({ t: Any => new Boxed26(t) }, classOf[Boxed26[Any]]), + ({ t: Any => new Boxed27(t) }, classOf[Boxed27[Any]]), + ({ t: Any => new Boxed28(t) }, classOf[Boxed28[Any]]), + ({ t: Any => new Boxed29(t) }, classOf[Boxed29[Any]]), + ({ t: Any => new Boxed30(t) }, classOf[Boxed30[Any]]), + ({ t: Any => new Boxed31(t) }, classOf[Boxed31[Any]]), + ({ t: Any => new Boxed32(t) }, classOf[Boxed32[Any]]), + ({ t: Any => new Boxed33(t) }, classOf[Boxed33[Any]]), + ({ t: Any => new Boxed34(t) }, classOf[Boxed34[Any]]), + ({ t: Any => new Boxed35(t) }, classOf[Boxed35[Any]]), + ({ t: Any => new Boxed36(t) }, classOf[Boxed36[Any]]), + ({ t: Any => new Boxed37(t) }, classOf[Boxed37[Any]]), + ({ t: Any => new Boxed38(t) }, classOf[Boxed38[Any]]), + ({ t: Any => new Boxed39(t) }, classOf[Boxed39[Any]]), + ({ t: Any => new Boxed40(t) }, classOf[Boxed40[Any]]), + ({ t: Any => new Boxed41(t) }, classOf[Boxed41[Any]]), + ({ t: Any => new Boxed42(t) }, classOf[Boxed42[Any]]), + ({ t: Any => new Boxed43(t) }, classOf[Boxed43[Any]]), + ({ t: Any => new Boxed44(t) }, classOf[Boxed44[Any]]), + ({ t: Any => new Boxed45(t) }, classOf[Boxed45[Any]]), + ({ t: Any => new Boxed46(t) }, classOf[Boxed46[Any]]), + ({ t: Any => new Boxed47(t) }, classOf[Boxed47[Any]]), + ({ t: Any => new Boxed48(t) }, classOf[Boxed48[Any]]), + ({ t: Any => new Boxed49(t) }, classOf[Boxed49[Any]]), + ({ t: Any => new Boxed50(t) }, classOf[Boxed50[Any]]), + ({ t: Any => new Boxed51(t) }, classOf[Boxed51[Any]]), + ({ t: Any => new Boxed52(t) }, classOf[Boxed52[Any]]), + ({ t: Any => new Boxed53(t) }, classOf[Boxed53[Any]]), + ({ t: Any => new Boxed54(t) }, classOf[Boxed54[Any]]), + ({ t: Any => new Boxed55(t) }, classOf[Boxed55[Any]]), + ({ t: Any => new Boxed56(t) }, classOf[Boxed56[Any]]), + ({ t: Any => new Boxed57(t) }, classOf[Boxed57[Any]]), + ({ t: Any => new Boxed58(t) }, classOf[Boxed58[Any]]), + ({ t: Any => new Boxed59(t) }, classOf[Boxed59[Any]]), + ({ t: Any => new Boxed60(t) }, classOf[Boxed60[Any]]), + ({ t: Any => new Boxed61(t) }, classOf[Boxed61[Any]]), + ({ t: Any => new Boxed62(t) }, classOf[Boxed62[Any]]), + ({ t: Any => new Boxed63(t) }, classOf[Boxed63[Any]]), + ({ t: Any => new Boxed64(t) }, classOf[Boxed64[Any]]), + ({ t: Any => new Boxed65(t) }, classOf[Boxed65[Any]]), + ({ t: Any => new Boxed66(t) }, classOf[Boxed66[Any]]), + ({ t: Any => new Boxed67(t) }, classOf[Boxed67[Any]]), + ({ t: Any => new Boxed68(t) }, classOf[Boxed68[Any]]), + ({ t: Any => new Boxed69(t) }, classOf[Boxed69[Any]]), + ({ t: Any => new Boxed70(t) }, classOf[Boxed70[Any]]), + ({ t: Any => new Boxed71(t) }, classOf[Boxed71[Any]]), + ({ t: Any => new Boxed72(t) }, classOf[Boxed72[Any]]), + ({ t: Any => new Boxed73(t) }, classOf[Boxed73[Any]]), + ({ t: Any => new Boxed74(t) }, classOf[Boxed74[Any]]), + ({ t: Any => new Boxed75(t) }, classOf[Boxed75[Any]]), + ({ t: Any => new Boxed76(t) }, classOf[Boxed76[Any]]), + ({ t: Any => new Boxed77(t) }, classOf[Boxed77[Any]]), + ({ t: Any => new Boxed78(t) }, classOf[Boxed78[Any]]), + ({ t: Any => new Boxed79(t) }, classOf[Boxed79[Any]]), + ({ t: Any => new Boxed80(t) }, classOf[Boxed80[Any]]), + ({ t: Any => new Boxed81(t) }, classOf[Boxed81[Any]]), + ({ t: Any => new Boxed82(t) }, classOf[Boxed82[Any]]), + ({ t: Any => new Boxed83(t) }, classOf[Boxed83[Any]]), + ({ t: Any => new Boxed84(t) }, classOf[Boxed84[Any]]), + ({ t: Any => new Boxed85(t) }, classOf[Boxed85[Any]]), + ({ t: Any => new Boxed86(t) }, classOf[Boxed86[Any]]), + ({ t: Any => new Boxed87(t) }, classOf[Boxed87[Any]]), + ({ t: Any => new Boxed88(t) }, classOf[Boxed88[Any]]), + ({ t: Any => new Boxed89(t) }, classOf[Boxed89[Any]]), + ({ t: Any => new Boxed90(t) }, classOf[Boxed90[Any]]), + ({ t: Any => new Boxed91(t) }, classOf[Boxed91[Any]]), + ({ t: Any => new Boxed92(t) }, classOf[Boxed92[Any]]), + ({ t: Any => new Boxed93(t) }, classOf[Boxed93[Any]]), + ({ t: Any => new Boxed94(t) }, classOf[Boxed94[Any]]), + ({ t: Any => new Boxed95(t) }, classOf[Boxed95[Any]]), + ({ t: Any => new Boxed96(t) }, classOf[Boxed96[Any]]), + ({ t: Any => new Boxed97(t) }, classOf[Boxed97[Any]]), + ({ t: Any => new Boxed98(t) }, classOf[Boxed98[Any]]), + ({ t: Any => new Boxed99(t) }, classOf[Boxed99[Any]])) + + private[this] val boxes: AtomicReference[List[(Any => Boxed[Any], Class[_ <: Boxed[Any]])]] = + new AtomicReference(allBoxes) + + def allClasses: Seq[Class[_ <: Boxed[_]]] = allBoxes.map(_._2) + + def next[K]: (K => Boxed[K], Class[Boxed[K]]) = boxes.get match { + case list @ (h :: tail) if boxes.compareAndSet(list, tail) => + h.asInstanceOf[(K => Boxed[K], Class[Boxed[K]])] + case (h :: tail) => next[K] // Try again + case Nil => sys.error("Exhausted the boxed classes") + } +} diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/CascadingBinaryComparator.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/CascadingBinaryComparator.scala new file mode 100644 index 0000000000..885b1d76b9 --- /dev/null +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/CascadingBinaryComparator.scala @@ -0,0 +1,76 @@ +/* +Copyright 2015 Twitter, Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + +http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package com.twitter.scalding.serialization + +import java.io.InputStream +import java.util.Comparator +import cascading.flow.FlowDef +import cascading.tuple.{ Fields, Hasher => CHasher, StreamComparator } + +import scala.util.{ Failure, Success, Try } +/** + * This is the type that should be fed to cascading to enable binary comparators + */ +class CascadingBinaryComparator[T](ob: OrderedSerialization[T]) extends Comparator[T] + with StreamComparator[InputStream] + with CHasher[T] + with Serializable { + + override def compare(a: T, b: T) = ob.compare(a, b) + override def hashCode(t: T): Int = ob.hash(t) + override def compare(a: InputStream, b: InputStream) = + ob.compareBinary(a, b).unsafeToInt +} + +object CascadingBinaryComparator { + /** + * This method will walk the flowDef and make sure all the + * groupBy/cogroups are using a CascadingBinaryComparator + */ + def checkForOrderedSerialization(fd: FlowDef): Try[Unit] = { + // Get the asScala enrichments locally? + import collection.JavaConverters._ + import cascading.pipe._ + import com.twitter.scalding.RichPipe + + // all successes or empty returns success + def reduce(it: TraversableOnce[Try[Unit]]): Try[Unit] = + it.find(_.isFailure).getOrElse(Success(())) + + def check(s: Splice): Try[Unit] = { + val m = s.getKeySelectors.asScala + if (m.isEmpty) Failure(new Exception(s"Splice must have KeySelectors: $s")) + else { + reduce(m.map { + case (pipename, fields) => + Try { + if (fields.getComparators()(0).isInstanceOf[com.twitter.scalding.serialization.CascadingBinaryComparator[_]]) + () + else sys.error(s"pipe: $s, fields: $fields, comparators: ${fields.getComparators.toList}") + } + }) + } + } + + val allPipes: Set[Pipe] = fd.getTails.asScala.map(p => RichPipe(p).upstreamPipes).flatten.toSet + reduce(allPipes.iterator.map { + case gb: GroupBy => check(gb) + case cg: CoGroup => check(cg) + case _ => Success(()) + }) + } +} diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Hasher.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Hasher.scala new file mode 100644 index 0000000000..eab23cee78 --- /dev/null +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Hasher.scala @@ -0,0 +1,84 @@ +/* +Copyright 2015 Twitter, Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + +http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ +package com.twitter.scalding.serialization + +// Be careful using this, the product/array or similar will attempt to call system hash codes. +import scala.util.hashing.MurmurHash3 +/** + * This is a specialized typeclass to make it easier to implement Serializations. + * The specialization *should* mean that there is no boxing and if the JIT + * does its work, Hasher should compose well (via collections, Tuple2, Option, Either) + */ +trait Hasher[@specialized(Boolean, Byte, Short, Int, Long, Float, Double) -T] { + @inline + def hash(i: T): Int +} + +object Hasher { + import MurmerHashUtils._ + final val seed = 0xf7ca7fd2 + + @inline + def hash[@specialized(Boolean, Byte, Short, Int, Long, Float, Double) T]( + i: T)(implicit h: Hasher[T]): Int = h.hash(i) + + /* + * Instances below + */ + implicit val unit: Hasher[Unit] = new Hasher[Unit] { + @inline + def hash(i: Unit) = 0 + } + implicit val boolean: Hasher[Boolean] = new Hasher[Boolean] { + // Here we use the two largest mersenne primes + @inline + def hash(i: Boolean) = if (i) Int.MaxValue else ((1 << 19) - 1) + } + implicit val byte: Hasher[Byte] = new Hasher[Byte] { + @inline + def hash(i: Byte) = hashInt(i.toInt) + } + implicit val short: Hasher[Short] = new Hasher[Short] { + @inline + def hash(i: Short) = hashInt(i.toInt) + } + + implicit val int: Hasher[Int] = new Hasher[Int] { + @inline + def hash(i: Int) = hashInt(i) + } + + // java way to refer to int, alias in naming + val integer = int + + implicit val long: Hasher[Long] = new Hasher[Long] { + @inline + def hash(i: Long) = hashLong(i) + } + + implicit val float: Hasher[Float] = new Hasher[Float] { + @inline + def hash(i: Float) = hashInt(java.lang.Float.valueOf(i).intValue) + } + implicit val double: Hasher[Double] = new Hasher[Double] { + @inline + def hash(i: Double) = hashLong(i.longValue) + } + implicit val string: Hasher[String] = new Hasher[String] { + @inline + def hash(i: String) = MurmurHash3.stringHash(i) + } +} diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/JavaStreamEnrichments.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/JavaStreamEnrichments.scala new file mode 100644 index 0000000000..c5ef9256bf --- /dev/null +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/JavaStreamEnrichments.scala @@ -0,0 +1,228 @@ +/* +Copyright 2015 Twitter, Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + +http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ +package com.twitter.scalding.serialization + +import java.io.{ ByteArrayInputStream, ByteArrayOutputStream, InputStream, OutputStream, EOFException } + +object JavaStreamEnrichments { + def eof: Nothing = throw new EOFException() + + implicit class RichByteArrayOutputStream(val baos: ByteArrayOutputStream) extends AnyVal { + def toInputStream: ByteArrayInputStream = new ByteArrayInputStream(baos.toByteArray) + } + + def sizeBytes(i: Int): Int = + if (i < ((1 << 8) - 1)) 1 + else { + if (i < ((1 << 16) - 1)) { + 3 + } else { + 7 + } + } + + /** + * This has a lot of methods from DataInputStream without + * having to allocate to get them + * This code is similar to those algorithms + */ + implicit class RichInputStream(val s: InputStream) extends AnyVal { + /** + * If s supports marking, we mark it. Otherwise we read the needed + * bytes out into a ByteArrayStream and return that. + * This is intended for the case where you need possibly + * read size bytes but may stop early, then skip this exact + * number of bytes. + * Intended use is: + * {code} + * val size = 100 + * val marked = s.markOrBuffer(size) + * val y = fn(marked) + * marked.reset + * marked.skipFully(size) + * {/code} + */ + def markOrBuffer(size: Int): InputStream = { + val ms = if (s.markSupported) s else { + val buf = new Array[Byte](size) + s.readFully(buf) + new ByteArrayInputStream(buf) + } + // Make sure we can reset after we read this many bytes + ms.mark(size) + ms + } + + def readBoolean: Boolean = (readUnsignedByte != 0) + + /** + * Like read, but throws eof on error + */ + def readByte: Byte = readUnsignedByte.toByte + + def readUnsignedByte: Int = { + // Note that Java, when you read a byte, returns a Int holding an unsigned byte. + // if the value is < 0, you hit EOF. + val c1 = s.read + if (c1 < 0) eof else c1 + } + def readUnsignedShort: Int = { + val c1 = s.read + val c2 = s.read + if ((c1 | c2) < 0) eof else ((c1 << 8) | c2) + } + + final def readFully(bytes: Array[Byte]): Unit = readFully(bytes, 0, bytes.length) + + final def readFully(bytes: Array[Byte], offset: Int, len: Int): Unit = { + if (len < 0) throw new IndexOutOfBoundsException() + + @annotation.tailrec + def go(o: Int, l: Int): Unit = + if (l == 0) () + else { + val count = s.read(bytes, o, l) + if (count < 0) eof + else go(o + count, l - count) + } + go(offset, len) + } + + def readDouble: Double = java.lang.Double.longBitsToDouble(readLong) + def readFloat: Float = java.lang.Float.intBitsToFloat(readInt) + + def readInt: Int = { + val c1 = s.read + val c2 = s.read + val c3 = s.read + val c4 = s.read + // This is the algorithm from DataInputStream + if ((c1 | c2 | c3 | c4) < 0) eof else ((c1 << 24) | (c2 << 16) | (c3 << 8) | c4) + } + def readLong: Long = { + // This is the algorithm from DataInputStream + val buf = new Array[Byte](8) + readFully(buf) + (buf(0).toLong << 56) + + ((buf(1) & 255).toLong << 48) + + ((buf(2) & 255).toLong << 40) + + ((buf(3) & 255).toLong << 32) + + ((buf(4) & 255).toLong << 24) + + ((buf(5) & 255) << 16) + + ((buf(6) & 255) << 8) + + (buf(7) & 255) + } + + def readShort: Short = { + val c1 = s.read + val c2 = s.read + // This is the algorithm from DataInputStream + if ((c1 | c2) < 0) eof else ((c1 << 8) | c2).toShort + } + + /** + * This reads a varInt encoding that only encodes non-negative + * numbers. It uses: + * 1 byte for values 0 - 255, + * 3 bytes for 256 - 65535, + * 7 bytes for 65536 - Int.MaxValue + */ + final def readSize: Int = { + val c1 = readUnsignedByte + if (c1 < ((1 << 8) - 1)) c1 + else { + val c2 = readUnsignedShort + if (c2 < ((1 << 16) - 1)) c2 + else readInt + } + } + + final def skipFully(count: Long): Unit = { + @annotation.tailrec + def go(c: Long): Unit = { + val skipped = s.skip(c) + if (skipped == c) () + else if (skipped == 0L) throw new Exception(s"count, c, skipped = ${(count, c, skipped)}") + else go(c - skipped) + } + if (count != 0L) go(count) else () + } + } + + implicit class RichOutputStream(val s: OutputStream) extends AnyVal { + def writeBoolean(b: Boolean): Unit = if (b) s.write(1: Byte) else s.write(0: Byte) + + def writeBytes(b: Array[Byte], off: Int, len: Int): Unit = { + s.write(b, off, len) + } + + def writeByte(b: Byte): Unit = s.write(b) + + def writeBytes(b: Array[Byte]): Unit = writeBytes(b, 0, b.length) + + /** + * This reads a varInt encoding that only encodes non-negative + * numbers. It uses: + * 1 byte for values 0 - 255, + * 3 bytes for 256 - 65535, + * 7 bytes for 65536 - Int.MaxValue + */ + def writeSize(i: Int): Unit = { + require(i >= 0, s"sizes must be non-negative: ${i}") + if (i < ((1 << 8) - 1)) s.write(i.toByte) + else { + s.write(-1: Byte) + if (i < ((1 << 16) - 1)) { + val b1 = (i >> 8).toByte + val b2 = (i & 0xFF).toByte + s.write(b1) + s.write(b2) + } else { + s.write(-1: Byte) + s.write(-1: Byte) + writeInt(i) + } + } + } + + def writeDouble(d: Double): Unit = writeLong(java.lang.Double.doubleToLongBits(d)) + + def writeFloat(f: Float): Unit = writeInt(java.lang.Float.floatToIntBits(f)) + + def writeLong(l: Long): Unit = { + s.write((l >>> 56).toByte) + s.write(((l >>> 48) & 0xFF).toByte) + s.write(((l >>> 40) & 0xFF).toByte) + s.write((l >>> 32).toByte) + s.write((l >>> 24).toByte) + s.write(((l >>> 16) & 0xFF).toByte) + s.write(((l >>> 8) & 0xFF).toByte) + s.write((l & 0xFF).toByte) + } + + def writeInt(i: Int): Unit = { + s.write((i >>> 24).toByte) + s.write(((i >>> 16) & 0xFF).toByte) + s.write(((i >>> 8) & 0xFF).toByte) + s.write((i & 0xFF).toByte) + } + + def writeShort(sh: Short): Unit = { + s.write(((sh >>> 8) & 0xFF).toByte) + s.write((sh & 0xFF).toByte) + } + } +} diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala index 5dbe4241ab..fd6cce090b 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala @@ -20,8 +20,6 @@ import java.io.OutputStream import java.io.Serializable 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.io.{ Input, Output } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Laws.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Laws.scala new file mode 100644 index 0000000000..6ecdcb996e --- /dev/null +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Laws.scala @@ -0,0 +1,31 @@ +/* +Copyright 2015 Twitter, Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + +http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ +package com.twitter.scalding.serialization + +sealed trait Law[T] { + def name: String + def satisfied(a: T, b: T, c: T): Boolean +} + +case class Law1[T](override val name: String, check: T => Boolean) extends Law[T] { + def satisfied(a: T, b: T, c: T) = check(a) +} +case class Law2[T](override val name: String, check: (T, T) => Boolean) extends Law[T] { + def satisfied(a: T, b: T, c: T) = check(a, b) +} +case class Law3[T](override val name: String, check: (T, T, T) => Boolean) extends Law[T] { + def satisfied(a: T, b: T, c: T) = check(a, b, c) +} diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/MurmerHashUtils.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/MurmerHashUtils.scala new file mode 100644 index 0000000000..736b4421a6 --- /dev/null +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/MurmerHashUtils.scala @@ -0,0 +1,114 @@ +/* +Copyright 2015 Twitter, Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + +http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ +package com.twitter.scalding.serialization + +import java.io.InputStream +import scala.reflect.ClassTag +import scala.collection.generic.CanBuildFrom +import java.lang.Integer.{ rotateLeft => rotl } + +// Be careful using this, the product/array or similar will attempt to call system hash codes. +import scala.util.hashing.MurmurHash3 + +// Taking a few functions from: +// https://guava-libraries.googlecode.com/git/guava/src/com/google/common/hash/Murmur3_32HashFunction.java +object MurmerHashUtils { + final val seed = 0xf7ca7fd2 + private final val C1: Int = 0xcc9e2d51 + private final val C2: Int = 0x1b873593 + + final def hashInt(input: Int): Int = { + val k1 = mixK1(input) + val h1 = mixH1(seed, k1) + + fmix(h1, 4) // length of int is 4 bytes + } + + final def hashLong(input: Long): Int = { + val low = input.toInt + val high = (input >>> 32).toInt + + var k1 = mixK1(low) + var h1 = mixH1(seed, k1) + + k1 = mixK1(high) + h1 = mixH1(h1, k1) + + fmix(h1, 8) // 8 bytes + } + + final def hashUnencodedChars(input: CharSequence): Int = { + var h1 = seed; + + // step through the CharSequence 2 chars at a time + var i = 0 + while (i < input.length) { + var k1 = input.charAt(i - 1) | (input.charAt(i) << 16) + k1 = mixK1(k1) + h1 = mixH1(h1, k1) + i += 2 + } + + // deal with any remaining characters + if ((input.length() & 1) == 1) { + var k1: Int = input.charAt(input.length() - 1) + k1 = mixK1(k1) + h1 ^= k1 + } + + fmix(h1, (Character.SIZE / java.lang.Byte.SIZE) * input.length()) + } + + final def mixK1(k1Input: Int): Int = { + var k1 = k1Input + k1 *= C1 + k1 = Integer.rotateLeft(k1, 15) + k1 *= C2 + k1 + } + + final def mixH1(h1Input: Int, k1Input: Int): Int = { + var h1 = h1Input + var k1 = k1Input + h1 ^= k1 + h1 = Integer.rotateLeft(h1, 13) + h1 = h1 * 5 + 0xe6546b64 + h1 + } + + // Finalization mix - force all bits of a hash block to avalanche + final def fmix(h1Input: Int, length: Int): Int = { + var h1 = h1Input + h1 ^= length + h1 ^= h1 >>> 16 + h1 *= 0x85ebca6b + h1 ^= h1 >>> 13 + h1 *= 0xc2b2ae35 + h1 ^= h1 >>> 16 + h1 + } + + def iteratorHash[T](a: Iterator[T])(hashFn: T => Int): Int = { + var h1 = seed + var i = 0 + while (a.hasNext) { + var k1 = hashFn(a.next) + h1 = mixH1(h1, k1) + i += 1 + } + fmix(h1, i) + } +} diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/OrderedSerialization.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/OrderedSerialization.scala new file mode 100644 index 0000000000..04ff78c368 --- /dev/null +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/OrderedSerialization.scala @@ -0,0 +1,147 @@ +/* +Copyright 2015 Twitter, Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + +http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package com.twitter.scalding.serialization + +import java.io.{ ByteArrayInputStream, InputStream, OutputStream } +import scala.util.{ Failure, Success, Try } +import scala.util.control.NonFatal + +/** + * In large-scale partitioning algorithms, we often use sorting. + * This typeclass represents something we can efficiently serialize + * with an added law: that we can (hopefully fast) compare the raw + * data. + */ +trait OrderedSerialization[T] extends Ordering[T] with Serialization[T] { + /** + * This compares two InputStreams. After this call, the position in + * the InputStreams is mutated to be the end of the record. + */ + def compareBinary(a: InputStream, b: InputStream): OrderedSerialization.Result +} + +object OrderedSerialization { + /** + * Represents the result of a comparison that might fail due + * to an error deserializing + */ + sealed trait Result { + /** + * Throws if the items cannot be compared + */ + def unsafeToInt: Int + def toTry: Try[Int] + } + /** + * Create a Result from an Int. + */ + def resultFrom(i: Int): Result = + if (i > 0) Greater + else if (i < 0) Less + else Equal + + def resultFrom(t: Try[Int]): Result = t match { + case Success(i) => resultFrom(i) + case Failure(e) => CompareFailure(e) + } + + final case class CompareFailure(ex: Throwable) extends Result { + def unsafeToInt = throw ex + def toTry = Failure(ex) + } + case object Greater extends Result { + val unsafeToInt = 1 + val toTry = Success(unsafeToInt) + } + case object Equal extends Result { + val unsafeToInt = 0 + val toTry = Success(unsafeToInt) + } + case object Less extends Result { + val unsafeToInt = -1 + val toTry = Success(unsafeToInt) + } + + def compare[T](a: T, b: T)(implicit ord: OrderedSerialization[T]): Int = + ord.compare(a, b) + + def compareBinary[T](a: InputStream, b: InputStream)(implicit ord: OrderedSerialization[T]): Result = + ord.compareBinary(a, b) + + def writeThenCompare[T](a: T, b: T)(implicit ordb: OrderedSerialization[T]): Result = { + val abytes = Serialization.toBytes(a) + val bbytes = Serialization.toBytes(b) + val ain = new ByteArrayInputStream(abytes) + val bin = new ByteArrayInputStream(bbytes) + ordb.compareBinary(ain, bin) + } + + /** + * This is slow, but always an option. Avoid this if you can, especially for large items + */ + def readThenCompare[T: OrderedSerialization](as: InputStream, bs: InputStream): Result = try resultFrom { + val a = Serialization.read[T](as) + val b = Serialization.read[T](bs) + compare(a.get, b.get) + } catch { + case NonFatal(e) => CompareFailure(e) + } + + /** + * The the serialized comparison matches the unserialized comparison + */ + def compareBinaryMatchesCompare[T](implicit ordb: OrderedSerialization[T]): Law2[T] = + Law2("compare(a, b) == compareBinary(aBin, bBin)", + { (a: T, b: T) => resultFrom(ordb.compare(a, b)) == writeThenCompare(a, b) }) + + /** + * ordering must be transitive. If this is not so, sort-based partitioning + * will be broken + */ + def orderingTransitive[T](implicit ordb: OrderedSerialization[T]): Law3[T] = + Law3("transitivity", + { (a: T, b: T, c: T) => + !(ordb.lteq(a, b) && ordb.lteq(b, c)) || ordb.lteq(a, c) + }) + + def allLaws[T: OrderedSerialization]: Iterable[Law[T]] = + Serialization.allLaws ++ List(compareBinaryMatchesCompare[T], orderingTransitive[T]) +} + +/** + * This may be useful when a type is used deep in a tuple or case class, and in that case + * the earlier comparators will have likely already done the work. Be aware that avoiding + * deserialization on compare usually very helpful. + * + * Note: it is your responsibility that the hash in serialization is consistent + * with the ordering (if equivalent in the ordering, the hash must match). + */ +final case class DeserializingOrderedSerialization[T](serialization: Serialization[T], + ordering: Ordering[T]) extends OrderedSerialization[T] { + + final override def read(i: InputStream) = serialization.read(i) + final override def write(o: OutputStream, t: T) = serialization.write(o, t) + final override def hash(t: T) = serialization.hash(t) + final override def compare(a: T, b: T) = ordering.compare(a, b) + final override def compareBinary(a: InputStream, b: InputStream) = + try OrderedSerialization.resultFrom { + compare(read(a).get, read(b).get) + } + catch { + case NonFatal(e) => OrderedSerialization.CompareFailure(e) + } +} diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/PositionInputStream.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/PositionInputStream.scala new file mode 100644 index 0000000000..e419c722cc --- /dev/null +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/PositionInputStream.scala @@ -0,0 +1,78 @@ +/* +Copyright 2015 Twitter, Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + +http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ +package com.twitter.scalding.serialization + +import java.io.InputStream +import JavaStreamEnrichments._ + +object PositionInputStream { + def apply(in: InputStream): PositionInputStream = in match { + case p: PositionInputStream => p + case nonPos => new PositionInputStream(nonPos) + } +} + +class PositionInputStream(val wraps: InputStream) extends InputStream { + private[this] var pos: Long = 0L + private[this] var markPos: Long = -1L + def position: Long = pos + + override def available = wraps.available + + override def close() { wraps.close() } + + override def mark(limit: Int) { + wraps.mark(limit) + markPos = pos + } + + override val markSupported: Boolean = wraps.markSupported + + override def read: Int = { + val result = wraps.read + // returns -1 on eof, otherwise non-negative number + if (result >= 0) pos += 1 + result + } + override def read(bytes: Array[Byte]): Int = read(bytes, 0, bytes.length) + + override def read(bytes: Array[Byte], off: Int, len: Int): Int = { + val count = wraps.read(bytes, off, len) + if (count > 0) pos += count + count + } + + override def reset() { + wraps.reset() + pos = markPos + } + + override def skip(n: Long): Long = { + require(n >= 0, "Must seek fowards") + val count = skip(n) + if (count > 0) pos += count + count + } + + /** + * This throws an exception if it can't set the position to what you give it. + */ + def seekToPosition(p: Long) { + require(p >= pos, s"Can't seek backwards, at position $pos, trying to goto $p") + wraps.skipFully(p - pos) + pos = p + } +} diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Reader.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Reader.scala new file mode 100644 index 0000000000..92062b4fd0 --- /dev/null +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Reader.scala @@ -0,0 +1,130 @@ +/* +Copyright 2015 Twitter, Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + +http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ +package com.twitter.scalding.serialization + +import java.io.InputStream +import scala.reflect.ClassTag +import scala.collection.generic.CanBuildFrom + +/** + * This is a specialized typeclass to make it easier to implement Serializations. + * The specialization *should* mean that there is no boxing and if the JIT + * does its work, Reader should compose well (via collections, Tuple2, Option, Either) + */ +trait Reader[@specialized(Boolean, Byte, Short, Int, Long, Float, Double) +T] { + def read(is: InputStream): T +} + +object Reader { + import JavaStreamEnrichments._ + + def read[@specialized(Boolean, Byte, Short, Int, Long, Float, Double) T]( + is: InputStream)(implicit r: Reader[T]): T = r.read(is) + /* + * Instances below + */ + implicit val unit: Reader[Unit] = new Reader[Unit] { + def read(is: InputStream) = () + } + implicit val boolean: Reader[Boolean] = new Reader[Boolean] { + def read(is: InputStream) = is.readBoolean + } + implicit val byte: Reader[Byte] = new Reader[Byte] { + def read(is: InputStream) = is.readByte + } + implicit val short: Reader[Short] = new Reader[Short] { + def read(is: InputStream) = is.readShort + } + implicit val int: Reader[Int] = new Reader[Int] { + def read(is: InputStream) = is.readInt + } + implicit val long: Reader[Long] = new Reader[Long] { + def read(is: InputStream) = is.readLong + } + implicit val float: Reader[Float] = new Reader[Float] { + def read(is: InputStream) = is.readFloat + } + implicit val double: Reader[Double] = new Reader[Double] { + def read(is: InputStream) = is.readDouble + } + implicit val string: Reader[String] = new Reader[String] { + def read(is: InputStream) = { + val size = is.readSize + val bytes = new Array[Byte](size) + is.readFully(bytes) + new String(bytes, "UTF-8") + } + } + + implicit def option[T: Reader]: Reader[Option[T]] = new Reader[Option[T]] { + val r = implicitly[Reader[T]] + def read(is: InputStream) = + if (is.readByte == (0: Byte)) None + else Some(r.read(is)) + } + + implicit def either[L: Reader, R: Reader]: Reader[Either[L, R]] = new Reader[Either[L, R]] { + val lRead = implicitly[Reader[L]] + val rRead = implicitly[Reader[R]] + def read(is: InputStream) = + if (is.readByte == (0: Byte)) Left(lRead.read(is)) + else Right(rRead.read(is)) + } + + implicit def tuple2[T1: Reader, T2: Reader]: Reader[(T1, T2)] = new Reader[(T1, T2)] { + val r1 = implicitly[Reader[T1]] + val r2 = implicitly[Reader[T2]] + def read(is: InputStream) = (r1.read(is), r2.read(is)) + } + + implicit def array[@specialized(Boolean, Byte, Short, Int, Long, Float, Double) T: Reader: ClassTag]: Reader[Array[T]] = + new Reader[Array[T]] { + val readerT = implicitly[Reader[T]] + def read(is: InputStream) = { + val size = is.readSize + val res = new Array[T](size) + @annotation.tailrec + def go(p: Int): Unit = + if (p == size) () + else { + res(p) = readerT.read(is) + go(p + 1) + } + go(0) + res + } + } + + // Scala seems to have issues with this being implicit + def collection[T: Reader, C](implicit cbf: CanBuildFrom[Nothing, T, C]): Reader[C] = new Reader[C] { + val readerT = implicitly[Reader[T]] + def read(is: InputStream): C = { + val builder = cbf() + val size = is.readSize + builder.sizeHint(size) + @annotation.tailrec + def go(idx: Int): Unit = + if (idx == size) () + else { + builder += readerT.read(is) + go(idx + 1) + } + + go(0) + builder.result + } + } +} diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization.scala new file mode 100644 index 0000000000..03fb695edb --- /dev/null +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization.scala @@ -0,0 +1,149 @@ +/* +Copyright 2015 Twitter, Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + +http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package com.twitter.scalding.serialization + +import java.io.{ ByteArrayInputStream, ByteArrayOutputStream, InputStream, OutputStream, Serializable } + +import scala.util.{ Failure, Success, Try } +import scala.util.hashing.Hashing + +/** + * This is a base Input/OutputStream-based serialization typeclass + * This is useful for value serialization in hadoop when we don't + * need to do key sorting for partitioning. + * + * This serialization typeclass must serialize equivalent objects + * identically to be lawful. Given that constraint, we can always + * get an Equiv and Hashing from a Serialization (by doing byte-wise + * equivalence or byte-wise hashing). + * + * A serialization always gives a hash because one can just + * serialize and then hash the bytes. You might prefer another + * implementation. This must satisfy: + * (!equiv(a, b)) || (hash(a) == hash(b)) + */ +trait Serialization[T] extends Equiv[T] with Hashing[T] with Serializable { + def read(in: InputStream): Try[T] + def write(out: OutputStream, t: T): Try[Unit] + /** + * If all items have a static size, this returns Some, else None + */ + def staticSize: Option[Int] = None + /** + * returns Some if the size is cheap to calculate. + * otherwise the caller should just serialize into an ByteArrayOutputStream + */ + def dynamicSize(t: T): Option[Int] = None +} + +object Serialization { + import JavaStreamEnrichments._ + /** + * This is a constant for us to reuse in Serialization.write + */ + val successUnit: Try[Unit] = Success(()) + + def equiv[T](a: T, b: T)(implicit ser: Serialization[T]): Boolean = + ser.equiv(a, b) + + def hash[T](t: T)(implicit ser: Serialization[T]): Int = + ser.hash(t) + + def read[T](in: InputStream)(implicit ser: Serialization[T]): Try[T] = + ser.read(in) + + def write[T](out: OutputStream, t: T)(implicit ser: Serialization[T]): Try[Unit] = + ser.write(out, t) + + def toBytes[T: Serialization](t: T): Array[Byte] = { + val baos = new ByteArrayOutputStream + write(baos, t).get // this should only throw on OOM + baos.toByteArray + } + + def fromBytes[T: Serialization](b: Array[Byte]): Try[T] = + read(new ByteArrayInputStream(b)) + + private def roundTrip[T](t: T)(implicit ser: Serialization[T]): T = { + val baos = new ByteArrayOutputStream + ser.write(baos, t).get // should never throw on a ByteArrayOutputStream + ser.read(baos.toInputStream).get + } + + /** + * Do these two items write equivalently? + */ + def writeEquiv[T: Serialization](a: T, b: T): Boolean = + java.util.Arrays.equals(toBytes(a), toBytes(b)) + + /** + * write followed by read should give an equivalent T + * + * This is a law that serialization must follow. It is here for + * documentation and for use within tests without any dependence on + * specific test frameworks. + * + * forAll(roundTripLaw[T]) in a valid test in scalacheck style + */ + def roundTripLaw[T: Serialization]: Law1[T] = + Law1("roundTrip", { (t: T) => equiv(roundTrip(t), t) }) + + /** + * If two items are equal, they should serialize byte for byte equivalently + */ + def serializationIsEquivalence[T: Serialization]: Law2[T] = + Law2("equiv(a, b) == (write(a) == write(b))", { (t1: T, t2: T) => + equiv(t1, t2) == writeEquiv(t1, t2) + }) + + def hashCodeImpliesEquality[T: Serialization]: Law2[T] = + Law2("equiv(a, b) => hash(a) == hash(b)", { (t1: T, t2: T) => + !equiv(t1, t2) || (hash(t1) == hash(t2)) + }) + + def reflexivity[T: Serialization]: Law1[T] = + Law1("equiv(a, a) == true", { (t1: T) => equiv(t1, t1) }) + + /** + * The sizes must match and be correct if they are present + */ + def sizeLaw[T: Serialization]: Law1[T] = + Law1("staticSize.orElse(dynamicSize(t)).map { _ == toBytes(t).length }", + { (t: T) => + val ser = implicitly[Serialization[T]] + (ser.staticSize, ser.dynamicSize(t)) match { + case (Some(s), Some(d)) if d == s => toBytes(t).length == s + case (Some(s), _) => false // if static exists it must match dynamic + case (None, Some(d)) => toBytes(t).length == d + case (None, None) => true // can't tell + } + }) + + def transitivity[T: Serialization]: Law3[T] = + Law3("equiv(a, b) && equiv(b, c) => equiv(a, c)", + { (t1: T, t2: T, t3: T) => + !(equiv(t1, t2) && equiv(t2, t3)) || equiv(t1, t3) + }) + + def allLaws[T: Serialization]: Iterable[Law[T]] = + List(roundTripLaw, + serializationIsEquivalence, + hashCodeImpliesEquality, + reflexivity, + sizeLaw, + transitivity) +} diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization2.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization2.scala new file mode 100644 index 0000000000..e87edb7958 --- /dev/null +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization2.scala @@ -0,0 +1,64 @@ +/* +Copyright 2015 Twitter, Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + +http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package com.twitter.scalding.serialization + +import java.io.{ ByteArrayInputStream, ByteArrayOutputStream, InputStream, OutputStream, Serializable } + +import scala.util.{ Failure, Success, Try } + +class Serialization2[A, B](val serA: Serialization[A], val serB: Serialization[B]) extends Serialization[(A, B)] { + private val MAX_PRIME = Int.MaxValue // turns out MaxValue is a prime, which we want below + override def hash(x: (A, B)) = serA.hash(x._1) + MAX_PRIME * serB.hash(x._2) + override def equiv(x: (A, B), y: (A, B)): Boolean = + serA.equiv(x._1, y._1) && serB.equiv(x._2, y._2) + + override def read(in: InputStream): Try[(A, B)] = { + val a = serA.read(in) + val b = serB.read(in) + (a, b) match { + case (Success(a), Success(b)) => Success((a, b)) + case (Failure(e), _) => Failure(e) + case (_, Failure(e)) => Failure(e) + } + } + + override def write(out: OutputStream, a: (A, B)): Try[Unit] = { + val resA = serA.write(out, a._1) + if (resA.isSuccess) serB.write(out, a._2) + else resA + } +} + +class OrderedSerialization2[A, B](val ordA: OrderedSerialization[A], + val ordB: OrderedSerialization[B]) extends Serialization2[A, B](ordA, ordB) with OrderedSerialization[(A, B)] { + override def compare(x: (A, B), y: (A, B)) = { + val ca = ordA.compare(x._1, y._1) + if (ca != 0) ca + else ordB.compare(x._2, y._2) + } + override def compareBinary(a: InputStream, b: InputStream) = { + // This mutates the buffers and advances them. Only keep reading if they are different + val cA = ordA.compareBinary(a, b) + // we have to read the second ones to skip + val cB = ordB.compareBinary(a, b) + cA match { + case OrderedSerialization.Equal => cB + case f @ OrderedSerialization.CompareFailure(_) => f + case _ => cA // the first is not equal + } + } +} diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/StringOrderedSerialization.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/StringOrderedSerialization.scala new file mode 100644 index 0000000000..1e64e94459 --- /dev/null +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/StringOrderedSerialization.scala @@ -0,0 +1,116 @@ +/* +Copyright 2015 Twitter, Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + +http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package com.twitter.scalding.serialization + +import java.io.{ InputStream, OutputStream } +import scala.util.{ Failure, Success, Try } +import scala.util.control.NonFatal + +import JavaStreamEnrichments._ + +object StringOrderedSerialization { + final def binaryIntCompare(leftSize: Int, seekingLeft: InputStream, rightSize: Int, seekingRight: InputStream): Int = { + val toCheck = math.min(leftSize, rightSize) + // we can check longs at a time this way: + val longs = toCheck / 8 + + @annotation.tailrec + def compareLong(count: Int): Int = + if (count == 0) 0 + else { + val cmp = UnsignedComparisons.unsignedLongCompare(seekingLeft.readLong, seekingRight.readLong) + if (cmp == 0) compareLong(count - 1) + else cmp + } + + /* + * This algorithm only works if count in {0, 1, 2, 3}. Since we only + * call it that way below it is safe. + */ + def compareBytes(count: Int): Int = + if ((count & 2) == 2) { + // there are 2 or 3 bytes to read + val cmp = Integer.compare(seekingLeft.readUnsignedShort, + seekingRight.readUnsignedShort) + if (cmp != 0) cmp + else if (count == 3) Integer.compare(seekingLeft.readUnsignedByte, + seekingRight.readUnsignedByte) + else 0 + } else { + // there are 0 or 1 bytes to read + if (count == 0) 0 + else Integer.compare(seekingLeft.readUnsignedByte, + seekingRight.readUnsignedByte) + } + + /** + * Now we start by comparing blocks of longs, then maybe 1 int, then 0 - 3 bytes + */ + val lc = compareLong(longs) + if (lc != 0) lc + else { + val remaining = (toCheck - 8 * longs) + val read1Int = (remaining >= 4) + + val ic = if (read1Int) UnsignedComparisons.unsignedIntCompare(seekingLeft.readInt, seekingRight.readInt) else 0 + if (ic != 0) ic + else { + val bytes = remaining - (if (read1Int) 4 else 0) + val bc = compareBytes(bytes) + if (bc != 0) bc + else { + // the size is the fallback when the prefixes match: + Integer.compare(leftSize, rightSize) + } + } + } + } +} + +class StringOrderedSerialization extends OrderedSerialization[String] { + import StringOrderedSerialization._ + override def hash(s: String) = s.hashCode + override def compare(a: String, b: String) = a.compareTo(b) + override def read(in: InputStream) = try { + val byteString = new Array[Byte](in.readSize) + in.readFully(byteString) + Success(new String(byteString, "UTF-8")) + } catch { case NonFatal(e) => Failure(e) } + + override def write(b: OutputStream, s: String) = try { + val bytes = s.getBytes("UTF-8") + b.writeSize(bytes.length) + b.writeBytes(bytes) + Serialization.successUnit + } catch { case NonFatal(e) => Failure(e) } + + override def compareBinary(lhs: InputStream, rhs: InputStream) = try { + val leftSize = lhs.readSize + val rightSize = rhs.readSize + + val seekingLeft = PositionInputStream(lhs) + val seekingRight = PositionInputStream(rhs) + + val leftStart = seekingLeft.position + val rightStart = seekingRight.position + + val res = OrderedSerialization.resultFrom(binaryIntCompare(leftSize, seekingLeft, rightSize, seekingRight)) + seekingLeft.seekToPosition(leftStart + leftSize) + seekingRight.seekToPosition(rightStart + rightSize) + res + } +} diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/UnsignedComparisons.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/UnsignedComparisons.scala new file mode 100644 index 0000000000..a15a0664ec --- /dev/null +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/UnsignedComparisons.scala @@ -0,0 +1,49 @@ +/* +Copyright 2015 Twitter, Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + +http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package com.twitter.scalding.serialization + +object UnsignedComparisons { + final def unsignedLongCompare(a: Long, b: Long): Int = if (a == b) 0 else { + // We only get into this block when the a != b, so it has to be the last + // block + val firstBitXor = (a ^ b) & (1L << 63) + // If both are on the same side of zero, normal compare works + if (firstBitXor == 0) java.lang.Long.compare(a, b) + else if (b >= 0) 1 + else -1 + } + final def unsignedIntCompare(a: Int, b: Int): Int = if (a == b) 0 else { + val firstBitXor = (a ^ b) & (1 << 31) + // If both are on the same side of zero, normal compare works + if (firstBitXor == 0) Integer.compare(a, b) + else if (b >= 0) 1 + else -1 + } + final def unsignedShortCompare(a: Short, b: Short): Int = if (a == b) 0 else { + // We have to convert to bytes to Int on JVM to do + // anything anyway, so might as well compare in that space + def fromShort(x: Short): Int = if (x < 0) x + (1 << 16) else x.toInt + Integer.compare(fromShort(a), fromShort(b)) + } + final def unsignedByteCompare(a: Byte, b: Byte): Int = if (a == b) 0 else { + // We have to convert to bytes to Int on JVM to do + // anything anyway, so might as well compare in that space + def fromByte(x: Byte): Int = if (x < 0) x + (1 << 8) else x.toInt + Integer.compare(fromByte(a), fromByte(b)) + } +} + diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/WrappedSerialization.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/WrappedSerialization.scala new file mode 100644 index 0000000000..1bda3f179c --- /dev/null +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/WrappedSerialization.scala @@ -0,0 +1,121 @@ +/* +Copyright 2014 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.serialization + +import org.apache.hadoop.io.serializer.{ Serialization => HSerialization, Deserializer, Serializer } +import org.apache.hadoop.conf.{ Configurable, Configuration } + +import java.io.{ DataInputStream, DataOutputStream, InputStream, OutputStream } +import com.twitter.bijection.{ Injection, JavaSerializationInjection, Base64String } +import scala.collection.JavaConverters._ + +/** + * WrappedSerialization wraps a value in a wrapper class that + * has an associated Binary that is used to deserialize + * items wrapped in the wrapper + */ +class WrappedSerialization[T] extends HSerialization[T] with Configurable { + + import WrappedSerialization.ClassSerialization + + private var conf: Option[Configuration] = None + private var serializations: Iterable[ClassSerialization[_]] = Nil + + override def getConf: Configuration = conf.get + override def setConf(config: Configuration) { + conf = Some(config) + serializations = WrappedSerialization.getBinary(config) + } + + def accept(c: Class[_]): Boolean = + serializations.exists { case (cls, _) => cls == c } + + def getSerialization(c: Class[T]): Option[Serialization[T]] = + serializations.collectFirst { case (cls, b) if cls == c => b } + // This cast should never fail since we matched the class + .asInstanceOf[Option[Serialization[T]]] + + def getSerializer(c: Class[T]): Serializer[T] = + new BinarySerializer(getSerialization(c).getOrElse(sys.error(s"Class: ${c} not found"))) + + def getDeserializer(c: Class[T]): Deserializer[T] = + new BinaryDeserializer(getSerialization(c).getOrElse(sys.error(s"Class: ${c} not found"))) + +} + +class BinarySerializer[T](buf: Serialization[T]) extends Serializer[T] { + private var out: OutputStream = _ + def open(os: OutputStream): Unit = { + out = os + } + def close(): Unit = { out = null } + def serialize(t: T): Unit = { + require(out != null, "OutputStream is null") + buf.write(out, t).get + } +} + +class BinaryDeserializer[T](buf: Serialization[T]) extends Deserializer[T] { + private var is: InputStream = _ + def open(i: InputStream): Unit = { is = i } + def close(): Unit = { is = null } + def deserialize(t: T): T = { + require(is != null, "InputStream is null") + buf.read(is).get + } +} + +object WrappedSerialization { + type ClassSerialization[T] = (Class[T], Serialization[T]) + + private def getSerializer[U]: Injection[Externalizer[U], String] = { + implicit val initialInj = JavaSerializationInjection[Externalizer[U]] + Injection.connect[Externalizer[U], Array[Byte], Base64String, String] + } + + private def serialize[T](b: T): String = + getSerializer[T](Externalizer(b)) + + private def deserialize[T](str: String): T = + getSerializer[T].invert(str).get.get + + private val confKey = "com.twitter.scalding.serialization.WrappedSerialization" + + def rawSetBinary(bufs: Iterable[ClassSerialization[_]], fn: (String, String) => Unit) = { + fn(confKey, bufs.map { case (cls, buf) => s"${cls.getName}:${serialize(buf)}" }.mkString(",")) + } + def setBinary(conf: Configuration, bufs: Iterable[ClassSerialization[_]]): Unit = + rawSetBinary(bufs, { case (k, v) => conf.set(k, v) }) + + def getBinary(conf: Configuration): Iterable[ClassSerialization[_]] = + conf + .iterator + .asScala + .map { it => + (it.getKey, it.getValue) + } + .filter(_._1.startsWith(confKey)) + .map { + case (_, clsbuf) => + clsbuf.split(":") match { + case Array(className, serialization) => + // Jump through a hoop to get scalac happy + def deser[T](cls: Class[T]): ClassSerialization[T] = (cls, deserialize[Serialization[T]](serialization)) + deser(conf.getClassByName(className)) + case _ => sys.error(s"ill formed bufferables: ${clsbuf}") + } + }.toList +} diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Writer.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Writer.scala new file mode 100644 index 0000000000..4db85ef171 --- /dev/null +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Writer.scala @@ -0,0 +1,128 @@ +/* +Copyright 2015 Twitter, Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + +http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ +package com.twitter.scalding.serialization + +import java.io.OutputStream + +/** + * This is a specialized typeclass to make it easier to implement Serializations. + * The specialization *should* mean that there is no boxing and if the JIT + * does its work, Writer should compose well (via collections, Tuple2, Option, Either) + */ +trait Writer[@specialized(Boolean, Byte, Short, Int, Long, Float, Double) -T] { + def write(os: OutputStream, t: T): Unit +} + +object Writer { + import JavaStreamEnrichments._ + + def write[@specialized(Boolean, Byte, Short, Int, Long, Float, Double) T](os: OutputStream, + t: T)(implicit w: Writer[T]): Unit = + w.write(os, t) + /* + * Instances below + */ + implicit val unit: Writer[Unit] = new Writer[Unit] { + def write(os: OutputStream, u: Unit) = () + } + implicit val boolean: Writer[Boolean] = new Writer[Boolean] { + def write(os: OutputStream, b: Boolean) = os.writeBoolean(b) + } + implicit val byte: Writer[Byte] = new Writer[Byte] { + def write(os: OutputStream, b: Byte) = os.write(b) + } + implicit val short: Writer[Short] = new Writer[Short] { + def write(os: OutputStream, s: Short) = os.writeShort(s) + } + implicit val int: Writer[Int] = new Writer[Int] { + def write(os: OutputStream, s: Int) = os.writeInt(s) + } + implicit val long: Writer[Long] = new Writer[Long] { + def write(os: OutputStream, s: Long) = os.writeLong(s) + } + implicit val float: Writer[Float] = new Writer[Float] { + def write(os: OutputStream, s: Float) = os.writeFloat(s) + } + implicit val double: Writer[Double] = new Writer[Double] { + def write(os: OutputStream, s: Double) = os.writeDouble(s) + } + implicit val string: Writer[String] = new Writer[String] { + def write(os: OutputStream, s: String) = { + val bytes = s.getBytes("UTF-8") + os.writeSize(bytes.length) + os.writeBytes(bytes) + } + } + + implicit def option[T: Writer]: Writer[Option[T]] = new Writer[Option[T]] { + val w = implicitly[Writer[T]] + def write(os: OutputStream, t: Option[T]) = + if (t.isDefined) { + os.write(1: Byte) + w.write(os, t.get) + } else os.write(0: Byte) + } + + implicit def either[L: Writer, R: Writer]: Writer[Either[L, R]] = new Writer[Either[L, R]] { + val lw = implicitly[Writer[L]] + val rw = implicitly[Writer[R]] + def write(os: OutputStream, e: Either[L, R]) = e match { + case Left(l) => + os.write(0: Byte) + lw.write(os, l) + case Right(r) => + os.write(1: Byte) + rw.write(os, r) + } + } + + implicit def tuple2[T1: Writer, T2: Writer]: Writer[(T1, T2)] = new Writer[(T1, T2)] { + val w1 = implicitly[Writer[T1]] + val w2 = implicitly[Writer[T2]] + def write(os: OutputStream, tup: (T1, T2)) = { + w1.write(os, tup._1) + w2.write(os, tup._2) + } + } + + implicit def array[@specialized(Boolean, Byte, Short, Int, Long, Float, Double) T: Writer]: Writer[Array[T]] = + new Writer[Array[T]] { + val writerT = implicitly[Writer[T]] + def write(os: OutputStream, a: Array[T]) = { + val size = a.length + os.writeSize(size) + @annotation.tailrec + def go(p: Int): Unit = + if (p == size) () + else { writerT.write(os, a(p)); go(p + 1) } + + go(0) + } + } + + // Scala has problems with this being implicit + def collection[T: Writer, C <: Iterable[T]]: Writer[C] = new Writer[C] { + val writerT = implicitly[Writer[T]] + def write(os: OutputStream, c: C) = { + val size = c.size + os.writeSize(size) + c.foreach { t: T => + writerT.write(os, t) + } + } + } +} + diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/CoGrouped.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/CoGrouped.scala index 3f654f63bc..33a5066fe0 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/CoGrouped.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/CoGrouped.scala @@ -204,70 +204,85 @@ trait CoGrouped[K, +R] extends KeyedListLike[K, R, CoGrouped] with CoGroupable[K val ord = keyOrdering TypedPipeFactory({ (flowDef, mode) => - val newPipe = if (firstCount == inputs.size) { - /** - * This is a self-join - * Cascading handles this by sending the data only once, spilling to disk if - * the groups don't fit in RAM, then doing the join on this one set of data. - * This is fundamentally different than the case where the first item is - * not repeated. That case is below - */ - val NUM_OF_SELF_JOINS = firstCount - 1 - new CoGroup(assignName(inputs.head.toPipe[(Any, Any)](("key", "value"))(flowDef, mode, tup2Setter)), - RichFields(StringField("key")(ord, None)), - NUM_OF_SELF_JOINS, - outFields(firstCount), - new DistinctCoGroupJoiner(firstCount, joinFunction)) - } else if (firstCount == 1) { - /** - * As long as the first one appears only once, we can handle self joins on the others: - * Cascading does this by maybe spilling all the streams other than the first item. - * This is handled by a different CoGroup constructor than the above case. - */ - def renamePipe(idx: Int, p: TypedPipe[(K, Any)]): Pipe = - p.toPipe[(K, Any)](List("key%d".format(idx), "value%d".format(idx)))(flowDef, mode, tup2Setter) - - // This is tested for the properties we need (non-reordering) - val distincts = CoGrouped.distinctBy(inputs)(identity) - val dsize = distincts.size - val isize = inputs.size - - val groupFields: Array[Fields] = (0 until dsize) - .map { idx => RichFields(StringField("key%d".format(idx))(ord, None)) } - .toArray - - val pipes: Array[Pipe] = distincts - .zipWithIndex - .map { case (item, idx) => assignName(renamePipe(idx, item)) } - .toArray - - val cjoiner = if (isize != dsize) { - // avoid capturing anything other than the mapping ints: - val mapping: Map[Int, Int] = inputs.zipWithIndex.map { - case (item, idx) => - idx -> distincts.indexWhere(_ == item) - }.toMap - - new CoGroupedJoiner(isize, joinFunction) { - val distinctSize = dsize - def distinctIndexOf(orig: Int) = mapping(orig) + val newPipe = Grouped.maybeBox[K, Any](ord) { (tupset, ordKeyField) => + if (firstCount == inputs.size) { + /** + * This is a self-join + * Cascading handles this by sending the data only once, spilling to disk if + * the groups don't fit in RAM, then doing the join on this one set of data. + * This is fundamentally different than the case where the first item is + * not repeated. That case is below + */ + val NUM_OF_SELF_JOINS = firstCount - 1 + new CoGroup(assignName(inputs.head.toPipe[(K, Any)](("key", "value"))(flowDef, mode, + tupset)), + ordKeyField, + NUM_OF_SELF_JOINS, + outFields(firstCount), + new DistinctCoGroupJoiner(firstCount, Grouped.keyGetter(ord), joinFunction)) + } else if (firstCount == 1) { + + def keyId(idx: Int): String = "key%d".format(idx) + /** + * As long as the first one appears only once, we can handle self joins on the others: + * Cascading does this by maybe spilling all the streams other than the first item. + * This is handled by a different CoGroup constructor than the above case. + */ + def renamePipe(idx: Int, p: TypedPipe[(K, Any)]): Pipe = + p.toPipe[(K, Any)](List(keyId(idx), "value%d".format(idx)))(flowDef, mode, + tupset) + + // This is tested for the properties we need (non-reordering) + val distincts = CoGrouped.distinctBy(inputs)(identity) + val dsize = distincts.size + val isize = inputs.size + + def makeFields(id: Int): Fields = { + val comp = ordKeyField.getComparators.apply(0) + val fieldName = keyId(id) + val f = new Fields(fieldName) + f.setComparator(fieldName, comp) + f } - } else new DistinctCoGroupJoiner(isize, joinFunction) - new CoGroup(pipes, groupFields, outFields(dsize), cjoiner) - } else { - /** - * This is non-trivial to encode in the type system, so we throw this exception - * at the planning phase. - */ - sys.error("Except for self joins, where you are joining something with only itself,\n" + - "left-most pipe can only appear once. Firsts: " + - inputs.collect { case x if x == inputs.head => x }.toString) + val groupFields: Array[Fields] = (0 until dsize) + .map(makeFields) + .toArray + + val pipes: Array[Pipe] = distincts + .zipWithIndex + .map { case (item, idx) => assignName(renamePipe(idx, item)) } + .toArray + + val cjoiner = if (isize != dsize) { + // avoid capturing anything other than the mapping ints: + val mapping: Map[Int, Int] = inputs.zipWithIndex.map { + case (item, idx) => + idx -> distincts.indexWhere(_ == item) + }.toMap + + new CoGroupedJoiner(isize, Grouped.keyGetter(ord), joinFunction) { + val distinctSize = dsize + def distinctIndexOf(orig: Int) = mapping(orig) + } + } else new DistinctCoGroupJoiner(isize, Grouped.keyGetter(ord), joinFunction) + + new CoGroup(pipes, groupFields, outFields(dsize), cjoiner) + } else { + /** + * This is non-trivial to encode in the type system, so we throw this exception + * at the planning phase. + */ + sys.error("Except for self joins, where you are joining something with only itself,\n" + + "left-most pipe can only appear once. Firsts: " + + inputs.collect { case x if x == inputs.head => x }.toString) + } } /* * the CoGrouped only populates the first two fields, the second two * are null. We then project out at the end of the method. */ + val pipeWithRed = RichPipe.setReducers(newPipe, reducers.getOrElse(-1)).project('key, 'value) //Construct the new TypedPipe TypedPipe.from[(K, R)](pipeWithRed, ('key, 'value))(flowDef, mode, tuple2Converter) @@ -275,7 +290,7 @@ trait CoGrouped[K, +R] extends KeyedListLike[K, R, CoGrouped] with CoGroupable[K } } -abstract class CoGroupedJoiner[K](inputSize: Int, joinFunction: (K, Iterator[CTuple], Seq[Iterable[CTuple]]) => Iterator[Any]) extends CJoiner { +abstract class CoGroupedJoiner[K](inputSize: Int, getter: TupleGetter[K], joinFunction: (K, Iterator[CTuple], Seq[Iterable[CTuple]]) => Iterator[Any]) extends CJoiner { val distinctSize: Int def distinctIndexOf(originalPos: Int): Int @@ -288,11 +303,10 @@ abstract class CoGroupedJoiner[K](inputSize: Int, joinFunction: (K, Iterator[CTu override def getIterator(jc: JoinerClosure) = { val iters = (0 until distinctSize).map { jc.getIterator(_).asScala.buffered } - val key = iters + val keyTuple = iters .collectFirst { case iter if iter.nonEmpty => iter.head } .get // One of these must have a key - .getObject(0) - .asInstanceOf[K] + val key = getter.get(keyTuple, 0) val leftMost = iters.head @@ -315,8 +329,9 @@ abstract class CoGroupedJoiner[K](inputSize: Int, joinFunction: (K, Iterator[CTu // If all the input pipes are unique, this works: class DistinctCoGroupJoiner[K](count: Int, + getter: TupleGetter[K], joinFunction: (K, Iterator[CTuple], Seq[Iterable[CTuple]]) => Iterator[Any]) - extends CoGroupedJoiner[K](count, joinFunction) { + extends CoGroupedJoiner[K](count, getter, joinFunction) { val distinctSize = count def distinctIndexOf(idx: Int) = idx } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/Grouped.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/Grouped.scala index c25f0f7ca9..4e821417bd 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/Grouped.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/Grouped.scala @@ -23,12 +23,22 @@ import com.twitter.scalding.TupleConverter.tuple2Converter import com.twitter.scalding.TupleSetter.tup2Setter import com.twitter.scalding._ +import com.twitter.scalding.serialization.{ + Boxed, + BoxedOrderedSerialization, + CascadingBinaryComparator, + OrderedSerialization, + WrappedSerialization +} import cascading.flow.FlowDef import cascading.pipe.Pipe -import cascading.tuple.Fields - +import cascading.property.ConfigDef +import cascading.tuple.{ Fields, Tuple => CTuple } +import java.util.Comparator import scala.collection.JavaConverters._ +import scala.util.Try +import scala.collection.immutable.Queue import Dsl._ @@ -77,15 +87,81 @@ object Grouped { def apply[K, V](pipe: TypedPipe[(K, V)])(implicit ordering: Ordering[K]): Grouped[K, V] = IdentityReduce(ordering, pipe, None) - def keySorting[T](ord: Ordering[T]): Fields = sorting("key", ord) - def valueSorting[T](implicit ord: Ordering[T]): Fields = sorting("value", ord) + def valueSorting[V](ord: Ordering[V]): Fields = Field.singleOrdered[V]("value")(ord) - def sorting[T](key: String, ord: Ordering[T]): Fields = { - val f = new Fields(key) - f.setComparator(key, ord) - f + /** + * If we are using OrderedComparable, we need to box the key + * to prevent other serializers from handling the key + */ + def maybeBox[K, V](ord: Ordering[K])(op: (TupleSetter[(K, V)], Fields) => Pipe): Pipe = ord match { + case ordser: OrderedSerialization[K] => + val (boxfn, cls) = Boxed.next[K] + val ts = tup2Setter[(Boxed[K], V)].contraMap { kv1: (K, V) => (boxfn(kv1._1), kv1._2) } + val boxordSer = BoxedOrderedSerialization(boxfn, ordser) + val keyF = new Fields("key") + keyF.setComparator("key", new CascadingBinaryComparator(boxordSer)) + val pipe = op(ts, keyF) + + case class ToVisit[T](queue: Queue[T], inQueue: Set[T]) { + def maybeAdd(t: T): ToVisit[T] = if (inQueue(t)) this else { + ToVisit(queue :+ t, inQueue + t) + } + def next: Option[(T, ToVisit[T])] = + if (inQueue.isEmpty) None + else Some((queue.head, ToVisit(queue.tail, inQueue - queue.head))) + } + + @annotation.tailrec + def go(p: Pipe, visited: Set[Pipe], toVisit: ToVisit[Pipe]): Set[Pipe] = { + val notSeen: Set[Pipe] = p.getPrevious.filter(i => !visited.contains(i)).toSet + val nextVisited: Set[Pipe] = visited + p + val nextToVisit = notSeen.foldLeft(toVisit) { case (prev, n) => prev.maybeAdd(n) } + + nextToVisit.next match { + case Some((h, innerNextToVisit)) => go(h, nextVisited, innerNextToVisit) + case _ => nextVisited + } + } + + val allPipes = go(pipe, Set[Pipe](), ToVisit[Pipe](Queue.empty, Set.empty)) + + WrappedSerialization.rawSetBinary(List((cls, boxordSer)), + { + case (k, v) => + allPipes.foreach { p => + p.getStepConfigDef().setProperty(k + cls, v) + } + }) + pipe + case _ => + val ts = tup2Setter[(K, V)] + val keyF = Field.singleOrdered("key")(ord) + op(ts, keyF) } + def tuple2Conv[K, V](ord: Ordering[K]): TupleConverter[(K, V)] = + ord match { + case _: OrderedSerialization[_] => + tuple2Converter[Boxed[K], V].andThen { kv => + (kv._1.get, kv._2) + } + case _ => tuple2Converter[K, V] + } + def keyConverter[K](ord: Ordering[K]): TupleConverter[K] = + ord match { + case _: OrderedSerialization[_] => + TupleConverter.singleConverter[Boxed[K]].andThen(_.get) + case _ => TupleConverter.singleConverter[K] + } + def keyGetter[K](ord: Ordering[K]): TupleGetter[K] = + ord match { + case _: OrderedSerialization[K] => + new TupleGetter[K] { + def get(tup: CTuple, i: Int) = tup.getObject(i).asInstanceOf[Boxed[K]].get + } + case _ => TupleGetter.castingGetter + } + def addEmptyGuard[K, V1, V2](fn: (K, Iterator[V1]) => Iterator[V2]): (K, Iterator[V1]) => Iterator[V2] = { (key: K, iter: Iterator[V1]) => if (iter.nonEmpty) fn(key, iter) else Iterator.empty } @@ -126,14 +202,15 @@ sealed trait ReduceStep[K, V1] extends KeyedPipe[K] { */ def mapped: TypedPipe[(K, V1)] // make the pipe and group it, only here because it is common - protected def groupOp[V2](gb: GroupBuilder => GroupBuilder): TypedPipe[(K, V2)] = { + protected def groupOp[V2](gb: GroupBuilder => GroupBuilder): TypedPipe[(K, V2)] = TypedPipeFactory({ (fd, mode) => - val reducedPipe = mapped - .toPipe(Grouped.kvFields)(fd, mode, tup2Setter) - .groupBy(Grouped.keySorting(keyOrdering))(gb) - TypedPipe.from(reducedPipe, Grouped.kvFields)(fd, mode, tuple2Converter[K, V2]) + val pipe = Grouped.maybeBox[K, V1](keyOrdering) { (tupleSetter, fields) => + mapped + .toPipe(Grouped.kvFields)(fd, mode, tupleSetter) + .groupBy(fields)(gb) + } + TypedPipe.from(pipe, Grouped.kvFields)(fd, mode, Grouped.tuple2Conv[K, V2](keyOrdering)) }) - } } case class IdentityReduce[K, V1]( @@ -370,7 +447,7 @@ case class ValueSortedReduce[K, V1, V2]( groupOp { _.sortBy(vSort) .every(new cascading.pipe.Every(_, Grouped.valueField, - new TypedBufferOp(reduceFn, Grouped.valueField), Fields.REPLACE)) + new TypedBufferOp(Grouped.keyConverter(keyOrdering), reduceFn, Grouped.valueField), Fields.REPLACE)) .reducers(reducers.getOrElse(-1)) } } @@ -409,7 +486,7 @@ case class IteratorMappedReduce[K, V1, V2]( override lazy val toTypedPipe = groupOp { _.every(new cascading.pipe.Every(_, Grouped.valueField, - new TypedBufferOp(reduceFn, Grouped.valueField), Fields.REPLACE)) + new TypedBufferOp(Grouped.keyConverter(keyOrdering), reduceFn, Grouped.valueField), Fields.REPLACE)) .reducers(reducers.getOrElse(-1)) } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/HashJoinable.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/HashJoinable.scala index 940b1a96fb..903cf5ea54 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/HashJoinable.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/HashJoinable.scala @@ -49,9 +49,9 @@ trait HashJoinable[K, +V] extends CoGroupable[K, V] with KeyedPipe[K] { TypedPipeFactory({ (fd, mode) => val newPipe = new HashJoin( RichPipe.assignName(mapside.toPipe(('key, 'value))(fd, mode, tup2Setter)), - RichFields(StringField("key")(keyOrdering, None)), + Field.singleOrdered("key")(keyOrdering), mapped.toPipe(('key1, 'value1))(fd, mode, tup2Setter), - RichFields(StringField("key1")(keyOrdering, None)), + Field.singleOrdered("key1")(keyOrdering), new HashJoiner(joinFunction, joiner)) //Construct the new TypedPipe diff --git a/scalding-core/src/test/scala/com/twitter/scalding/mathematics/SizeHintTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/mathematics/SizeHintTest.scala index 37e0e3f924..09e26128f3 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/mathematics/SizeHintTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/mathematics/SizeHintTest.scala @@ -25,7 +25,7 @@ import org.scalacheck.Gen._ object SizeHintProps extends Properties("SizeHint") { - val noClueGen = value(NoClue) + val noClueGen = const(NoClue) val finiteHintGen = for ( rows <- choose(-1L, 1000000L); diff --git a/scalding-core/src/test/scala/com/twitter/scalding/serialization/JavaStreamEnrichmentsProperties.scala b/scalding-core/src/test/scala/com/twitter/scalding/serialization/JavaStreamEnrichmentsProperties.scala new file mode 100644 index 0000000000..9f29d5f10a --- /dev/null +++ b/scalding-core/src/test/scala/com/twitter/scalding/serialization/JavaStreamEnrichmentsProperties.scala @@ -0,0 +1,99 @@ +/* +Copyright 2015 Twitter, Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + +http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ +package com.twitter.scalding.serialization + +import org.scalacheck.Arbitrary +import org.scalacheck.Properties +import org.scalacheck.Prop +import org.scalacheck.Prop.forAll +import org.scalacheck.Gen +import org.scalacheck.Prop._ + +import JavaStreamEnrichments._ +import java.io._ + +import scala.collection.generic.CanBuildFrom + +object JavaStreamEnrichmentsProperties extends Properties("JavaStreamEnrichmentsProperties") { + + def output = new ByteArrayOutputStream + + // The default Array[Equiv] is reference. WAT!? + implicit def aeq[T: Equiv]: Equiv[Array[T]] = new Equiv[Array[T]] { + def equiv(a: Array[T], b: Array[T]): Boolean = { + val teq = Equiv[T] + @annotation.tailrec + def go(pos: Int): Boolean = + if (pos == a.length) true + else { + teq.equiv(a(pos), b(pos)) && go(pos + 1) + } + + (a.length == b.length) && go(0) + } + } + implicit def teq[T1: Equiv, T2: Equiv]: Equiv[(T1, T2)] = new Equiv[(T1, T2)] { + def equiv(a: (T1, T2), b: (T1, T2)) = { + Equiv[T1].equiv(a._1, b._1) && + Equiv[T2].equiv(a._2, b._2) + } + } + + def writeRead[T: Equiv](g: Gen[T], w: (T, OutputStream) => Unit, r: InputStream => T): Prop = + forAll(g) { t => + val test = output + w(t, test) + Equiv[T].equiv(r(test.toInputStream), t) + } + def writeRead[T: Equiv: Arbitrary](w: (T, OutputStream) => Unit, r: InputStream => T): Prop = + writeRead(implicitly[Arbitrary[T]].arbitrary, w, r) + + property("Can (read/write)Size") = writeRead(Gen.chooseNum(0, Int.MaxValue), + { (i: Int, os) => os.writeSize(i) }, { _.readSize }) + + property("Can (read/write)Float") = writeRead( + { (i: Float, os) => os.writeFloat(i) }, { _.readFloat }) + + property("Can (read/write)Array[Byte]") = writeRead( + // Use list because Array has a shitty toString + { (b: List[Byte], os) => os.writeSize(b.size); os.writeBytes(b.toArray) }, + { is => + val bytes = new Array[Byte](is.readSize) + is.readFully(bytes) + bytes.toList + }) + + property("Can (read/write)Boolean") = writeRead( + { (i: Boolean, os) => os.writeBoolean(i) }, { _.readBoolean }) + + property("Can (read/write)Double") = writeRead( + { (i: Double, os) => os.writeDouble(i) }, { _.readDouble }) + + property("Can (read/write)Int") = writeRead(Gen.chooseNum(Int.MinValue, Int.MaxValue), + { (i: Int, os) => os.writeInt(i) }, { _.readInt }) + + property("Can (read/write)Long") = writeRead(Gen.chooseNum(Long.MinValue, Long.MaxValue), + { (i: Long, os) => os.writeLong(i) }, { _.readLong }) + + property("Can (read/write)Short") = writeRead(Gen.chooseNum(Short.MinValue, Short.MaxValue), + { (i: Short, os) => os.writeShort(i) }, { _.readShort }) + + property("Can (read/write)UnsignedByte") = writeRead(Gen.chooseNum(0, (1 << 8) - 1), + { (i: Int, os) => os.write(i.toByte) }, { _.readUnsignedByte }) + + property("Can (read/write)UnsignedShort") = writeRead(Gen.chooseNum(0, (1 << 16) - 1), + { (i: Int, os) => os.writeShort(i.toShort) }, { _.readUnsignedShort }) +} diff --git a/scalding-core/src/test/scala/com/twitter/scalding/serialization/SerializationProperties.scala b/scalding-core/src/test/scala/com/twitter/scalding/serialization/SerializationProperties.scala new file mode 100644 index 0000000000..a73ea684dc --- /dev/null +++ b/scalding-core/src/test/scala/com/twitter/scalding/serialization/SerializationProperties.scala @@ -0,0 +1,127 @@ +/* +Copyright 2015 Twitter, Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + +http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ +package com.twitter.scalding.serialization + +import org.scalacheck.Arbitrary +import org.scalacheck.Properties +import org.scalacheck.Prop +import org.scalacheck.Prop.forAll +import org.scalacheck.Gen +import org.scalacheck.Prop._ + +import JavaStreamEnrichments._ +import java.io._ +import scala.util.Try + +object LawTester { + def apply[T: Arbitrary](base: String, laws: Iterable[Law[T]]): Properties = + new LawTester(implicitly[Arbitrary[T]].arbitrary, base, laws) {} +} + +abstract class LawTester[T](g: Gen[T], base: String, laws: Iterable[Law[T]]) extends Properties(base) { + laws.foreach { + case Law1(name, fn) => property(name) = forAll(g)(fn) + case Law2(name, fn) => property(name) = forAll(g, g)(fn) + case Law3(name, fn) => property(name) = forAll(g, g, g)(fn) + } +} + +object SerializationProperties extends Properties("SerializationProperties") { + + import OrderedSerialization.{ resultFrom, CompareFailure, readThenCompare } + + implicit val intOrderedSerialization: OrderedSerialization[Int] = new OrderedSerialization[Int] { + def read(in: InputStream) = Try(Reader.read[Int](in)) + def write(o: OutputStream, t: Int) = Try(Writer.write[Int](o, t)) + def hash(t: Int) = t.hashCode + def compare(a: Int, b: Int) = java.lang.Integer.compare(a, b) + def compareBinary(a: InputStream, b: InputStream) = + readThenCompare(a, b)(this) + } + + implicit val stringOrdSer: OrderedSerialization[String] = new StringOrderedSerialization + + implicit def tuple[A: OrderedSerialization, B: OrderedSerialization]: OrderedSerialization[(A, B)] = + new OrderedSerialization2[A, B](implicitly, implicitly) + + def serializeSequenceCompare[T: OrderedSerialization](g: Gen[T]): Prop = forAll(Gen.listOf(g)) { list => + // make sure the list is even in size: + val pairList = (if (list.size % 2 == 1) list.tail else list).grouped(2) + val baos1 = new ByteArrayOutputStream + val baos2 = new ByteArrayOutputStream + pairList.foreach { + case Seq(a, b) => + Serialization.write(baos1, a) + Serialization.write(baos2, b) + case _ => sys.error("unreachable") + } + // now the compares must match: + val in1 = baos1.toInputStream + val in2 = baos2.toInputStream + pairList.forall { + case Seq(a, b) => + OrderedSerialization.compareBinary[T](in1, in2) == + OrderedSerialization.resultFrom(OrderedSerialization.compare(a, b)) + case _ => sys.error("unreachable") + } + } + + def serializeSequenceCompare[T: OrderedSerialization: Arbitrary]: Prop = + serializeSequenceCompare[T](implicitly[Arbitrary[T]].arbitrary) + + def serializeSequenceEquiv[T: Serialization](g: Gen[T]): Prop = forAll(Gen.listOf(g)) { list => + // make sure the list is even in size: + val pairList = (if (list.size % 2 == 1) list.tail else list).grouped(2) + val baos1 = new ByteArrayOutputStream + val baos2 = new ByteArrayOutputStream + pairList.foreach { + case Seq(a, b) => + Serialization.write(baos1, a) + Serialization.write(baos2, b) + case _ => sys.error("unreachable") + } + // now the compares must match: + val in1 = baos1.toInputStream + val in2 = baos2.toInputStream + pairList.forall { + case Seq(a, b) => + val rta = Serialization.read[T](in1).get + val rtb = Serialization.read[T](in2).get + Serialization.equiv(a, rta) && Serialization.equiv(b, rtb) + case _ => sys.error("unreachable") + } + } + def serializeSequenceEquiv[T: Serialization: Arbitrary]: Prop = + serializeSequenceEquiv[T](implicitly[Arbitrary[T]].arbitrary) + + property("sequences compare well [Int]") = serializeSequenceCompare[Int] + property("sequences equiv well [Int]") = serializeSequenceEquiv[Int] + property("sequences compare well [(Int, Int)]") = serializeSequenceCompare[(Int, Int)] + property("sequences equiv well [(Int, Int)]") = serializeSequenceEquiv[(Int, Int)] + + property("sequences compare well [String]") = serializeSequenceCompare[String] + property("sequences equiv well [String]") = serializeSequenceEquiv[String] + property("sequences compare well [(String, String)]") = serializeSequenceCompare[(String, String)] + property("sequences equiv well [(String, String)]") = serializeSequenceEquiv[(String, String)] + + // Test the independent, non-sequenced, laws as well + include(LawTester("Int Ordered", OrderedSerialization.allLaws[Int])) + include(LawTester("(Int, Int) Ordered", OrderedSerialization.allLaws[(Int, Int)])) + include(LawTester("String Ordered", OrderedSerialization.allLaws[String])) + include(LawTester("(String, Int) Ordered", OrderedSerialization.allLaws[(String, Int)])) + include(LawTester("(Int, String) Ordered", OrderedSerialization.allLaws[(Int, String)])) + include(LawTester("(String, String) Ordered", OrderedSerialization.allLaws[(String, String)])) +} diff --git a/scalding-core/src/test/scala/com/twitter/scalding/serialization/UnsignedComparisonLaws.scala b/scalding-core/src/test/scala/com/twitter/scalding/serialization/UnsignedComparisonLaws.scala new file mode 100644 index 0000000000..bb258d61af --- /dev/null +++ b/scalding-core/src/test/scala/com/twitter/scalding/serialization/UnsignedComparisonLaws.scala @@ -0,0 +1,41 @@ +package com.twitter.scalding.serialization + +import org.scalacheck.Arbitrary +import org.scalacheck.Properties +import org.scalacheck.Prop.forAll +import org.scalacheck.Gen.choose +import org.scalacheck.Prop._ + +object UnsignedComparisonLaws extends Properties("UnsignedComparisonLaws") { + + property("UnsignedLongCompare works") = forAll { (l1: Long, l2: Long) => + val cmp = UnsignedComparisons.unsignedLongCompare(l1, l2) + (l1 >= 0, l2 >= 0) match { + case (true, true) => cmp == java.lang.Long.compare(l1, l2) + case (true, false) => cmp < 0 // negative is bigger + case (false, true) => cmp > 0 + case (false, false) => cmp == java.lang.Long.compare(l1 & Long.MaxValue, l2 & Long.MaxValue) + } + } + property("UnsignedIntCompare works") = forAll { (l1: Int, l2: Int) => + val cmp = UnsignedComparisons.unsignedIntCompare(l1, l2) + (l1 >= 0, l2 >= 0) match { + case (true, true) => cmp == java.lang.Integer.compare(l1, l2) + case (true, false) => cmp < 0 // negative is bigger + case (false, true) => cmp > 0 + case (false, false) => cmp == java.lang.Integer.compare(l1 & Int.MaxValue, l2 & Int.MaxValue) + } + } + property("UnsignedByteCompare works") = forAll { (l1: Byte, l2: Byte) => + def clamp(i: Int) = if (i > 0) 1 else if (i < 0) -1 else 0 + val cmp = clamp(UnsignedComparisons.unsignedByteCompare(l1, l2)) + (l1 >= 0, l2 >= 0) match { + case (true, true) => cmp == clamp(java.lang.Byte.compare(l1, l2)) + case (true, false) => cmp < 0 // negative is bigger + case (false, true) => cmp > 0 + // Convert to positive ints + case (false, false) => cmp == java.lang.Integer.compare(l1 & Byte.MaxValue, l2 & Byte.MaxValue) + } + } +} + diff --git a/scalding-core/src/test/scala/com/twitter/scalding/serialization/WriterReaderProperties.scala b/scalding-core/src/test/scala/com/twitter/scalding/serialization/WriterReaderProperties.scala new file mode 100644 index 0000000000..a4e9fab9fd --- /dev/null +++ b/scalding-core/src/test/scala/com/twitter/scalding/serialization/WriterReaderProperties.scala @@ -0,0 +1,102 @@ +/* +Copyright 2015 Twitter, Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + +http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ +package com.twitter.scalding.serialization + +import org.scalacheck.Arbitrary +import org.scalacheck.Properties +import org.scalacheck.Prop +import org.scalacheck.Prop.forAll +import org.scalacheck.Gen +import org.scalacheck.Prop._ + +import JavaStreamEnrichments._ +import java.io._ + +import scala.collection.generic.CanBuildFrom + +object WriterReaderProperties extends Properties("WriterReaderProperties") { + + def output = new ByteArrayOutputStream + + // The default Array[Equiv] is reference. WAT!? + implicit def aeq[T: Equiv]: Equiv[Array[T]] = new Equiv[Array[T]] { + def equiv(a: Array[T], b: Array[T]): Boolean = { + val teq = Equiv[T] + @annotation.tailrec + def go(pos: Int): Boolean = + if (pos == a.length) true + else { + teq.equiv(a(pos), b(pos)) && go(pos + 1) + } + + (a.length == b.length) && go(0) + } + } + implicit def teq[T1: Equiv, T2: Equiv]: Equiv[(T1, T2)] = new Equiv[(T1, T2)] { + def equiv(a: (T1, T2), b: (T1, T2)) = { + Equiv[T1].equiv(a._1, b._1) && + Equiv[T2].equiv(a._2, b._2) + } + } + + def writerReader[T: Writer: Reader: Equiv](g: Gen[T]): Prop = + forAll(g) { t => + val test = output + Writer.write(test, t) + Equiv[T].equiv(Reader.read(test.toInputStream), t) + } + def writerReader[T: Writer: Reader: Equiv: Arbitrary]: Prop = + writerReader(implicitly[Arbitrary[T]].arbitrary) + + def writerReaderCollection[T: Writer: Reader, C <: Iterable[T]: Arbitrary: Equiv](implicit cbf: CanBuildFrom[Nothing, T, C]): Prop = + { + implicit val cwriter = Writer.collection[T, C] + implicit val creader = Reader.collection[T, C] + writerReader(implicitly[Arbitrary[C]].arbitrary) + } + + /* + * Test the Writer/Reader type-classes + */ + property("Unit Writer/Reader") = writerReader[Unit] + property("Boolean Writer/Reader") = writerReader[Boolean] + property("Byte Writer/Reader") = writerReader[Byte] + property("Short Writer/Reader") = writerReader[Short] + property("Int Writer/Reader") = writerReader[Int] + property("Long Writer/Reader") = writerReader[Long] + property("Float Writer/Reader") = writerReader[Float] + property("Double Writer/Reader") = writerReader[Double] + property("String Writer/Reader") = writerReader[String] + property("Array[Byte] Writer/Reader") = writerReader[Array[Byte]] + property("Array[Int] Writer/Reader") = writerReader[Array[Int]] + property("Array[String] Writer/Reader") = writerReader[Array[String]] + property("List[String] Writer/Reader") = + writerReaderCollection[String, List[String]] + property("(Int, Array[String]) Writer/Reader") = + writerReader[(Int, Array[String])] + + property("Option[(Int, Double)] Writer/Reader") = + writerReader[Option[(Int, Double)]] + + property("Option[Option[Unit]] Writer/Reader") = + writerReader[Option[Option[Unit]]] + + property("Either[Int, String] Writer/Reader") = + writerReader[Either[Int, String]] + + property("Map[Long, Byte] Writer/Reader") = + writerReaderCollection[(Long, Byte), Map[Long, Byte]] +} diff --git a/scalding-core/src/test/scala/com/twitter/scalding/typed/RequireOrderedSerializationTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/typed/RequireOrderedSerializationTest.scala new file mode 100644 index 0000000000..a096222518 --- /dev/null +++ b/scalding-core/src/test/scala/com/twitter/scalding/typed/RequireOrderedSerializationTest.scala @@ -0,0 +1,94 @@ +/* +Copyright 2015 Twitter, Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + +http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ +package com.twitter.scalding + +import com.twitter.scalding.serialization.CascadingBinaryComparator +import com.twitter.scalding.serialization.OrderedSerialization +import com.twitter.scalding.serialization.StringOrderedSerialization + +import org.scalatest.{ Matchers, WordSpec } + +class NoOrderdSerJob(args: Args) extends Job(args) { + + override def config = super.config + (Config.ScaldingRequireOrderedSerialization -> "true") + + TypedPipe.from(TypedTsv[(String, String)]("input")) + .group + .max + .write(TypedTsv[(String, String)]("output")) + + // This should fail + if (args.boolean("check")) { + CascadingBinaryComparator.checkForOrderedSerialization(flowDef).get + } +} + +class OrderdSerJob(args: Args) extends Job(args) { + + implicit def stringOS: OrderedSerialization[String] = new StringOrderedSerialization + + override def config = super.config + (Config.ScaldingRequireOrderedSerialization -> "true") + + TypedPipe.from(TypedTsv[(String, String)]("input")) + .group + .max + .write(TypedTsv[(String, String)]("output")) + + // This should not fail + if (args.boolean("check")) { + CascadingBinaryComparator.checkForOrderedSerialization(flowDef).get + } +} + +class RequireOrderedSerializationTest extends WordSpec with Matchers { + "A NoOrderedSerJob" should { + // This should throw + "throw with --check" in { + an[Exception] should be thrownBy { + (new NoOrderdSerJob(Mode.putMode(Local(true), Args("--check")))) + } + } + "not throw without --check" in { + (new NoOrderdSerJob(Mode.putMode(Local(true), Args("")))) + } + // throw if we try to run in: + "throw when run" in { + an[Exception] should be thrownBy { + JobTest(new NoOrderdSerJob(_)) + .source(TypedTsv[(String, String)]("input"), List(("a", "a"), ("b", "b"))) + .sink[(String, String)](TypedTsv[(String, String)]("output")) { outBuf => () } + .run + .finish + } + } + } + "A OrderedSerJob" should { + "not throw with --check" in { + // This should not throw + val osj = (new OrderdSerJob(Mode.putMode(Local(true), Args("--check")))) + } + // throw if we try to run in: + "run" in { + JobTest(new OrderdSerJob(_)) + .source(TypedTsv[(String, String)]("input"), List(("a", "a"), ("a", "b"), ("b", "b"))) + .sink[(String, String)](TypedTsv[(String, String)]("output")) { outBuf => + outBuf.toSet shouldBe Set(("a", "b"), ("b", "b")) + } + .run + .finish + } + } +} diff --git a/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/HadoopPlatformJobTest.scala b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/HadoopPlatformJobTest.scala index a5de9e29fd..c5e3cbb687 100644 --- a/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/HadoopPlatformJobTest.scala +++ b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/HadoopPlatformJobTest.scala @@ -90,6 +90,7 @@ case class HadoopPlatformJobTest( } def run { + System.setProperty("cascading.update.skip", "true") val job = initJob(cons) cluster.addClassSourceToClassPath(cons.getClass) cluster.addClassSourceToClassPath(job.getClass) diff --git a/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/HadoopSharedPlatformTest.scala b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/HadoopSharedPlatformTest.scala new file mode 100644 index 0000000000..325f83858a --- /dev/null +++ b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/HadoopSharedPlatformTest.scala @@ -0,0 +1,49 @@ +/* +Copyright 2014 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.platform + +import org.scalatest.{ BeforeAndAfterAll, Suite } + +trait HadoopSharedPlatformTest extends BeforeAndAfterAll { this: Suite => + org.apache.log4j.Logger.getLogger("org.apache.hadoop").setLevel(org.apache.log4j.Level.ERROR) + org.apache.log4j.Logger.getLogger("org.mortbay").setLevel(org.apache.log4j.Level.ERROR) + org.apache.log4j.Logger.getLogger("org.apache.hadoop.metrics2.util").setLevel(org.apache.log4j.Level.ERROR) + + val cluster = LocalCluster() + + def initialize() = cluster.initialize() + + override def beforeAll() { + cluster.synchronized { + initialize() + } + super.beforeAll() + } + + //TODO is there a way to buffer such that we see test results AFTER afterEach? Otherwise the results + // get lost in the logging + override def afterAll() { + try super.afterAll() + finally { + // Necessary because afterAll can be called from a different thread and we want to make sure that the state + // is visible. Note that this assumes there is no contention for LocalCluster (which LocalCluster ensures), + // otherwise there could be deadlock. + cluster.synchronized { + cluster.shutdown() + } + } + } +} diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/platform/PlatformTest.scala b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/platform/PlatformTest.scala index 3b3d05e7f8..07a790ed93 100644 --- a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/platform/PlatformTest.scala +++ b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/platform/PlatformTest.scala @@ -69,7 +69,7 @@ class TsvNoCacheJob(args: Args) extends Job(args) { // Keeping all of the specifications in the same tests puts the result output all together at the end. // This is useful given that the Hadoop MiniMRCluster and MiniDFSCluster spew a ton of logging. -class PlatformTests extends WordSpec with Matchers with HadoopPlatformTest { +class PlatformTests extends WordSpec with Matchers with HadoopSharedPlatformTest { org.apache.log4j.Logger.getLogger("org.apache.hadoop").setLevel(org.apache.log4j.Level.ERROR) org.apache.log4j.Logger.getLogger("org.mortbay").setLevel(org.apache.log4j.Level.ERROR) @@ -153,3 +153,51 @@ class IterableSourceDistinctTest extends WordSpec with Matchers with HadoopPlatf } } } + +object MultipleGroupByJobData { + val data: List[String] = { + val rnd = new scala.util.Random(22) + (0 until 20).map { _ => rnd.nextLong.toString }.toList + }.distinct +} + +class MultipleGroupByJob(args: Args) extends Job(args) { + import com.twitter.scalding.serialization._ + import MultipleGroupByJobData._ + implicit val stringOrdSer = new StringOrderedSerialization() + implicit val stringTup2OrdSer = new OrderedSerialization2(stringOrdSer, stringOrdSer) + val otherStream = TypedPipe.from(data).map{ k => (k, k) }.group + + TypedPipe.from(data) + .map{ k => (k, 1L) } + .group[String, Long](implicitly, stringOrdSer) + .sum + .map { + case (k, _) => + ((k, k), 1L) + } + .sumByKey[(String, String), Long](implicitly, stringTup2OrdSer, implicitly) + .map(_._1._1) + .map { t => + (t.toString, t) + } + .group + .leftJoin(otherStream) + .map(_._1) + .write(TypedTsv("output")) + +} + +class MultipleGroupByJobTest extends WordSpec with Matchers with HadoopPlatformTest { + "A grouped job" should { + import MultipleGroupByJobData._ + + "do some ops and not stamp on each other ordered serializations" in { + HadoopPlatformJobTest(new MultipleGroupByJob(_), cluster) + .source[String]("input", data) + .sink[String]("output") { _.toSet shouldBe data.map(_.toString).toSet } + .run + } + + } +} From a5e5f011336006542c050fbb3f59ee3adc33ac8f Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Wed, 25 Feb 2015 14:50:39 -1000 Subject: [PATCH 035/177] Don't turn on travis for the new macros --- .travis.yml | 31 ++++++++++++++++--------------- 1 file changed, 16 insertions(+), 15 deletions(-) diff --git a/.travis.yml b/.travis.yml index 4e239a7252..3d941de9bd 100644 --- a/.travis.yml +++ b/.travis.yml @@ -47,21 +47,22 @@ matrix: env: BUILD="base" TEST_TARGET="scalding-jdbc scalding-json" script: "scripts/run_test.sh" - - scala: 2.10.4 - env: BUILD="base" TEST_TARGET="scalding-macros" - script: "scripts/run_test.sh" - - - scala: 2.11.5 - env: BUILD="base" TEST_TARGET="scalding-macros" - script: "scripts/run_test.sh" - - - scala: 2.10.4 - env: BUILD="base" TEST_TARGET="scalding-commons-macros" - script: "scripts/run_test.sh" - - - scala: 2.11.5 - env: BUILD="base" TEST_TARGET="scalding-commons-macros" - script: "scripts/run_test.sh" +# not committed yet + # - scala: 2.10.4 + # env: BUILD="base" TEST_TARGET="scalding-macros" + # script: "scripts/run_test.sh" + + # - scala: 2.11.5 + # env: BUILD="base" TEST_TARGET="scalding-macros" + # script: "scripts/run_test.sh" + + # - scala: 2.10.4 + # env: BUILD="base" TEST_TARGET="scalding-commons-macros" + # script: "scripts/run_test.sh" + + # - scala: 2.11.5 + # env: BUILD="base" TEST_TARGET="scalding-commons-macros" + # script: "scripts/run_test.sh" - scala: 2.10.4 env: BUILD="base" TEST_TARGET="scalding-parquet scalding-parquet-scrooge" From c4e56bc63bbfb582380c271a35158ee3f3122d02 Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Wed, 25 Feb 2015 15:51:39 -1000 Subject: [PATCH 036/177] Add the macros for OrderedSerialization --- .travis.yml | 14 +- .../com/twitter/scalding/macros/Macros.scala | 3 + .../impl/OrderedBufferableProviderImpl.scala | 78 +++ .../CompileTimeLengthTypes.scala | 77 +++ .../ordered_serialization/ProductLike.scala | 169 ++++++ .../TreeOrderedBuf.scala | 319 +++++++++++ .../providers/BooleanOrderedBuf.scala | 64 +++ .../providers/ByteBufferOrderedBuf.scala | 99 ++++ .../providers/CaseClassOrderedBuf.scala | 87 +++ .../providers/EitherOrderedBuf.scala | 180 ++++++ .../providers/ImplicitOrderedBuf.scala | 86 +++ .../providers/OptionOrderedBuf.scala | 148 +++++ .../providers/PrimitiveOrderedBuf.scala | 114 ++++ .../providers/ProductOrderedBuf.scala | 124 +++++ .../providers/StringOrderedBuf.scala | 96 ++++ .../providers/TraversablesOrderedBuf.scala | 303 ++++++++++ .../providers/UnitOrderedBuf.scala | 62 +++ .../runtime_helpers/LengthCalculations.scala | 42 ++ .../runtime_helpers/TraversableHelpers.scala | 174 ++++++ .../macros/MacroOrderingProperties.scala | 526 ++++++++++++++++++ .../scalding/macros/MacrosUnitTests.scala | 2 +- .../macros/TraversableHelperLaws.scala | 50 ++ 22 files changed, 2809 insertions(+), 8 deletions(-) create mode 100644 scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/OrderedBufferableProviderImpl.scala create mode 100644 scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/CompileTimeLengthTypes.scala create mode 100644 scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/ProductLike.scala create mode 100644 scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/TreeOrderedBuf.scala create mode 100644 scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/BooleanOrderedBuf.scala create mode 100644 scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/ByteBufferOrderedBuf.scala create mode 100644 scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/CaseClassOrderedBuf.scala create mode 100644 scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/EitherOrderedBuf.scala create mode 100644 scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/ImplicitOrderedBuf.scala create mode 100644 scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/OptionOrderedBuf.scala create mode 100644 scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/PrimitiveOrderedBuf.scala create mode 100644 scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/ProductOrderedBuf.scala create mode 100644 scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/StringOrderedBuf.scala create mode 100644 scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/TraversablesOrderedBuf.scala create mode 100644 scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/UnitOrderedBuf.scala create mode 100644 scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/runtime_helpers/LengthCalculations.scala create mode 100644 scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/runtime_helpers/TraversableHelpers.scala create mode 100644 scalding-macros/src/test/scala/com/twitter/scalding/macros/MacroOrderingProperties.scala create mode 100644 scalding-macros/src/test/scala/com/twitter/scalding/macros/TraversableHelperLaws.scala diff --git a/.travis.yml b/.travis.yml index 3d941de9bd..c394d03735 100644 --- a/.travis.yml +++ b/.travis.yml @@ -47,15 +47,15 @@ matrix: env: BUILD="base" TEST_TARGET="scalding-jdbc scalding-json" script: "scripts/run_test.sh" -# not committed yet - # - scala: 2.10.4 - # env: BUILD="base" TEST_TARGET="scalding-macros" - # script: "scripts/run_test.sh" + - scala: 2.10.4 + env: BUILD="base" TEST_TARGET="scalding-macros" + script: "scripts/run_test.sh" - # - scala: 2.11.5 - # env: BUILD="base" TEST_TARGET="scalding-macros" - # script: "scripts/run_test.sh" + - scala: 2.11.5 + env: BUILD="base" TEST_TARGET="scalding-macros" + script: "scripts/run_test.sh" +# not committed yet # - scala: 2.10.4 # env: BUILD="base" TEST_TARGET="scalding-commons-macros" # script: "scripts/run_test.sh" diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/Macros.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/Macros.scala index 465a27ae5c..72d6bb9883 100644 --- a/scalding-macros/src/main/scala/com/twitter/scalding/macros/Macros.scala +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/Macros.scala @@ -20,6 +20,7 @@ import scala.language.experimental.macros import com.twitter.scalding._ import com.twitter.scalding.macros.impl._ import cascading.tuple.Fields +import com.twitter.scalding.serialization.OrderedSerialization object Macros { @@ -45,4 +46,6 @@ object Macros { def caseClassTypeDescriptor[T]: TypeDescriptor[T] = macro TypeDescriptorProviderImpl.caseClassTypeDescriptorImpl[T] def caseClassTypeDescriptorWithUnknown[T]: TypeDescriptor[T] = macro TypeDescriptorProviderImpl.caseClassTypeDescriptorWithUnknownImpl[T] + def orderedBufferSupplier[T]: OrderedSerialization[T] = macro com.twitter.scalding.macros.impl.OrderedSerializationProviderImpl[T] + } diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/OrderedBufferableProviderImpl.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/OrderedBufferableProviderImpl.scala new file mode 100644 index 0000000000..e4c9d60353 --- /dev/null +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/OrderedBufferableProviderImpl.scala @@ -0,0 +1,78 @@ +/* + Copyright 2014 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.macros.impl + +import scala.language.experimental.macros +import scala.reflect.macros.Context +import scala.util.Random + +import com.twitter.scalding.serialization.OrderedSerialization +import com.twitter.scalding.macros.impl.ordered_serialization._ +import com.twitter.scalding.macros.impl.ordered_serialization.providers._ + +object OrderedSerializationProviderImpl { + def normalizedDispatcher(c: Context)(buildDispatcher: => PartialFunction[c.Type, TreeOrderedBuf[c.type]]): PartialFunction[c.Type, TreeOrderedBuf[c.type]] = { + case tpe if (!tpe.toString.contains(ImplicitOrderedBuf.macroMarker) && !(tpe.normalize == tpe)) => + buildDispatcher(tpe.normalize) + } + + def scaldingBasicDispatchers(c: Context)(buildDispatcher: => PartialFunction[c.Type, TreeOrderedBuf[c.type]]): PartialFunction[c.Type, TreeOrderedBuf[c.type]] = { + + val primitiveDispatcher = PrimitiveOrderedBuf.dispatch(c) + val optionDispatcher = OptionOrderedBuf.dispatch(c)(buildDispatcher) + val eitherDispatcher = EitherOrderedBuf.dispatch(c)(buildDispatcher) + val caseClassDispatcher = CaseClassOrderedBuf.dispatch(c)(buildDispatcher) + val productDispatcher = ProductOrderedBuf.dispatch(c)(buildDispatcher) + val stringDispatcher = StringOrderedBuf.dispatch(c) + val traversablesDispatcher = TraversablesOrderedBuf.dispatch(c)(buildDispatcher) + val booleanDispatcher = BooleanOrderedBuf.dispatch(c) + val unitDispatcher = UnitOrderedBuf.dispatch(c) + val byteBufferDispatcher = ByteBufferOrderedBuf.dispatch(c) + + OrderedSerializationProviderImpl.normalizedDispatcher(c)(buildDispatcher) + .orElse(primitiveDispatcher) + .orElse(booleanDispatcher) + .orElse(unitDispatcher) + .orElse(optionDispatcher) + .orElse(eitherDispatcher) + .orElse(stringDispatcher) + .orElse(byteBufferDispatcher) + .orElse(traversablesDispatcher) + .orElse(caseClassDispatcher) + .orElse(productDispatcher) + } + + def fallbackImplicitDispatcher(c: Context): PartialFunction[c.Type, TreeOrderedBuf[c.type]] = + ImplicitOrderedBuf.dispatch(c) + + private def dispatcher(c: Context): PartialFunction[c.Type, TreeOrderedBuf[c.type]] = { + import c.universe._ + def buildDispatcher: PartialFunction[c.Type, TreeOrderedBuf[c.type]] = OrderedSerializationProviderImpl.dispatcher(c) + + scaldingBasicDispatchers(c)(buildDispatcher).orElse(fallbackImplicitDispatcher(c)).orElse { + case tpe: Type => c.abort(c.enclosingPosition, s"""Unable to find OrderedSerialization for type ${tpe}""") + } + } + + def apply[T](c: Context)(implicit T: c.WeakTypeTag[T]): c.Expr[OrderedSerialization[T]] = { + import c.universe._ + + val b: TreeOrderedBuf[c.type] = dispatcher(c)(T.tpe) + val res = TreeOrderedBuf.toOrderedSerialization[T](c)(b) + //println(res) + res + } +} diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/CompileTimeLengthTypes.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/CompileTimeLengthTypes.scala new file mode 100644 index 0000000000..8b419491b1 --- /dev/null +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/CompileTimeLengthTypes.scala @@ -0,0 +1,77 @@ +/* + Copyright 2014 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.macros.impl.ordered_serialization + +import scala.language.experimental.macros +import scala.reflect.macros.Context + +sealed trait CompileTimeLengthTypes[C <: Context] { + val ctx: C + def t: ctx.Tree +} +object CompileTimeLengthTypes { + + // Repesents an Int returning + object FastLengthCalculation { + def apply(c: Context)(tree: c.Tree): FastLengthCalculation[c.type] = + new FastLengthCalculation[c.type] { + override val ctx: c.type = c + override val t: c.Tree = tree + } + } + + trait FastLengthCalculation[C <: Context] extends CompileTimeLengthTypes[C] + + object MaybeLengthCalculation { + def apply(c: Context)(tree: c.Tree): MaybeLengthCalculation[c.type] = + new MaybeLengthCalculation[c.type] { + override val ctx: c.type = c + override val t: c.Tree = tree + } + } + + trait MaybeLengthCalculation[C <: Context] extends CompileTimeLengthTypes[C] + + object ConstantLengthCalculation { + def apply(c: Context)(intArg: Int): ConstantLengthCalculation[c.type] = + new ConstantLengthCalculation[c.type] { + override val toInt = intArg + override val ctx: c.type = c + override val t: c.Tree = { + import c.universe._ + q"$intArg" + } + } + } + + trait ConstantLengthCalculation[C <: Context] extends CompileTimeLengthTypes[C] { + def toInt: Int + } + + object NoLengthCalculationAvailable { + def apply(c: Context): NoLengthCalculationAvailable[c.type] = { + new NoLengthCalculationAvailable[c.type] { + override val ctx: c.type = c + override def t = { + import c.universe._ + q"""_root_.scala.sys.error("no length available")""" + } + } + } + } + + trait NoLengthCalculationAvailable[C <: Context] extends CompileTimeLengthTypes[C] +} diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/ProductLike.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/ProductLike.scala new file mode 100644 index 0000000000..17573c740e --- /dev/null +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/ProductLike.scala @@ -0,0 +1,169 @@ +/* + Copyright 2014 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.macros.impl.ordered_serialization + +import scala.language.experimental.macros +import scala.reflect.macros.Context + +import com.twitter.scalding._ + +object ProductLike { + def compareBinary(c: Context)(inputStreamA: c.TermName, inputStreamB: c.TermName)(elementData: List[(c.universe.Type, c.universe.TermName, TreeOrderedBuf[c.type])]): c.Tree = { + import c.universe._ + def freshT(id: String) = newTermName(c.fresh(id)) + + elementData.foldLeft(Option.empty[Tree]) { + case (existingTreeOpt, (tpe, accessorSymbol, tBuf)) => + existingTreeOpt match { + case Some(t) => + val lastCmp = freshT("lastCmp") + Some(q""" + val $lastCmp = $t + if($lastCmp != 0) { + $lastCmp + } else { + ${tBuf.compareBinary(inputStreamA, inputStreamB)} + } + """) + case None => + Some(tBuf.compareBinary(inputStreamA, inputStreamB)) + } + }.getOrElse(q"0") + } + + def hash(c: Context)(element: c.TermName)(elementData: List[(c.universe.Type, c.universe.TermName, TreeOrderedBuf[c.type])]): c.Tree = { + import c.universe._ + def freshT(id: String) = newTermName(c.fresh(id)) + + val currentHash = freshT("last") + + val hashUpdates = elementData.map { + case (tpe, accessorSymbol, tBuf) => + val target = freshT("target") + q""" + val $target = $element.$accessorSymbol + _root_.com.twitter.scalding.serialization.MurmerHashUtils.mixH1($currentHash, ${tBuf.hash(target)}) + """ + } + + q""" + var $currentHash: Int = _root_.com.twitter.scalding.serialization.MurmerHashUtils.seed + ..${hashUpdates} + _root_.com.twitter.scalding.serialization.MurmerHashUtils.fmix($currentHash, ${elementData.size}) + """ + } + + def put(c: Context)(inputStream: c.TermName, element: c.TermName)(elementData: List[(c.universe.Type, c.universe.TermName, TreeOrderedBuf[c.type])]): c.Tree = { + import c.universe._ + def freshT(id: String) = newTermName(c.fresh(id)) + val innerElement = freshT("innerElement") + + elementData.foldLeft(q"") { + case (existingTree, (tpe, accessorSymbol, tBuf)) => + q""" + $existingTree + val $innerElement = $element.$accessorSymbol + ${tBuf.put(inputStream, innerElement)} + """ + } + } + + def length(c: Context)(element: c.Tree)(elementData: List[(c.universe.Type, c.universe.TermName, TreeOrderedBuf[c.type])]): CompileTimeLengthTypes[c.type] = { + import c.universe._ + import CompileTimeLengthTypes._ + val (constSize, dynamicFunctions, maybeLength, noLength) = + elementData.foldLeft((0, Vector[c.Tree](), Vector[c.Tree](), 0)) { + case ((constantLength, dynamicLength, maybeLength, noLength), (tpe, accessorSymbol, tBuf)) => + + tBuf.length(q"$element.$accessorSymbol") match { + case const: ConstantLengthCalculation[_] => (constantLength + const.asInstanceOf[ConstantLengthCalculation[c.type]].toInt, dynamicLength, maybeLength, noLength) + case f: FastLengthCalculation[_] => (constantLength, dynamicLength :+ f.asInstanceOf[FastLengthCalculation[c.type]].t, maybeLength, noLength) + case m: MaybeLengthCalculation[_] => (constantLength, dynamicLength, maybeLength :+ m.asInstanceOf[MaybeLengthCalculation[c.type]].t, noLength) + case _: NoLengthCalculationAvailable[_] => (constantLength, dynamicLength, maybeLength, noLength + 1) + } + } + + val combinedDynamic = dynamicFunctions.foldLeft(q"""$constSize""") { + case (prev, t) => + q"$prev + $t" + } + + if (noLength > 0) { + NoLengthCalculationAvailable(c) + } else { + if (maybeLength.isEmpty && dynamicFunctions.isEmpty) { + ConstantLengthCalculation(c)(constSize) + } else { + if (maybeLength.isEmpty) { + FastLengthCalculation(c)(combinedDynamic) + } else { + + val const = q"_root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.ConstLen" + val dyn = q"_root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.DynamicLen" + val noLen = q"_root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.NoLengthCalculation" + // Contains an MaybeLength + val combinedMaybe: Tree = maybeLength.reduce { (hOpt, nxtOpt) => q"""$hOpt + $nxtOpt""" } + if (dynamicFunctions.nonEmpty || constSize != 0) { + MaybeLengthCalculation(c) (q""" + $combinedMaybe match { + case $const(l) => $dyn(l + $combinedDynamic) + case $dyn(l) => $dyn(l + $combinedDynamic) + case $noLen => $noLen + } + """) + } else { + MaybeLengthCalculation(c)(combinedMaybe) + } + } + } + } + } + + def compare(c: Context)(elementA: c.TermName, elementB: c.TermName)(elementData: List[(c.universe.Type, c.universe.TermName, TreeOrderedBuf[c.type])]): c.Tree = { + import c.universe._ + + def freshT(id: String) = newTermName(c.fresh(id)) + + val innerElementA = freshT("innerElementA") + val innerElementB = freshT("innerElementB") + + elementData.map { + case (tpe, accessorSymbol, tBuf) => + val curCmp = freshT("curCmp") + val cmpTree = q""" + val $curCmp: Int = { + val $innerElementA = $elementA.$accessorSymbol + val $innerElementB = $elementB.$accessorSymbol + ${tBuf.compare(innerElementA, innerElementB)} + } + """ + (cmpTree, curCmp) + } + .reverse // go through last to first + .foldLeft(None: Option[Tree]) { + case (Some(rest), (tree, valname)) => + Some( + q"""$tree; + if ($valname != 0) $valname + else { + $rest + } + """) + case (None, (tree, valname)) => Some(q"""$tree; $valname""") + } + .getOrElse(q"""0""") // all 0 size products are equal + } +} diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/TreeOrderedBuf.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/TreeOrderedBuf.scala new file mode 100644 index 0000000000..b2f4c5916a --- /dev/null +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/TreeOrderedBuf.scala @@ -0,0 +1,319 @@ +/* + Copyright 2014 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.macros.impl.ordered_serialization + +import com.twitter.scalding._ +import com.twitter.scalding.serialization.OrderedSerialization +import com.twitter.scalding.serialization.PositionInputStream +import java.io.InputStream +import scala.reflect.macros.Context +import scala.language.experimental.macros +import scala.util.control.NonFatal + +object CommonCompareBinary { + import com.twitter.scalding.serialization.JavaStreamEnrichments._ + + // If the lengths are equal and greater than this number + // we will compare on all the containing bytes + val minSizeForFulBinaryCompare = 24 + + /** + * This method will compare two InputStreams of given lengths + * If the inputsteam supports mark/reset (such as those backed by Array[Byte]), + * and the lengths are equal and longer than minSizeForFulBinaryCompare we first + * check if they are byte-for-byte identical, which is a cheap way to avoid doing + * potentially complex logic in innerCmp + * + * If the above fails to show them equal, we apply innerCmp. Note innerCmp does + * not need to seek each stream to the end of the records, that is handled by + * this method after innerCmp returns + */ + final def compareBinaryPrelude(inputStreamA: InputStream, + lenA: Int, + inputStreamB: InputStream, + lenB: Int)(innerCmp: (InputStream, InputStream) => Int) = { + try { + // First up validate the lengths passed make sense + require(lenA >= 0, "Length was " + lenA + "which is < 0, invalid") + require(lenB >= 0, "Length was " + lenB + "which is < 0, invalid") + + val earlyEqual: Boolean = if (lenA > minSizeForFulBinaryCompare && + (lenA == lenB) && + inputStreamA.markSupported && + inputStreamB.markSupported) { + inputStreamA.mark(lenA) + inputStreamB.mark(lenB) + + @annotation.tailrec + def arrayBytesSame(pos: Int): Boolean = + (pos >= lenA) || + ((inputStreamA.readByte == inputStreamB.readByte) && + arrayBytesSame(pos + 1)) + + arrayBytesSame(0) || { + // rewind if they don't match for doing the full compare + inputStreamA.reset() + inputStreamB.reset() + false + } + } else false + + val r = if (earlyEqual) { + 0 + } else { + val bufferedStreamA = PositionInputStream(inputStreamA) + val initialPositionA = bufferedStreamA.position + val bufferedStreamB = PositionInputStream(inputStreamB) + val initialPositionB = bufferedStreamB.position + + val innerR = innerCmp(bufferedStreamA, bufferedStreamB) + + bufferedStreamA.seekToPosition(initialPositionA + lenA) + bufferedStreamB.seekToPosition(initialPositionB + lenB) + innerR + } + + OrderedSerialization.resultFrom(r) + } catch { + case NonFatal(e) => OrderedSerialization.CompareFailure(e) + } + } + +} +object TreeOrderedBuf { + import CompileTimeLengthTypes._ + def toOrderedSerialization[T](c: Context)(t: TreeOrderedBuf[c.type])(implicit T: t.ctx.WeakTypeTag[T]): t.ctx.Expr[OrderedSerialization[T]] = { + import t.ctx.universe._ + def freshT(id: String) = newTermName(c.fresh(s"fresh_$id")) + val outputLength = freshT("outputLength") + + val innerLengthFn: Tree = { + val element = freshT("element") + + val fnBodyOpt = t.length(q"$element") match { + case _: NoLengthCalculationAvailable[_] => None + case const: ConstantLengthCalculation[_] => None + case f: FastLengthCalculation[_] => Some(q""" + _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.DynamicLen(${f.asInstanceOf[FastLengthCalculation[c.type]].t}) + """) + case m: MaybeLengthCalculation[_] => Some(m.asInstanceOf[MaybeLengthCalculation[c.type]].t) + } + + fnBodyOpt.map { fnBody => + q""" + @inline private[this] def payloadLength($element: $T): _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.MaybeLength = { + $fnBody + } + """ + }.getOrElse(q"()") + } + + def binaryLengthGen(typeName: Tree): (Tree, Tree) = { + val tempLen = freshT("tempLen") + val lensLen = freshT("lensLen") + val element = freshT("element") + val callDynamic = (q"""override def staticSize: Option[Int] = None""", + q""" + + override def dynamicSize($element: $typeName): Option[Int] = { + val $tempLen = payloadLength($element) match { + case _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.NoLengthCalculation => None + case _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.ConstLen(l) => Some(l) + case _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.DynamicLen(l) => Some(l) + } + (if ($tempLen.isDefined) { + // Avoid a closure here while we are geeking out + val innerLen = $tempLen.get + val $lensLen = sizeBytes(innerLen) + Some(innerLen + $lensLen) + } else None): Option[Int] + } + """) + t.length(q"$element") match { + case _: NoLengthCalculationAvailable[_] => (q""" + override def staticSize: Option[Int] = None""", q""" + override def dynamicSize($element: $typeName): Option[Int] = None""") + case const: ConstantLengthCalculation[_] => (q""" + override val staticSize: Option[Int] = Some(${const.toInt})""", q""" + override def dynamicSize($element: $typeName): Option[Int] = staticSize""") + case f: FastLengthCalculation[_] => callDynamic + case m: MaybeLengthCalculation[_] => callDynamic + } + } + + def putFnGen(outerbaos: TermName, element: TermName) = { + val baos = freshT("baos") + val len = freshT("len") + val oldPos = freshT("oldPos") + + /** + * This is the worst case: we have to serialize in a side buffer + * and then see how large it actually is. This happens for cases, like + * string, where the cost to see the serialized size is not cheaper than + * directly serializing. + */ + val noLenCalc = q""" + val $baos = new _root_.java.io.ByteArrayOutputStream + ${t.put(baos, element)} + val $len = $baos.size + $outerbaos.writeSize($len) + $baos.writeTo($outerbaos) + """ + + /** + * This is the case where the length is cheap to compute, either + * constant or easily computable from an instance. + */ + def withLenCalc(lenC: Tree) = q""" + val $len = $lenC + $outerbaos.writeSize($len) + ${t.put(outerbaos, element)} + """ + + t.length(q"$element") match { + case _: NoLengthCalculationAvailable[_] => noLenCalc + case _: ConstantLengthCalculation[_] => + q"""${t.put(outerbaos, element)}""" + case f: FastLengthCalculation[_] => + withLenCalc(f.asInstanceOf[FastLengthCalculation[c.type]].t) + case m: MaybeLengthCalculation[_] => + val tmpLenRes = freshT("tmpLenRes") + q""" + @inline def noLenCalc = { + $noLenCalc + } + @inline def withLenCalc(cnt: Int) = { + ${withLenCalc(q"cnt")} + } + val $tmpLenRes: _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.MaybeLength = payloadLength($element) + $tmpLenRes match { + case _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.NoLengthCalculation => noLenCalc + case _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.ConstLen(const) => withLenCalc(const) + case _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.DynamicLen(s) => withLenCalc(s) + } + """ + } + } + + def readLength(inputStream: TermName) = { + t.length(q"e") match { + case const: ConstantLengthCalculation[_] => q"${const.toInt}" + case _ => q"$inputStream.readSize" + } + } + + def discardLength(inputStream: TermName) = { + t.length(q"e") match { + case const: ConstantLengthCalculation[_] => q"()" + case _ => q"$inputStream.readSize" + } + } + + val lazyVariables = t.lazyOuterVariables.map { + case (n, t) => + val termName = newTermName(n) + q"""lazy val $termName = $t""" + } + + val element = freshT("element") + + val inputStreamA = freshT("inputStreamA") + val inputStreamB = freshT("inputStreamB") + + val lenA = freshT("lenA") + val lenB = freshT("lenB") + + t.ctx.Expr[OrderedSerialization[T]](q""" + new _root_.com.twitter.scalding.serialization.OrderedSerialization[$T] with _root_.com.twitter.bijection.macros.MacroGenerated { + import _root_.com.twitter.scalding.serialization.JavaStreamEnrichments._ + ..$lazyVariables + + private[this] val innerBinaryCompare = { ($inputStreamA: _root_.java.io.InputStream, $inputStreamB: _root_.java.io.InputStream) => + ${t.compareBinary(inputStreamA, inputStreamB)} + } + + override def compareBinary($inputStreamA: _root_.java.io.InputStream, $inputStreamB: _root_.java.io.InputStream): _root_.com.twitter.scalding.serialization.OrderedSerialization.Result = { + + val $lenA = ${readLength(inputStreamA)} + val $lenB = ${readLength(inputStreamB)} + + com.twitter.scalding.macros.impl.ordered_serialization.CommonCompareBinary.compareBinaryPrelude($inputStreamA, + $lenA, + $inputStreamB, + $lenB)(innerBinaryCompare) + } + + override def hash(passedInObjectToHash: $T): Int = { + ${t.hash(newTermName("passedInObjectToHash"))} + } + + // defines payloadLength private method + $innerLengthFn + + // static size: + ${binaryLengthGen(q"$T")._1} + + // dynamic size: + ${binaryLengthGen(q"$T")._2} + + override def read(from: _root_.java.io.InputStream): _root_.scala.util.Try[$T] = { + try { + ${discardLength(newTermName("from"))} + _root_.scala.util.Success(${t.get(newTermName("from"))}) + } catch { case _root_.scala.util.control.NonFatal(e) => + _root_.scala.util.Failure(e) + } + } + + override def write(into: _root_.java.io.OutputStream, e: $T): _root_.scala.util.Try[Unit] = { + try { + ${putFnGen(newTermName("into"), newTermName("e"))} + _root_.com.twitter.scalding.serialization.Serialization.successUnit + } catch { case _root_.scala.util.control.NonFatal(e) => + _root_.scala.util.Failure(e) + } + } + + override def compare(x: $T, y: $T): Int = { + ${t.compare(newTermName("x"), newTermName("y"))} + } + } + """) + } +} + +abstract class TreeOrderedBuf[C <: Context] { + val ctx: C + val tpe: ctx.Type + // Expected byte buffers to be in values a and b respestively, the tree has the value of the result + def compareBinary(inputStreamA: ctx.TermName, inputStreamB: ctx.TermName): ctx.Tree + // expects the thing to be tested on in the indiciated TermName + def hash(element: ctx.TermName): ctx.Tree + + // Place input in param 1, tree to return result in param 2 + def get(inputStreamA: ctx.TermName): ctx.Tree + + // BB input in param 1 + // Other input of type T in param 2 + def put(inputStream: ctx.TermName, element: ctx.TermName): ctx.Tree + + def compare(elementA: ctx.TermName, elementB: ctx.TermName): ctx.Tree + + def lazyOuterVariables: Map[String, ctx.Tree] + // Return the constant size or a tree + def length(element: ctx.universe.Tree): CompileTimeLengthTypes[ctx.type] + +} diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/BooleanOrderedBuf.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/BooleanOrderedBuf.scala new file mode 100644 index 0000000000..b237f27c4f --- /dev/null +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/BooleanOrderedBuf.scala @@ -0,0 +1,64 @@ +/* + Copyright 2014 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.macros.impl.ordered_serialization.providers + +import scala.language.experimental.macros +import scala.reflect.macros.Context + +import com.twitter.scalding._ + +import com.twitter.scalding.macros.impl.ordered_serialization.{ CompileTimeLengthTypes, ProductLike, TreeOrderedBuf } +import CompileTimeLengthTypes._ + +import java.nio.ByteBuffer +import com.twitter.scalding.serialization.OrderedSerialization + +object BooleanOrderedBuf { + def dispatch(c: Context): PartialFunction[c.Type, TreeOrderedBuf[c.type]] = { + case tpe if tpe =:= c.universe.typeOf[Boolean] => BooleanOrderedBuf(c)(tpe) + } + + def apply(c: Context)(outerType: c.Type): TreeOrderedBuf[c.type] = { + import c.universe._ + + new TreeOrderedBuf[c.type] { + override val ctx: c.type = c + override val tpe = outerType + + override def compareBinary(inputStreamA: ctx.TermName, inputStreamB: ctx.TermName) = + q"_root_.java.lang.Byte.compare($inputStreamA.readByte, $inputStreamB.readByte)" + + override def hash(element: ctx.TermName): ctx.Tree = + q"_root_.com.twitter.scalding.serialization.Hasher.boolean.hash($element)" + + override def put(inputStream: ctx.TermName, element: ctx.TermName) = + q"$inputStream.writeByte(if($element) (1: Byte) else (0: Byte))" + + override def get(inputStreamA: ctx.TermName): ctx.Tree = + q"($inputStreamA.readByte == (1: Byte))" + + def compare(elementA: ctx.TermName, elementB: ctx.TermName): ctx.Tree = + q"_root_.java.lang.Boolean.compare($elementA, $elementB)" + + override def length(element: Tree): CompileTimeLengthTypes[c.type] = + ConstantLengthCalculation(c)(1) + + override val lazyOuterVariables: Map[String, ctx.Tree] = + Map.empty + } + } +} + diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/ByteBufferOrderedBuf.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/ByteBufferOrderedBuf.scala new file mode 100644 index 0000000000..5d52749915 --- /dev/null +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/ByteBufferOrderedBuf.scala @@ -0,0 +1,99 @@ +/* + Copyright 2014 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.macros.impl.ordered_serialization.providers + +import scala.language.experimental.macros +import scala.reflect.macros.Context + +import com.twitter.scalding._ +import com.twitter.scalding.macros.impl.ordered_serialization.{ CompileTimeLengthTypes, ProductLike, TreeOrderedBuf } +import CompileTimeLengthTypes._ + +import java.nio.ByteBuffer +import com.twitter.scalding.serialization.OrderedSerialization + +object ByteBufferOrderedBuf { + def dispatch(c: Context): PartialFunction[c.Type, TreeOrderedBuf[c.type]] = { + case tpe if tpe =:= c.universe.typeOf[ByteBuffer] => ByteBufferOrderedBuf(c)(tpe) + } + + def apply(c: Context)(outerType: c.Type): TreeOrderedBuf[c.type] = { + import c.universe._ + + def freshT(id: String) = newTermName(c.fresh(id)) + + new TreeOrderedBuf[c.type] { + override val ctx: c.type = c + override val tpe = outerType + override def hash(element: ctx.TermName): ctx.Tree = q"$element.hashCode" + + override def compareBinary(inputStreamA: ctx.TermName, inputStreamB: ctx.TermName) = { + val lenA = freshT("lenA") + val lenB = freshT("lenB") + val queryLength = freshT("queryLength") + val incr = freshT("incr") + val state = freshT("state") + q""" + val $lenA: Int = $inputStreamA.readSize + val $lenB: Int = $inputStreamB.readSize + + val $queryLength = _root_.scala.math.min($lenA, $lenB) + var $incr = 0 + var $state = 0 + + while($incr < $queryLength && $state == 0) { + $state = _root_.java.lang.Byte.compare($inputStreamA.readByte, $inputStreamB.readByte) + $incr = $incr + 1 + } + if($state == 0) { + _root_.java.lang.Integer.compare($lenA, $lenB) + } else { + $state + } + """ + } + override def put(inputStream: ctx.TermName, element: ctx.TermName) = + q""" + $inputStream.writeSize($element.remaining) + $inputStream.writeBytes($element.array, $element.arrayOffset + $element.position, $element.remaining) + """ + + override def get(inputStream: ctx.TermName): ctx.Tree = { + val lenA = freshT("lenA") + val bytes = freshT("bytes") + q""" + val $lenA = $inputStream.readSize + val $bytes = new Array[Byte]($lenA) + $inputStream.readFully($bytes) + _root_.java.nio.ByteBuffer.wrap($bytes) + """ + } + override def compare(elementA: ctx.TermName, elementB: ctx.TermName): ctx.Tree = q""" + $elementA.compareTo($elementB) + """ + override def length(element: Tree): CompileTimeLengthTypes[c.type] = { + val tmpLen = freshT("tmpLen") + FastLengthCalculation(c)(q""" + val $tmpLen = $element.remaining + sizeBytes($tmpLen) + $tmpLen + """) + } + + def lazyOuterVariables: Map[String, ctx.Tree] = Map.empty + } + } +} + diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/CaseClassOrderedBuf.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/CaseClassOrderedBuf.scala new file mode 100644 index 0000000000..d286e854b2 --- /dev/null +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/CaseClassOrderedBuf.scala @@ -0,0 +1,87 @@ +/* + Copyright 2014 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.macros.impl.ordered_serialization.providers + +import scala.language.experimental.macros +import scala.reflect.macros.Context + +import com.twitter.scalding._ +import com.twitter.scalding.macros.impl.ordered_serialization.{ CompileTimeLengthTypes, ProductLike, TreeOrderedBuf } +import CompileTimeLengthTypes._ +import com.twitter.scalding.serialization.OrderedSerialization +import com.twitter.bijection.macros.impl.IsCaseClassImpl + +object CaseClassOrderedBuf { + def dispatch(c: Context)(buildDispatcher: => PartialFunction[c.Type, TreeOrderedBuf[c.type]]): PartialFunction[c.Type, TreeOrderedBuf[c.type]] = { + case tpe if tpe.typeSymbol.isClass && tpe.typeSymbol.asClass.isCaseClass && !tpe.typeConstructor.takesTypeArgs => + CaseClassOrderedBuf(c)(buildDispatcher, tpe) + } + + def apply(c: Context)(buildDispatcher: => PartialFunction[c.Type, TreeOrderedBuf[c.type]], outerType: c.Type): TreeOrderedBuf[c.type] = { + import c.universe._ + def freshT(id: String) = newTermName(c.fresh(id)) + + val dispatcher = buildDispatcher + val elementData: List[(c.universe.Type, TermName, TreeOrderedBuf[c.type])] = + outerType + .declarations + .collect { case m: MethodSymbol if m.isCaseAccessor => m } + .map { accessorMethod => + val fieldType = accessorMethod.returnType + val b: TreeOrderedBuf[c.type] = dispatcher(fieldType) + (fieldType, accessorMethod.name.toTermName, b) + }.toList + + new TreeOrderedBuf[c.type] { + override val ctx: c.type = c + override val tpe = outerType + override def compareBinary(inputStreamA: ctx.TermName, inputStreamB: ctx.TermName) = + ProductLike.compareBinary(c)(inputStreamA, inputStreamB)(elementData) + + override def hash(element: ctx.TermName): ctx.Tree = ProductLike.hash(c)(element)(elementData) + + override def put(inputStream: ctx.TermName, element: ctx.TermName) = + ProductLike.put(c)(inputStream, element)(elementData) + + override def get(inputStream: ctx.TermName): ctx.Tree = { + + val getValProcessor = elementData.map { + case (tpe, accessorSymbol, tBuf) => + val curR = freshT("curR") + val builderTree = q""" + val $curR = { + ${tBuf.get(inputStream)} + } + """ + (builderTree, curR) + } + q""" + ..${getValProcessor.map(_._1)} + ${outerType.typeSymbol.companionSymbol}(..${getValProcessor.map(_._2)}) + """ + } + override def compare(elementA: ctx.TermName, elementB: ctx.TermName): ctx.Tree = + ProductLike.compare(c)(elementA, elementB)(elementData) + + override val lazyOuterVariables: Map[String, ctx.Tree] = + elementData.map(_._3.lazyOuterVariables).reduce(_ ++ _) + + override def length(element: Tree) = + ProductLike.length(c)(element)(elementData) + } + } +} + diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/EitherOrderedBuf.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/EitherOrderedBuf.scala new file mode 100644 index 0000000000..aff24ebbb1 --- /dev/null +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/EitherOrderedBuf.scala @@ -0,0 +1,180 @@ +/* + Copyright 2015 Twitter, Inc. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + */ +package com.twitter.scalding.macros.impl.ordered_serialization.providers + +import scala.language.experimental.macros +import scala.reflect.macros.Context + +import com.twitter.scalding._ +import com.twitter.scalding.macros.impl.ordered_serialization.{ CompileTimeLengthTypes, ProductLike, TreeOrderedBuf } +import CompileTimeLengthTypes._ +import com.twitter.scalding.serialization.OrderedSerialization + +object EitherOrderedBuf { + def dispatch(c: Context)(buildDispatcher: => PartialFunction[c.Type, TreeOrderedBuf[c.type]]): PartialFunction[c.Type, TreeOrderedBuf[c.type]] = { + case tpe if tpe.erasure =:= c.universe.typeOf[Either[Any, Any]] => EitherOrderedBuf(c)(buildDispatcher, tpe) + } + + def apply(c: Context)(buildDispatcher: => PartialFunction[c.Type, TreeOrderedBuf[c.type]], outerType: c.Type): TreeOrderedBuf[c.type] = { + import c.universe._ + def freshT(id: String) = newTermName(c.fresh(id)) + val dispatcher = buildDispatcher + + val leftType = outerType.asInstanceOf[TypeRefApi].args(0) + val rightType = outerType.asInstanceOf[TypeRefApi].args(1) + val leftBuf: TreeOrderedBuf[c.type] = dispatcher(leftType) + val rightBuf: TreeOrderedBuf[c.type] = dispatcher(rightType) + + def genBinaryCompare(inputStreamA: TermName, inputStreamB: TermName) = { + val valueOfA = freshT("valueOfA") + val valueOfB = freshT("valueOfB") + val tmpHolder = freshT("tmpHolder") + q""" + val $valueOfA = $inputStreamA.readByte + val $valueOfB = $inputStreamB.readByte + val $tmpHolder = _root_.java.lang.Byte.compare($valueOfA, $valueOfB) + if($tmpHolder != 0) { + //they are different, return comparison on type + $tmpHolder + } else if($valueOfA == (0: _root_.scala.Byte)) { + // they are both Left: + ${leftBuf.compareBinary(inputStreamA, inputStreamB)} + } else { + // they are both Right: + ${rightBuf.compareBinary(inputStreamA, inputStreamB)} + } + """ + } + + def genHashFn(element: TermName) = { + val innerValue = freshT("innerValue") + q""" + if($element.isLeft) { + val $innerValue = $element.left.get + val x = ${leftBuf.hash(innerValue)} + // x * (2^31 - 1) which is a mersenne prime + (x << 31) - x + } + else { + val $innerValue = $element.right.get + // x * (2^19 - 1) which is a mersenne prime + val x = ${rightBuf.hash(innerValue)} + (x << 19) - x + } + """ + } + + def genGetFn(inputStreamA: TermName) = { + val tmpGetHolder = freshT("tmpGetHolder") + q""" + val $tmpGetHolder = $inputStreamA.readByte + if($tmpGetHolder == (0: _root_.scala.Byte)) Left(${leftBuf.get(inputStreamA)}) + else Right(${rightBuf.get(inputStreamA)}) + """ + } + + def genPutFn(inputStream: TermName, element: TermName) = { + val tmpPutVal = freshT("tmpPutVal") + val innerValue = freshT("innerValue") + q""" + if($element.isRight) { + $inputStream.writeByte(1: _root_.scala.Byte) + val $innerValue = $element.right.get + ${rightBuf.put(inputStream, innerValue)} + } else { + $inputStream.writeByte(0: _root_.scala.Byte) + val $innerValue = $element.left.get + ${leftBuf.put(inputStream, innerValue)} + } + """ + } + + def genCompareFn(elementA: TermName, elementB: TermName) = { + val aIsRight = freshT("aIsRight") + val bIsRight = freshT("bIsRight") + val innerValueA = freshT("innerValueA") + val innerValueB = freshT("innerValueB") + q""" + val $aIsRight = $elementA.isRight + val $bIsRight = $elementB.isRight + if(!$aIsRight) { + if (!$bIsRight) { + val $innerValueA = $elementA.left.get + val $innerValueB = $elementB.left.get + ${leftBuf.compare(innerValueA, innerValueB)} + } + else -1 // Left(_) < Right(_) + } + else { + if(!$bIsRight) 1 // Right(_) > Left(_) + else { // both are right + val $innerValueA = $elementA.right.get + val $innerValueB = $elementB.right.get + ${rightBuf.compare(innerValueA, innerValueB)} + } + } + """ + } + + new TreeOrderedBuf[c.type] { + override val ctx: c.type = c + override val tpe = outerType + override def compareBinary(inputStreamA: TermName, inputStreamB: TermName) = genBinaryCompare(inputStreamA, inputStreamB) + override def hash(element: TermName): ctx.Tree = genHashFn(element) + override def put(inputStream: TermName, element: TermName) = genPutFn(inputStream, element) + override def get(inputStreamA: TermName): ctx.Tree = genGetFn(inputStreamA) + override def compare(elementA: TermName, elementB: TermName): ctx.Tree = genCompareFn(elementA, elementB) + override val lazyOuterVariables: Map[String, ctx.Tree] = + rightBuf.lazyOuterVariables ++ leftBuf.lazyOuterVariables + override def length(element: Tree): CompileTimeLengthTypes[c.type] = { + + def tree(ctl: CompileTimeLengthTypes[_]): c.Tree = ctl.asInstanceOf[CompileTimeLengthTypes[c.type]].t + val dyn = q"""_root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.DynamicLen""" + + (leftBuf.length(q"$element.left.get"), rightBuf.length(q"$element.right.get")) match { + case (lconst: ConstantLengthCalculation[_], rconst: ConstantLengthCalculation[_]) if lconst.toInt == rconst.toInt => + // We got lucky, they are the same size: + ConstantLengthCalculation(c)(1 + rconst.toInt) + case (_: NoLengthCalculationAvailable[_], _) => NoLengthCalculationAvailable(c) + case (_, _: NoLengthCalculationAvailable[_]) => NoLengthCalculationAvailable(c) + case (left: MaybeLengthCalculation[_], right: MaybeLengthCalculation[_]) => + MaybeLengthCalculation(c)(q""" + if ($element.isLeft) { ${tree(left)} + $dyn(1) } + else { ${tree(right)} + $dyn(1) } + """) + case (left: MaybeLengthCalculation[_], right) => + MaybeLengthCalculation(c)(q""" + if ($element.isLeft) { ${tree(left)} + $dyn(1) } + else { $dyn(${tree(right)}) + $dyn(1) } + """) + case (left, right: MaybeLengthCalculation[_]) => + MaybeLengthCalculation(c)(q""" + if ($element.isLeft) { $dyn(${tree(left)}) + $dyn(1) } + else { ${tree(right)} + $dyn(1) } + """) + // Rest are constant, but different values or fast. So the result is fast + case (left, right) => + // They are different sizes. :( + FastLengthCalculation(c)(q""" + if($element.isLeft) { 1 + ${tree(left)} } + else { 1 + ${tree(right)} } + """) + } + } + } + } +} + diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/ImplicitOrderedBuf.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/ImplicitOrderedBuf.scala new file mode 100644 index 0000000000..2f40fb3b98 --- /dev/null +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/ImplicitOrderedBuf.scala @@ -0,0 +1,86 @@ +/* + Copyright 2014 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.macros.impl.ordered_serialization.providers + +import scala.language.experimental.macros +import scala.reflect.macros.Context + +import com.twitter.scalding._ +import com.twitter.scalding.serialization.OrderedSerialization +import com.twitter.scalding.macros.impl.ordered_serialization._ + +/* + A fall back ordered bufferable to look for the user to have an implicit in scope to satisfy the missing + type. This is for the case where its an opaque class to our macros where we can't figure out the fields +*/ +object ImplicitOrderedBuf { + val macroMarker = "MACROASKEDORDEREDSER" + + def dispatch(c: Context): PartialFunction[c.Type, TreeOrderedBuf[c.type]] = { + import c.universe._ + + val pf: PartialFunction[c.Type, TreeOrderedBuf[c.type]] = { + case tpe if !tpe.toString.contains(macroMarker) => ImplicitOrderedBuf(c)(tpe) + } + pf + } + + def apply(c: Context)(outerType: c.Type): TreeOrderedBuf[c.type] = { + import c.universe._ + def freshT(id: String) = newTermName(c.fresh(id)) + + val variableID = (outerType.typeSymbol.fullName.hashCode.toLong + Int.MaxValue.toLong).toString + val variableNameStr = s"orderedSer_$variableID" + val variableName = newTermName(variableNameStr) + val typeAlias = newTypeName(c.fresh("MACROASKEDORDEREDSER")) + val implicitInstanciator = q""" + type $typeAlias = $outerType + implicitly[_root_.com.twitter.scalding.serialization.OrderedSerialization[$typeAlias]]""" + + new TreeOrderedBuf[c.type] { + override val ctx: c.type = c + override val tpe = outerType + override def compareBinary(inputStreamA: ctx.TermName, inputStreamB: ctx.TermName) = + q"$variableName.compareBinary($inputStreamA, $inputStreamB).unsafeToInt" + override def hash(element: ctx.TermName): ctx.Tree = q"$variableName.hash($element)" + + override def put(inputStream: ctx.TermName, element: ctx.TermName) = + q"$variableName.write($inputStream, $element)" + + override def length(element: Tree) = + CompileTimeLengthTypes.MaybeLengthCalculation(c)(q""" + ($variableName.staticSize match { + case Some(s) => _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.ConstLen(s) + case None => + $variableName.dynamicSize($element) match { + case Some(s) => + _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.DynamicLen(s) + case None => + _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.NoLengthCalculation + } + }): _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.MaybeLength + """) + + override def get(inputStream: ctx.TermName): ctx.Tree = + q"$variableName.read($inputStream).get" + + override def compare(elementA: ctx.TermName, elementB: ctx.TermName): ctx.Tree = + q"$variableName.compare($elementA, $elementB)" + override val lazyOuterVariables = Map(variableNameStr -> implicitInstanciator) + } + } +} + diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/OptionOrderedBuf.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/OptionOrderedBuf.scala new file mode 100644 index 0000000000..29c2ed8a5d --- /dev/null +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/OptionOrderedBuf.scala @@ -0,0 +1,148 @@ +/* + Copyright 2014 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.macros.impl.ordered_serialization.providers + +import scala.language.experimental.macros +import scala.reflect.macros.Context + +import com.twitter.scalding._ +import com.twitter.scalding.macros.impl.ordered_serialization.{ CompileTimeLengthTypes, ProductLike, TreeOrderedBuf } +import CompileTimeLengthTypes._ +import com.twitter.scalding.serialization.OrderedSerialization + +object OptionOrderedBuf { + def dispatch(c: Context)(buildDispatcher: => PartialFunction[c.Type, TreeOrderedBuf[c.type]]): PartialFunction[c.Type, TreeOrderedBuf[c.type]] = { + case tpe if tpe.erasure =:= c.universe.typeOf[Option[Any]] => OptionOrderedBuf(c)(buildDispatcher, tpe) + } + + def apply(c: Context)(buildDispatcher: => PartialFunction[c.Type, TreeOrderedBuf[c.type]], outerType: c.Type): TreeOrderedBuf[c.type] = { + import c.universe._ + def freshT(id: String) = newTermName(c.fresh(id)) + val dispatcher = buildDispatcher + + val innerType = outerType.asInstanceOf[TypeRefApi].args.head + val innerBuf: TreeOrderedBuf[c.type] = dispatcher(innerType) + + def genBinaryCompare(inputStreamA: TermName, inputStreamB: TermName) = { + val valueOfA = freshT("valueOfA") + val valueOfB = freshT("valueOfB") + val tmpHolder = freshT("tmpHolder") + q""" + val $valueOfA = $inputStreamA.readByte + val $valueOfB = $inputStreamB.readByte + val $tmpHolder = _root_.java.lang.Byte.compare($valueOfA, $valueOfB) + if($tmpHolder != 0 || $valueOfA == (0: _root_.scala.Byte)) { + //either one is defined (different), or both are None (equal) + $tmpHolder + } else { + ${innerBuf.compareBinary(inputStreamA, inputStreamB)} + } + """ + } + + def genHashFn(element: TermName) = { + val innerValue = freshT("innerValue") + q""" + if($element.isEmpty) + 0 + else { + val $innerValue = $element.get + ${innerBuf.hash(innerValue)} + } + """ + } + + def genGetFn(inputStreamA: TermName) = { + val tmpGetHolder = freshT("tmpGetHolder") + q""" + val $tmpGetHolder = $inputStreamA.readByte + if($tmpGetHolder == (0: _root_.scala.Byte)) None + else Some(${innerBuf.get(inputStreamA)}) + """ + } + + def genPutFn(inputStream: TermName, element: TermName) = { + val tmpPutVal = freshT("tmpPutVal") + val innerValue = freshT("innerValue") + q""" + if($element.isDefined) { + $inputStream.writeByte(1: _root_.scala.Byte) + val $innerValue = $element.get + ${innerBuf.put(inputStream, innerValue)} + } else { + $inputStream.writeByte(0: _root_.scala.Byte) + } + """ + } + + def genCompareFn(elementA: TermName, elementB: TermName) = { + val aIsDefined = freshT("aIsDefined") + val bIsDefined = freshT("bIsDefined") + val innerValueA = freshT("innerValueA") + val innerValueB = freshT("innerValueB") + q""" + val $aIsDefined = $elementA.isDefined + val $bIsDefined = $elementB.isDefined + if(!$aIsDefined) { + if (!$bIsDefined) 0 // None == None + else -1 // None < Some(_) + } + else { + if(!$bIsDefined) 1 // Some > None + else { // both are defined + val $innerValueA = $elementA.get + val $innerValueB = $elementB.get + ${innerBuf.compare(innerValueA, innerValueB)} + } + } + """ + } + + new TreeOrderedBuf[c.type] { + override val ctx: c.type = c + override val tpe = outerType + override def compareBinary(inputStreamA: TermName, inputStreamB: TermName) = genBinaryCompare(inputStreamA, inputStreamB) + override def hash(element: TermName): ctx.Tree = genHashFn(element) + override def put(inputStream: TermName, element: TermName) = genPutFn(inputStream, element) + override def get(inputStreamA: TermName): ctx.Tree = genGetFn(inputStreamA) + override def compare(elementA: TermName, elementB: TermName): ctx.Tree = genCompareFn(elementA, elementB) + override val lazyOuterVariables: Map[String, ctx.Tree] = innerBuf.lazyOuterVariables + override def length(element: Tree): CompileTimeLengthTypes[c.type] = { + innerBuf.length(q"$element.get") match { + case const: ConstantLengthCalculation[_] => FastLengthCalculation(c)(q""" + if($element.isDefined) { 1 + ${const.toInt} } + else { 1 } + """) + case f: FastLengthCalculation[_] => + val t = f.asInstanceOf[FastLengthCalculation[c.type]].t + FastLengthCalculation(c)(q""" + if($element.isDefined) { 1 + $t } + else { 1 } + """) + case m: MaybeLengthCalculation[_] => + val t = m.asInstanceOf[MaybeLengthCalculation[c.type]].t + val dynlen = q"""_root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.DynamicLen""" + MaybeLengthCalculation(c)(q""" + if ($element.isDefined) { $t + $dynlen(1) } + else { $dynlen(1) } + """) + case _ => NoLengthCalculationAvailable(c) + } + } + } + } +} + diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/PrimitiveOrderedBuf.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/PrimitiveOrderedBuf.scala new file mode 100644 index 0000000000..6a04ab6149 --- /dev/null +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/PrimitiveOrderedBuf.scala @@ -0,0 +1,114 @@ +/* + Copyright 2014 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.macros.impl.ordered_serialization.providers + +import scala.language.experimental.macros +import scala.reflect.macros.Context + +import com.twitter.scalding._ +import com.twitter.scalding.macros.impl.ordered_serialization.{ CompileTimeLengthTypes, ProductLike, TreeOrderedBuf } +import CompileTimeLengthTypes._ +import java.nio.ByteBuffer +import com.twitter.scalding.serialization.OrderedSerialization + +object PrimitiveOrderedBuf { + def dispatch(c: Context): PartialFunction[c.Type, TreeOrderedBuf[c.type]] = { + case tpe if tpe =:= c.universe.typeOf[Byte] => PrimitiveOrderedBuf(c)(tpe, "Byte", "readByte", "writeByte", 1) + case tpe if tpe =:= c.universe.typeOf[Short] => PrimitiveOrderedBuf(c)(tpe, "Short", "readShort", "writeShort", 2) + case tpe if tpe =:= c.universe.typeOf[Int] => PrimitiveOrderedBuf(c)(tpe, "Integer", "readInt", "writeInt", 4) + case tpe if tpe =:= c.universe.typeOf[Long] => PrimitiveOrderedBuf(c)(tpe, "Long", "readLong", "writeLong", 8) + case tpe if tpe =:= c.universe.typeOf[Float] => PrimitiveOrderedBuf(c)(tpe, "Float", "readFloat", "writeFloat", 4) + case tpe if tpe =:= c.universe.typeOf[Double] => PrimitiveOrderedBuf(c)(tpe, "Double", "readDouble", "writeDouble", 8) + } + + def apply(c: Context)(outerType: c.Type, javaTypeStr: String, bbGetterStr: String, bbPutterStr: String, lenInBytes: Int): TreeOrderedBuf[c.type] = { + import c.universe._ + val bbGetter = newTermName(bbGetterStr) + val bbPutter = newTermName(bbPutterStr) + val javaType = newTermName(javaTypeStr) + + def freshT(id: String = "Product") = newTermName(c.fresh(s"fresh_$id")) + + def genBinaryCompare(inputStreamA: TermName, inputStreamB: TermName): Tree = + if (Set("Float", "Double").contains(javaTypeStr)) { + // These cannot be compared using byte-wise approach + q"""_root_.java.lang.$javaType.compare($inputStreamA.$bbGetter, $inputStreamB.$bbGetter)""" + } else { + // Big endian numbers can be compared byte by byte + (0 until lenInBytes).map { i => + if (i == 0) { + //signed for the first byte + q"""_root_.java.lang.Byte.compare($inputStreamA.readByte, $inputStreamB.readByte)""" + } else { + q"""_root_.java.lang.Integer.compare($inputStreamA.readUnsignedByte, $inputStreamB.readUnsignedByte)""" + } + } + .toList + .reverse // go through last to first + .foldLeft(None: Option[Tree]) { + case (Some(rest), next) => + val nextV = newTermName("next") + Some( + q"""val $nextV = $next + if ($nextV != 0) $nextV + else { + $rest + } + """) + case (None, next) => Some(q"""$next""") + }.get // there must be at least one item because no primitive is zero bytes + } + + def genCompareFn(compareInputA: TermName, compareInputB: TermName): Tree = { + val clamp = Set("Byte", "Short").contains(javaTypeStr) + val compareFn = if (clamp) { + val cmpTmpVal = freshT("cmpTmpVal") + + q""" + val $cmpTmpVal = _root_.java.lang.$javaType.compare($compareInputA, $compareInputB) + if($cmpTmpVal < 0) { + -1 + } else if($cmpTmpVal > 0) { + 1 + } else { + 0 + } + """ + } else { + q""" + _root_.java.lang.$javaType.compare($compareInputA, $compareInputB) + """ + } + compareFn + } + + // used in the hasher + val typeLowerCase = newTermName(javaTypeStr.toLowerCase) + + new TreeOrderedBuf[c.type] { + override val ctx: c.type = c + override val tpe = outerType + override def compareBinary(inputStreamA: ctx.TermName, inputStreamB: ctx.TermName) = genBinaryCompare(inputStreamA, inputStreamB) + override def hash(element: ctx.TermName): ctx.Tree = q"_root_.com.twitter.scalding.serialization.Hasher.$typeLowerCase.hash($element)" + override def put(inputStream: ctx.TermName, element: ctx.TermName) = q"$inputStream.$bbPutter($element)" + override def get(inputStream: ctx.TermName): ctx.Tree = q"$inputStream.$bbGetter" + override def compare(elementA: ctx.TermName, elementB: ctx.TermName): ctx.Tree = genCompareFn(elementA, elementB) + override def length(element: Tree): CompileTimeLengthTypes[c.type] = ConstantLengthCalculation(c)(lenInBytes) + override val lazyOuterVariables: Map[String, ctx.Tree] = Map.empty + } + } +} + diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/ProductOrderedBuf.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/ProductOrderedBuf.scala new file mode 100644 index 0000000000..d58133cdd4 --- /dev/null +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/ProductOrderedBuf.scala @@ -0,0 +1,124 @@ +/* + Copyright 2014 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.macros.impl.ordered_serialization.providers + +import scala.language.experimental.macros +import scala.reflect.macros.Context + +import com.twitter.scalding._ +import com.twitter.scalding.macros.impl.ordered_serialization.{ CompileTimeLengthTypes, ProductLike, TreeOrderedBuf } +import CompileTimeLengthTypes._ +import java.nio.ByteBuffer +import com.twitter.scalding.serialization.OrderedSerialization +import com.twitter.bijection.macros.impl.IsCaseClassImpl + +object ProductOrderedBuf { + def dispatch(c: Context)(buildDispatcher: => PartialFunction[c.Type, TreeOrderedBuf[c.type]]): PartialFunction[c.Type, TreeOrderedBuf[c.type]] = { + import c.universe._ + val validTypes: List[Type] = List(typeOf[Product1[Any]], + typeOf[Product2[Any, Any]], + typeOf[Product3[Any, Any, Any]], + typeOf[Product4[Any, Any, Any, Any]], + typeOf[Product5[Any, Any, Any, Any, Any]], + typeOf[Product6[Any, Any, Any, Any, Any, Any]], + typeOf[Product7[Any, Any, Any, Any, Any, Any, Any]], + typeOf[Product8[Any, Any, Any, Any, Any, Any, Any, Any]], + typeOf[Product9[Any, Any, Any, Any, Any, Any, Any, Any, Any]], + typeOf[Product10[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]], + typeOf[Product11[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]], + typeOf[Product12[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]], + typeOf[Product13[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]], + typeOf[Product14[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]], + typeOf[Product15[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]], + typeOf[Product16[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]], + typeOf[Product17[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]], + typeOf[Product18[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]], + typeOf[Product19[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]], + typeOf[Product20[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]], + typeOf[Product21[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]], + typeOf[Product22[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]]) + + def validType(curType: Type): Boolean = { + validTypes.find{ t => curType <:< t }.isDefined + } + + def symbolFor(subType: Type): Type = { + val superType = validTypes.find{ t => subType.erasure <:< t }.get + subType.baseType(superType.typeSymbol) + } + + val pf: PartialFunction[c.Type, TreeOrderedBuf[c.type]] = { + case tpe if validType(tpe.erasure) => ProductOrderedBuf(c)(buildDispatcher, tpe, symbolFor(tpe)) + } + pf + } + + def apply(c: Context)(buildDispatcher: => PartialFunction[c.Type, TreeOrderedBuf[c.type]], originalType: c.Type, outerType: c.Type): TreeOrderedBuf[c.type] = { + import c.universe._ + def freshT(id: String) = newTermName(c.fresh(id)) + + val dispatcher = buildDispatcher + val elementData: List[(c.universe.Type, TermName, TreeOrderedBuf[c.type])] = + outerType + .declarations + .collect { case m: MethodSymbol => m } + .filter(m => m.name.toTermName.toString.startsWith("_")) + .map { accessorMethod => + val fieldType = accessorMethod.returnType.asSeenFrom(outerType, outerType.typeSymbol.asClass) + val b: TreeOrderedBuf[c.type] = dispatcher(fieldType) + (fieldType, accessorMethod.name.toTermName, b) + }.toList + + new TreeOrderedBuf[c.type] { + override val ctx: c.type = c + override val tpe = outerType + override def compareBinary(inputStreamA: ctx.TermName, inputStreamB: ctx.TermName) = + ProductLike.compareBinary(c)(inputStreamA, inputStreamB)(elementData) + + override def hash(element: ctx.TermName): ctx.Tree = ProductLike.hash(c)(element)(elementData) + + override def put(inputStream: ctx.TermName, element: ctx.TermName) = + ProductLike.put(c)(inputStream, element)(elementData) + + override def get(inputStream: ctx.TermName): ctx.Tree = { + + val getValProcessor = elementData.map { + case (tpe, accessorSymbol, tBuf) => + val curR = freshT("curR") + val builderTree = q""" + val $curR = { + ${tBuf.get(inputStream)} + } + """ + (builderTree, curR) + } + q""" + ..${getValProcessor.map(_._1)} + new ${originalType}(..${getValProcessor.map(_._2)}) + """ + } + override def compare(elementA: ctx.TermName, elementB: ctx.TermName): ctx.Tree = + ProductLike.compare(c)(elementA, elementB)(elementData) + + override val lazyOuterVariables: Map[String, ctx.Tree] = + elementData.map(_._3.lazyOuterVariables).reduce(_ ++ _) + + override def length(element: Tree) = + ProductLike.length(c)(element)(elementData) + } + } +} + diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/StringOrderedBuf.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/StringOrderedBuf.scala new file mode 100644 index 0000000000..ab549feba0 --- /dev/null +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/StringOrderedBuf.scala @@ -0,0 +1,96 @@ +/* + Copyright 2014 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.macros.impl.ordered_serialization.providers + +import scala.language.experimental.macros +import scala.reflect.macros.Context + +import com.twitter.scalding._ +import com.twitter.scalding.macros.impl.ordered_serialization.{ CompileTimeLengthTypes, ProductLike, TreeOrderedBuf } +import CompileTimeLengthTypes._ +import java.nio.ByteBuffer +import com.twitter.scalding.serialization.OrderedSerialization + +object StringOrderedBuf { + def dispatch(c: Context): PartialFunction[c.Type, TreeOrderedBuf[c.type]] = { + case tpe if tpe =:= c.universe.typeOf[String] => StringOrderedBuf(c)(tpe) + } + + def apply(c: Context)(outerType: c.Type): TreeOrderedBuf[c.type] = { + import c.universe._ + + def freshT(id: String) = newTermName(c.fresh(id)) + + new TreeOrderedBuf[c.type] { + override val ctx: c.type = c + override val tpe = outerType + override def compareBinary(inputStreamA: ctx.TermName, inputStreamB: ctx.TermName) = { + val lenA = freshT("lenA") + val lenB = freshT("lenB") + + q""" + val $lenA = $inputStreamA.readSize + val $lenB = $inputStreamB.readSize + _root_.com.twitter.scalding.serialization.StringOrderedSerialization.binaryIntCompare($lenA, + $inputStreamA, + $lenB, + $inputStreamB) + """ + } + + override def hash(element: ctx.TermName): ctx.Tree = q"_root_.com.twitter.scalding.serialization.Hasher.string.hash($element)" + + override def put(inputStream: ctx.TermName, element: ctx.TermName) = { + val bytes = freshT("bytes") + val len = freshT("len") + q""" + val $bytes = $element.getBytes("UTF-8") + val $len = $bytes.length + $inputStream.writeSize($len) + if($len > 0) { + $inputStream.write($bytes) + } + """ + } + override def get(inputStream: ctx.TermName): ctx.Tree = { + val len = freshT("len") + val strBytes = freshT("strBytes") + q""" + val $len = $inputStream.readSize + if($len > 0) { + val $strBytes = new Array[Byte]($len) + $inputStream.readFully($strBytes) + new String($strBytes, "UTF-8") + } else { + "" + } + """ + } + override def compare(elementA: ctx.TermName, elementB: ctx.TermName) = + q"""$elementA.compareTo($elementB)""" + + override val lazyOuterVariables: Map[String, ctx.Tree] = Map.empty + override def length(element: Tree): CompileTimeLengthTypes[c.type] = MaybeLengthCalculation(c)(q""" + if($element.isEmpty) { + _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.DynamicLen(1) + } else { + _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.NoLengthCalculation + } + """) + } + } +} + diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/TraversablesOrderedBuf.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/TraversablesOrderedBuf.scala new file mode 100644 index 0000000000..36e5b84daa --- /dev/null +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/TraversablesOrderedBuf.scala @@ -0,0 +1,303 @@ +/* + Copyright 2014 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.macros.impl.ordered_serialization.providers + +import scala.language.experimental.macros +import scala.reflect.macros.Context +import java.io.InputStream + +import com.twitter.scalding._ +import com.twitter.scalding.macros.impl.ordered_serialization.{ CompileTimeLengthTypes, ProductLike, TreeOrderedBuf } +import CompileTimeLengthTypes._ +import com.twitter.scalding.serialization.OrderedSerialization +import scala.reflect.ClassTag + +import scala.{ collection => sc } +import scala.collection.{ immutable => sci } + +sealed trait ShouldSort +case object DoSort extends ShouldSort +case object NoSort extends ShouldSort + +sealed trait MaybeArray +case object IsArray extends MaybeArray +case object NotArray extends MaybeArray + +object TraversablesOrderedBuf { + def dispatch(c: Context)(buildDispatcher: => PartialFunction[c.Type, TreeOrderedBuf[c.type]]): PartialFunction[c.Type, TreeOrderedBuf[c.type]] = { + case tpe if tpe.erasure =:= c.universe.typeOf[Iterable[Any]] => TraversablesOrderedBuf(c)(buildDispatcher, tpe, NoSort, NotArray) + case tpe if tpe.erasure =:= c.universe.typeOf[sci.Iterable[Any]] => TraversablesOrderedBuf(c)(buildDispatcher, tpe, NoSort, NotArray) + case tpe if tpe.erasure =:= c.universe.typeOf[List[Any]] => TraversablesOrderedBuf(c)(buildDispatcher, tpe, NoSort, NotArray) + case tpe if tpe.erasure =:= c.universe.typeOf[sci.List[Any]] => TraversablesOrderedBuf(c)(buildDispatcher, tpe, NoSort, NotArray) + case tpe if tpe.erasure =:= c.universe.typeOf[Seq[Any]] => TraversablesOrderedBuf(c)(buildDispatcher, tpe, NoSort, NotArray) + case tpe if tpe.erasure =:= c.universe.typeOf[sc.Seq[Any]] => TraversablesOrderedBuf(c)(buildDispatcher, tpe, NoSort, NotArray) + case tpe if tpe.erasure =:= c.universe.typeOf[sci.Seq[Any]] => TraversablesOrderedBuf(c)(buildDispatcher, tpe, NoSort, NotArray) + case tpe if tpe.erasure =:= c.universe.typeOf[Vector[Any]] => TraversablesOrderedBuf(c)(buildDispatcher, tpe, NoSort, NotArray) + case tpe if tpe.erasure =:= c.universe.typeOf[sci.Vector[Any]] => TraversablesOrderedBuf(c)(buildDispatcher, tpe, NoSort, NotArray) + case tpe if tpe.erasure =:= c.universe.typeOf[IndexedSeq[Any]] => TraversablesOrderedBuf(c)(buildDispatcher, tpe, NoSort, NotArray) + case tpe if tpe.erasure =:= c.universe.typeOf[sci.IndexedSeq[Any]] => TraversablesOrderedBuf(c)(buildDispatcher, tpe, NoSort, NotArray) + case tpe if tpe.erasure =:= c.universe.typeOf[sci.Queue[Any]] => TraversablesOrderedBuf(c)(buildDispatcher, tpe, NoSort, NotArray) + // Arrays are special in that the erasure doesn't do anything + case tpe if tpe.typeSymbol == c.universe.typeOf[Array[Any]].typeSymbol => TraversablesOrderedBuf(c)(buildDispatcher, tpe, NoSort, IsArray) + // The erasure of a non-covariant is Set[_], so we need that here for sets + case tpe if tpe.erasure =:= c.universe.typeOf[Set[Any]].erasure => TraversablesOrderedBuf(c)(buildDispatcher, tpe, DoSort, NotArray) + case tpe if tpe.erasure =:= c.universe.typeOf[sc.Set[Any]].erasure => TraversablesOrderedBuf(c)(buildDispatcher, tpe, DoSort, NotArray) + case tpe if tpe.erasure =:= c.universe.typeOf[sci.Set[Any]].erasure => TraversablesOrderedBuf(c)(buildDispatcher, tpe, DoSort, NotArray) + case tpe if tpe.erasure =:= c.universe.typeOf[sci.HashSet[Any]].erasure => TraversablesOrderedBuf(c)(buildDispatcher, tpe, DoSort, NotArray) + case tpe if tpe.erasure =:= c.universe.typeOf[sci.ListSet[Any]].erasure => TraversablesOrderedBuf(c)(buildDispatcher, tpe, DoSort, NotArray) + + case tpe if tpe.erasure =:= c.universe.typeOf[Map[Any, Any]].erasure => TraversablesOrderedBuf(c)(buildDispatcher, tpe, DoSort, NotArray) + case tpe if tpe.erasure =:= c.universe.typeOf[sc.Map[Any, Any]].erasure => TraversablesOrderedBuf(c)(buildDispatcher, tpe, DoSort, NotArray) + case tpe if tpe.erasure =:= c.universe.typeOf[sci.Map[Any, Any]].erasure => TraversablesOrderedBuf(c)(buildDispatcher, tpe, DoSort, NotArray) + case tpe if tpe.erasure =:= c.universe.typeOf[sci.HashMap[Any, Any]].erasure => TraversablesOrderedBuf(c)(buildDispatcher, tpe, DoSort, NotArray) + case tpe if tpe.erasure =:= c.universe.typeOf[sci.ListMap[Any, Any]].erasure => TraversablesOrderedBuf(c)(buildDispatcher, tpe, DoSort, NotArray) + } + + def apply(c: Context)(buildDispatcher: => PartialFunction[c.Type, TreeOrderedBuf[c.type]], + outerType: c.Type, + maybeSort: ShouldSort, + maybeArray: MaybeArray): TreeOrderedBuf[c.type] = { + + import c.universe._ + def freshT(id: String) = newTermName(c.fresh(s"fresh_$id")) + + val dispatcher = buildDispatcher + + val companionSymbol = outerType.typeSymbol.companionSymbol + + // When dealing with a map we have 2 type args, and need to generate the tuple type + // it would correspond to if we .toList the Map. + val innerType = if (outerType.asInstanceOf[TypeRefApi].args.size == 2) { + val (tpe1, tpe2) = (outerType.asInstanceOf[TypeRefApi].args(0), outerType.asInstanceOf[TypeRefApi].args(1)) + val containerType = typeOf[Tuple2[Any, Any]].asInstanceOf[TypeRef] + import compat._ + TypeRef.apply(containerType.pre, containerType.sym, List(tpe1, tpe2)) + } else { + outerType.asInstanceOf[TypeRefApi].args.head + } + + val innerTypes = outerType.asInstanceOf[TypeRefApi].args + + val innerBuf: TreeOrderedBuf[c.type] = dispatcher(innerType) + // TODO it would be nice to capture one instance of this rather + // than allocate in every call in the materialized class + val ioa = freshT("ioa") + val iob = freshT("iob") + val innerOrd = q""" + new _root_.scala.math.Ordering[${innerBuf.tpe}] { + def compare(a: ${innerBuf.tpe}, b: ${innerBuf.tpe}) = { + val $ioa = a + val $iob = b + ${innerBuf.compare(ioa, iob)} + } + } + """ + + new TreeOrderedBuf[c.type] { + override val ctx: c.type = c + override val tpe = outerType + override def compareBinary(inputStreamA: ctx.TermName, inputStreamB: ctx.TermName) = { + val innerCompareFn = freshT("innerCompareFn") + val a = freshT("a") + val b = freshT("b") + q""" + val $innerCompareFn = { (a: _root_.java.io.InputStream, b: _root_.java.io.InputStream) => + val $a = a + val $b = b + ${innerBuf.compareBinary(a, b)} + }; + _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.TraversableHelpers.rawCompare($inputStreamA, $inputStreamB)($innerCompareFn) + """ + } + + override def put(inputStream: ctx.TermName, element: ctx.TermName) = { + val asArray = freshT("asArray") + val bytes = freshT("bytes") + val len = freshT("len") + val pos = freshT("pos") + val innerElement = freshT("innerElement") + val cmpRes = freshT("cmpRes") + + maybeSort match { + case DoSort => + q""" + val $len = $element.size + $inputStream.writeSize($len) + + if($len > 0) { + val $asArray = $element.toArray[${innerBuf.tpe}] + // Sorting on the in-memory is the same as binary + _root_.scala.util.Sorting.quickSort[${innerBuf.tpe}]($asArray)($innerOrd) + var $pos = 0 + while($pos < $len) { + val $innerElement = $asArray($pos) + ${innerBuf.put(inputStream, innerElement)} + $pos += 1 + } + } + """ + case NoSort => + q""" + val $len: Int = $element.size + $inputStream.writeSize($len) + $element.foreach { case $innerElement => + ${innerBuf.put(inputStream, innerElement)} + } + """ + } + + } + override def hash(element: ctx.TermName): ctx.Tree = { + val currentHash = freshT("currentHash") + val len = freshT("len") + val target = freshT("target") + maybeSort match { + case NoSort => + q""" + var $currentHash: Int = _root_.com.twitter.scalding.serialization.MurmerHashUtils.seed + var $len = 0 + $element.foreach { t => + val $target = t + $currentHash = + _root_.com.twitter.scalding.serialization.MurmerHashUtils.mixH1($currentHash, ${innerBuf.hash(target)}) + // go ahead and compute the length so we don't traverse twice for lists + $len += 1 + } + _root_.com.twitter.scalding.serialization.MurmerHashUtils.fmix($currentHash, $len) + """ + case DoSort => + // We actually don't sort here, which would be expensive, but combine with a commutative operation + // so the order that we see items won't matter. For this we use XOR + q""" + var $currentHash: Int = _root_.com.twitter.scalding.serialization.MurmerHashUtils.seed + var $len = 0 + $element.foreach { t => + val $target = t + $currentHash = $currentHash ^ ${innerBuf.hash(target)} + $len += 1 + } + // Might as well be fancy when we mix in the length + _root_.com.twitter.scalding.serialization.MurmerHashUtils.fmix($currentHash, $len) + """ + } + } + + override def get(inputStream: ctx.TermName): ctx.Tree = { + val len = freshT("len") + val firstVal = freshT("firstVal") + val travBuilder = freshT("travBuilder") + val iter = freshT("iter") + val extractionTree = maybeArray match { + case IsArray => + q"""val $travBuilder = new Array[..$innerTypes]($len) + var $iter = 0 + while($iter < $len) { + $travBuilder($iter) = ${innerBuf.get(inputStream)} + $iter = $iter + 1 + } + $travBuilder : $outerType + """ + case NotArray => + q"""val $travBuilder = $companionSymbol.newBuilder[..$innerTypes] + $travBuilder.sizeHint($len) + var $iter = 0 + while($iter < $len) { + $travBuilder += ${innerBuf.get(inputStream)} + $iter = $iter + 1 + } + $travBuilder.result : $outerType + """ + } + q""" + val $len: Int = $inputStream.readSize + if($len > 0) { + if($len == 1) { + val $firstVal: $innerType = ${innerBuf.get(inputStream)} + $companionSymbol.apply($firstVal) : $outerType + } else { + $extractionTree : $outerType + } + } else { + $companionSymbol.empty : $outerType + } + """ + } + + override def compare(elementA: ctx.TermName, elementB: ctx.TermName): ctx.Tree = { + + val a = freshT("a") + val b = freshT("b") + val cmpFnName = freshT("cmpFnName") + maybeSort match { + case DoSort => + q""" + _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.TraversableHelpers.sortedCompare[${innerBuf.tpe}]($elementA, $elementB)($innerOrd) + """ + + case NoSort => + q""" + _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.TraversableHelpers.iteratorCompare[${innerBuf.tpe}]($elementA.iterator, $elementB.iterator)($innerOrd) + """ + } + + } + + override val lazyOuterVariables: Map[String, ctx.Tree] = innerBuf.lazyOuterVariables + + override def length(element: Tree): CompileTimeLengthTypes[c.type] = { + + innerBuf.length(q"$element.head") match { + case const: ConstantLengthCalculation[_] => + FastLengthCalculation(c)(q"""{ + sizeBytes($element.size) + $element.size * ${const.toInt} + }""") + case m: MaybeLengthCalculation[_] => + val maybeRes = freshT("maybeRes") + MaybeLengthCalculation(c)(q""" + if($element.isEmpty) { + val sizeOfZero = 1 // writing the constant 0, for length, takes 1 byte + _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.DynamicLen(sizeOfZero) + } else { + val maybeRes = ${m.asInstanceOf[MaybeLengthCalculation[c.type]].t} + maybeRes match { + case _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.ConstLen(constSize) => + val sizeOverhead = sizeBytes($element.size) + _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.DynamicLen(constSize * $element.size + sizeOverhead) + + // todo maybe we should support this case + // where we can visit every member of the list relatively fast to ask + // its length. Should we care about sizes instead maybe? + case _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.DynamicLen(_) => + _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.NoLengthCalculation + case _ => _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.NoLengthCalculation + } + } + """) + // Something we can't workout the size of ahead of time + case _ => MaybeLengthCalculation(c)(q""" + if($element.isEmpty) { + val sizeOfZero = 1 // writing the constant 0, for length, takes 1 byte + _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.DynamicLen(sizeOfZero) + } else { + _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.NoLengthCalculation + } + """) + } + } + } + } +} + diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/UnitOrderedBuf.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/UnitOrderedBuf.scala new file mode 100644 index 0000000000..bbce0c81a8 --- /dev/null +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/UnitOrderedBuf.scala @@ -0,0 +1,62 @@ +/* + Copyright 2014 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.macros.impl.ordered_serialization.providers + +import scala.language.experimental.macros +import scala.reflect.macros.Context + +import com.twitter.scalding._ +import com.twitter.scalding.macros.impl.ordered_serialization.{ CompileTimeLengthTypes, ProductLike, TreeOrderedBuf } +import CompileTimeLengthTypes._ +import java.nio.ByteBuffer +import com.twitter.scalding.serialization.OrderedSerialization + +object UnitOrderedBuf { + def dispatch(c: Context): PartialFunction[c.Type, TreeOrderedBuf[c.type]] = { + case tpe if tpe =:= c.universe.typeOf[Unit] => UnitOrderedBuf(c)(tpe) + } + + def apply(c: Context)(outerType: c.Type): TreeOrderedBuf[c.type] = { + import c.universe._ + + new TreeOrderedBuf[c.type] { + override val ctx: c.type = c + override val tpe = outerType + + override def compareBinary(inputStreamA: ctx.TermName, inputStreamB: ctx.TermName) = + q"0" + + override def hash(element: ctx.TermName): ctx.Tree = + q"0" + + override def put(inputStream: ctx.TermName, element: ctx.TermName) = + q"()" + + override def get(inputStreamA: ctx.TermName): ctx.Tree = + q"()" + + def compare(elementA: ctx.TermName, elementB: ctx.TermName): ctx.Tree = + q"0" + + override def length(element: Tree): CompileTimeLengthTypes[c.type] = + ConstantLengthCalculation(c)(0) + + override val lazyOuterVariables: Map[String, ctx.Tree] = + Map.empty + } + } +} + diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/runtime_helpers/LengthCalculations.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/runtime_helpers/LengthCalculations.scala new file mode 100644 index 0000000000..22e5738009 --- /dev/null +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/runtime_helpers/LengthCalculations.scala @@ -0,0 +1,42 @@ +/* + Copyright 2014 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.macros.impl.ordered_serialization.runtime_helpers + +/** + * There is a Monoid on MaybeLength, with + * ConstLen(0) being the zero. + */ +sealed trait MaybeLength { + def +(that: MaybeLength): MaybeLength +} + +case object NoLengthCalculation extends MaybeLength { + def +(that: MaybeLength): MaybeLength = this +} +case class ConstLen(toInt: Int) extends MaybeLength { + def +(that: MaybeLength): MaybeLength = that match { + case ConstLen(c) => ConstLen(toInt + c) + case DynamicLen(d) => DynamicLen(toInt + d) + case NoLengthCalculation => NoLengthCalculation + } +} +case class DynamicLen(toInt: Int) extends MaybeLength { + def +(that: MaybeLength): MaybeLength = that match { + case ConstLen(c) => DynamicLen(toInt + c) + case DynamicLen(d) => DynamicLen(toInt + d) + case NoLengthCalculation => NoLengthCalculation + } +} diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/runtime_helpers/TraversableHelpers.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/runtime_helpers/TraversableHelpers.scala new file mode 100644 index 0000000000..8939204847 --- /dev/null +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/runtime_helpers/TraversableHelpers.scala @@ -0,0 +1,174 @@ +/* + Copyright 2014 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.macros.impl.ordered_serialization.runtime_helpers + +import java.io.InputStream +import scala.collection.mutable.Buffer + +object TraversableHelpers { + import com.twitter.scalding.serialization.JavaStreamEnrichments._ + + final def rawCompare(inputStreamA: InputStream, inputStreamB: InputStream)(consume: (InputStream, InputStream) => Int): Int = { + val lenA = inputStreamA.readSize + val lenB = inputStreamB.readSize + + val minLen = math.min(lenA, lenB) + var incr = 0 + var curIncr = 0 + while (incr < minLen && curIncr == 0) { + curIncr = consume(inputStreamA, inputStreamB) + incr = incr + 1 + } + + if (curIncr != 0) curIncr + else java.lang.Integer.compare(lenA, lenB) + } + + final def iteratorCompare[T](iteratorA: Iterator[T], iteratorB: Iterator[T])(implicit ord: Ordering[T]): Int = { + @annotation.tailrec + def result: Int = + if (iteratorA.isEmpty) { + if (iteratorB.isEmpty) 0 + else -1 // a is shorter + } else { + if (iteratorB.isEmpty) 1 // a is longer + else { + val cmp = ord.compare(iteratorA.next, iteratorB.next) + if (cmp != 0) cmp + else result + } + } + + result + } + + final def iteratorEquiv[T](iteratorA: Iterator[T], iteratorB: Iterator[T])(implicit eq: Equiv[T]): Boolean = { + @annotation.tailrec + def result: Boolean = + if (iteratorA.isEmpty) iteratorB.isEmpty + else if (iteratorB.isEmpty) false // not empty != empty + else eq.equiv(iteratorA.next, iteratorB.next) && result + + result + } + /** + * This returns the same result as + * + * implicit val o = ord + * Ordering[Iterable[T]].compare(travA.toList.sorted, travB.toList.sorted) + * + * but it does not do a full sort. Instead it uses a partial quicksort approach + * the complexity should be O(N + M) rather than O(N log N + M log M) for the full + * sort case + */ + final def sortedCompare[T](travA: Iterable[T], travB: Iterable[T])(implicit ord: Ordering[T]): Int = { + def compare(startA: Int, endA: Int, a: Buffer[T], startB: Int, endB: Int, b: Buffer[T]): Int = + if (startA == endA) { + if (startB == endB) 0 // both empty + else -1 // empty is smaller than non-empty + } else if (startB == endB) 1 // non-empty is bigger than empty + else { + @annotation.tailrec + def partition(pivot: T, pivotStart: Int, pivotEnd: Int, endX: Int, x: Buffer[T]): (Int, Int) = { + if (pivotEnd >= endX) (pivotStart, pivotEnd) + else { + val t = x(pivotEnd) + val cmp = ord.compare(t, pivot) + if (cmp == 0) { + // the pivot section grows by 1 to include test + partition(pivot, pivotStart, pivotEnd + 1, endX, x) + } else if (cmp > 0) { + // test is bigger, swap it with the end and move the end down: + val newEnd = endX - 1 + val end = x(newEnd) + x(newEnd) = t + x(pivotEnd) = end + // now try again: + partition(pivot, pivotStart, pivotEnd, newEnd, x) + } else { + // t < pivot so we need to push this value below the pivots: + val ps = x(pivotStart) // might not be pivot if the pivot size is 0 + x(pivotStart) = t + x(pivotEnd) = ps + partition(pivot, pivotStart + 1, pivotEnd + 1, endX, x) + } + } + } + val pivot = a(startA) + val (aps, ape) = partition(pivot, startA, startA + 1, endA, a) + val (bps, bpe) = partition(pivot, startB, startB, endB, b) + + val asublen = aps - startA + val bsublen = bps - startB + if (asublen != bsublen) { + // comparing to the longer is enough + // because one of them will then include pivots which are larger + val longer = math.max(asublen, bsublen) + def extend(s: Int, e: Int) = math.min(s + longer, e) + + if (asublen != 0) { + /* + * We can safely recurse because startA does not hold pivot, so we won't + * do the same algorithm + */ + compare(startA, extend(startA, endA), a, startB, extend(startB, endB), b) + } else { + /* + * We know that startB does not have the pivot, because if it did, bsublen == 0 + * and both are equal, which is not true in this branch. + * we can reverse the recursion to ensure we get a different pivot + */ + -compare(startB, extend(startB, endB), b, startA, extend(startA, endA), a) + } + } else { + // the prefixes are the same size + val cmp = compare(startA, aps, a, startB, bps, b) + if (cmp != 0) cmp + else { + // we need to look into the pivot area. We don't need to check + // for equality on the overlapped pivot range + val apsize = ape - aps + val bpsize = bpe - bps + val minpsize = math.min(apsize, bpsize) + val acheck = aps + minpsize + val bcheck = bps + minpsize + if (apsize != bpsize && + acheck < endA && + bcheck < endB) { + // exactly one of them has a pivot value + ord.compare(a(acheck), b(bcheck)) + } else { + // at least one of them or both is empty, and we pick it up above + compare(aps + minpsize, endA, a, bps + minpsize, endB, b) + } + } + } + } + + /** + * If we are equal unsorted, we are equal. + * this is useful because often scala will build identical sets + * exactly the same way, so this fast check will work. + */ + if (iteratorEquiv(travA.iterator, travB.iterator)(ord)) 0 + else { + // Let's do the more expensive, potentially full sort, algorithm + val a = travA.toBuffer + val b = travB.toBuffer + compare(0, a.size, a, 0, b.size, b) + } + } +} diff --git a/scalding-macros/src/test/scala/com/twitter/scalding/macros/MacroOrderingProperties.scala b/scalding-macros/src/test/scala/com/twitter/scalding/macros/MacroOrderingProperties.scala new file mode 100644 index 0000000000..0a1b11eb7c --- /dev/null +++ b/scalding-macros/src/test/scala/com/twitter/scalding/macros/MacroOrderingProperties.scala @@ -0,0 +1,526 @@ +/* +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.macros + +import org.scalatest.{ FunSuite, ShouldMatchers } +import org.scalatest.prop.Checkers +import org.scalatest.prop.PropertyChecks +import scala.language.experimental.macros +import com.twitter.scalding.serialization.{ OrderedSerialization, Law, Law1, Law2, Law3, Serialization } +import java.nio.ByteBuffer +import org.scalacheck.Arbitrary.{ arbitrary => arb } +import java.io.{ ByteArrayOutputStream, InputStream } + +import com.twitter.bijection.Bufferable +import org.scalacheck.{ Arbitrary, Gen, Prop } +import com.twitter.scalding.serialization.JavaStreamEnrichments + +import scala.collection.immutable.Queue + +trait LowerPriorityImplicit { + implicit def primitiveOrderedBufferSupplier[T] = macro com.twitter.scalding.macros.impl.OrderedSerializationProviderImpl[T] +} + +object LawTester { + def apply[T: Arbitrary](laws: Iterable[Law[T]]): Prop = + apply(implicitly[Arbitrary[T]].arbitrary, laws) + + def apply[T](g: Gen[T], laws: Iterable[Law[T]]): Prop = + laws.foldLeft(true: Prop) { + case (soFar, Law1(name, fn)) => soFar && Prop.forAll(g)(fn).label(name) + case (soFar, Law2(name, fn)) => soFar && Prop.forAll(g, g)(fn).label(name) + case (soFar, Law3(name, fn)) => soFar && Prop.forAll(g, g, g)(fn).label(name) + } +} + +object ByteBufferArb { + implicit def arbitraryTestTypes: Arbitrary[ByteBuffer] = Arbitrary { + for { + aBinary <- Gen.alphaStr.map(s => ByteBuffer.wrap(s.getBytes("UTF-8"))) + } yield aBinary + } +} +object TestCC { + import ByteBufferArb._ + implicit def arbitraryTestCC: Arbitrary[TestCC] = Arbitrary { + for { + aInt <- arb[Int] + aLong <- arb[Long] + aDouble <- arb[Double] + anOption <- arb[Option[Int]] + anStrOption <- arb[Option[String]] + anOptionOfAListOfStrings <- arb[Option[List[String]]] + aBB <- arb[ByteBuffer] + } yield TestCC(aInt, aLong, anOption, aDouble, anStrOption, anOptionOfAListOfStrings, aBB) + } +} +case class TestCC(a: Int, b: Long, c: Option[Int], d: Double, e: Option[String], f: Option[List[String]], aBB: ByteBuffer) + +object MyData { + implicit def arbitraryTestCC: Arbitrary[MyData] = Arbitrary { + for { + aInt <- arb[Int] + anOption <- arb[Option[Long]] + } yield new MyData(aInt, anOption) + } +} + +class MyData(override val _1: Int, override val _2: Option[Long]) extends Product2[Int, Option[Long]] { + override def canEqual(that: Any): Boolean = that match { + case o: MyData => this._1 == o._1 && this._2 == o._2 + case _ => false + } +} + +object MacroOpaqueContainer { + import java.io._ + implicit val myContainerOrderedSerializer = new OrderedSerialization[MacroOpaqueContainer] { + val intOrderedSerialization = _root_.com.twitter.scalding.macros.Macros.orderedBufferSupplier[Int] + + override def hash(s: MacroOpaqueContainer) = intOrderedSerialization.hash(s.myField) ^ Int.MaxValue + override def compare(a: MacroOpaqueContainer, b: MacroOpaqueContainer) = intOrderedSerialization.compare(a.myField, b.myField) + + override def read(in: InputStream) = intOrderedSerialization.read(in).map(MacroOpaqueContainer(_)) + + override def write(b: OutputStream, s: MacroOpaqueContainer) = intOrderedSerialization.write(b, s.myField) + + override def compareBinary(lhs: InputStream, rhs: InputStream) = intOrderedSerialization.compareBinary(lhs, rhs) + override def staticSize = Some(4) + } + + implicit def arbitraryMacroOpaqueContainer: Arbitrary[MacroOpaqueContainer] = Arbitrary { + for { + aInt <- arb[Int] + } yield MacroOpaqueContainer(aInt) + } + + def apply(d: Int): MacroOpaqueContainer = new MacroOpaqueContainer(d) +} + +class MacroOpaqueContainer(val myField: Int) { +} + +object Container { + implicit def arbitraryInnerCaseClass: Arbitrary[InnerCaseClass] = Arbitrary { + for { + anOption <- arb[Set[Double]] + } yield InnerCaseClass(anOption) + } + + type SetAlias = Set[Double] + case class InnerCaseClass(e: SetAlias) +} +class MacroOrderingProperties extends FunSuite with PropertyChecks with ShouldMatchers with LowerPriorityImplicit { + type SetAlias = Set[Double] + + import ByteBufferArb._ + import Container.arbitraryInnerCaseClass + + import OrderedSerialization.{ compare => oBufCompare } + + def gen[T: Arbitrary]: Gen[T] = implicitly[Arbitrary[T]].arbitrary + + def collectionArb[C[_], T: Arbitrary](implicit cbf: collection.generic.CanBuildFrom[Nothing, T, C[T]]): Arbitrary[C[T]] = Arbitrary { + gen[List[T]].map { l => + val builder = cbf() + l.foreach { builder += _ } + builder.result + } + } + + def serialize[T](t: T)(implicit orderedBuffer: OrderedSerialization[T]): InputStream = + serializeSeq(List(t)) + + def serializeSeq[T](t: Seq[T])(implicit orderedBuffer: OrderedSerialization[T]): InputStream = { + import JavaStreamEnrichments._ + + val baos = new ByteArrayOutputStream + t.foreach({ e => + orderedBuffer.write(baos, e) + }) + baos.toInputStream + } + + def rt[T](t: T)(implicit orderedBuffer: OrderedSerialization[T]) = { + val buf = serialize[T](t) + orderedBuffer.read(buf).get + } + + def rawCompare[T](a: T, b: T)(implicit obuf: OrderedSerialization[T]): Int = + obuf.compareBinary(serialize(a), serialize(b)).unsafeToInt + + def checkManyExplicit[T](i: List[(T, T)])(implicit obuf: OrderedSerialization[T]) = { + val serializedA = serializeSeq(i.map(_._1)) + val serializedB = serializeSeq(i.map(_._2)) + i.foreach { + case (a, b) => + val compareBinary = obuf.compareBinary(serializedA, serializedB).unsafeToInt + val compareMem = obuf.compare(a, b) + if (compareBinary < 0) { + assert(compareMem < 0, s"Compare binary: $compareBinary, and compareMem : $compareMem must have the same sign") + } else if (compareBinary > 0) { + assert(compareMem > 0, s"Compare binary: $compareBinary, and compareMem : $compareMem must have the same sign") + } + } + } + + def checkMany[T: Arbitrary](implicit obuf: OrderedSerialization[T]) = forAll { i: List[(T, T)] => + checkManyExplicit(i) + } + + def clamp(i: Int): Int = + i match { + case x if x < 0 => -1 + case x if x > 0 => 1 + case x => 0 + } + + def checkWithInputs[T](a: T, b: T)(implicit obuf: OrderedSerialization[T]) { + val rta = rt(a) // before we do anything ensure these don't throw + val rtb = rt(b) // before we do anything ensure these don't throw + val asize = Serialization.toBytes(a).length + if (obuf.dynamicSize(a).isDefined) { + assert(obuf.dynamicSize(a).get == asize, "dynamic size matches the correct value") + } + if (obuf.staticSize.isDefined) { + assert(obuf.dynamicSize(a).get == asize, "dynamic size matches the correct value") + assert(obuf.staticSize.get == asize, "dynamic size matches the correct value") + } + assert(oBufCompare(rta, a) === 0, s"A should be equal to itself after an RT -- ${rt(a)}") + assert(oBufCompare(rtb, b) === 0, s"B should be equal to itself after an RT-- ${rt(b)}") + assert(oBufCompare(a, b) + oBufCompare(b, a) === 0, "In memory comparasons make sense") + assert(rawCompare(a, b) + rawCompare(b, a) === 0, "When adding the raw compares in inverse order they should sum to 0") + assert(oBufCompare(rta, rtb) === oBufCompare(a, b), "Comparing a and b with ordered bufferables compare after a serialization RT") + } + + def checkAreSame[T](a: T, b: T)(implicit obuf: OrderedSerialization[T]) { + val rta = rt(a) // before we do anything ensure these don't throw + val rtb = rt(b) // before we do anything ensure these don't throw + assert(oBufCompare(rta, a) === 0, s"A should be equal to itself after an RT -- ${rt(a)}") + assert(oBufCompare(rtb, b) === 0, "B should be equal to itself after an RT-- ${rt(b)}") + assert(oBufCompare(a, b) === 0, "In memory comparasons make sense") + assert(oBufCompare(b, a) === 0, "In memory comparasons make sense") + assert(rawCompare(a, b) === 0, "When adding the raw compares in inverse order they should sum to 0") + assert(rawCompare(b, a) === 0, "When adding the raw compares in inverse order they should sum to 0") + assert(oBufCompare(rta, rtb) === 0, "Comparing a and b with ordered bufferables compare after a serialization RT") + } + + def check[T: Arbitrary](implicit obuf: OrderedSerialization[T]) = { + Checkers.check(LawTester(OrderedSerialization.allLaws)) + forAll(minSuccessful(500)) { (a: T, b: T) => checkWithInputs(a, b) } + } + + test("Test out Unit") { + primitiveOrderedBufferSupplier[Unit] + + check[Unit] + checkMany[Unit] + } + + test("Test out Int") { + primitiveOrderedBufferSupplier[Int] + + check[Int] + checkMany[Int] + } + + test("Test out Long") { + check[Long] + } + + test("Test out Short") { + + check[Short] + } + + test("Test out Float") { + + check[Float] + } + + test("Test out Boolean") { + primitiveOrderedBufferSupplier[Boolean] + check[Boolean] + } + + test("Test out ByteBuffer") { + primitiveOrderedBufferSupplier[ByteBuffer] + check[ByteBuffer] + } + + test("Test out List[Float]") { + primitiveOrderedBufferSupplier[List[Float]] + check[List[Float]] + } + test("Test out Queue[Int]") { + implicit val isa = collectionArb[Queue, Int] + primitiveOrderedBufferSupplier[Queue[Int]] + check[Queue[Int]] + } + test("Test out IndexedSeq[Int]") { + implicit val isa = collectionArb[IndexedSeq, Int] + primitiveOrderedBufferSupplier[IndexedSeq[Int]] + check[IndexedSeq[Int]] + } + test("Test out HashSet[Int]") { + import scala.collection.immutable.HashSet + implicit val isa = collectionArb[HashSet, Int] + primitiveOrderedBufferSupplier[HashSet[Int]] + check[HashSet[Int]] + } + test("Test out ListSet[Int]") { + import scala.collection.immutable.ListSet + implicit val isa = collectionArb[ListSet, Int] + primitiveOrderedBufferSupplier[ListSet[Int]] + check[ListSet[Int]] + } + + test("Test out List[String]") { + primitiveOrderedBufferSupplier[List[String]] + check[List[String]] + } + + test("Test out List[List[String]]") { + val oBuf = primitiveOrderedBufferSupplier[List[List[String]]] + assert(oBuf.dynamicSize(List(List("sdf"))) === None) + check[List[List[String]]] + } + + test("Test out List[Int]") { + primitiveOrderedBufferSupplier[List[Int]] + check[List[Int]] + } + + test("Test out SetAlias") { + primitiveOrderedBufferSupplier[SetAlias] + check[SetAlias] + } + + test("Container.InnerCaseClass") { + primitiveOrderedBufferSupplier[Container.InnerCaseClass] + check[Container.InnerCaseClass] + } + + test("Test out Seq[Int]") { + primitiveOrderedBufferSupplier[Seq[Int]] + check[Seq[Int]] + } + test("Test out scala.collection.Seq[Int]") { + primitiveOrderedBufferSupplier[scala.collection.Seq[Int]] + check[scala.collection.Seq[Int]] + } + + test("Test out Array[Byte]") { + primitiveOrderedBufferSupplier[Array[Byte]] + check[Array[Byte]] + } + + test("Test out Vector[Int]") { + primitiveOrderedBufferSupplier[Vector[Int]] + check[Vector[Int]] + } + + test("Test out Iterable[Int]") { + primitiveOrderedBufferSupplier[Iterable[Int]] + check[Iterable[Int]] + } + + test("Test out Set[Int]") { + primitiveOrderedBufferSupplier[Set[Int]] + check[Set[Int]] + } + + test("Test out Set[Double]") { + primitiveOrderedBufferSupplier[Set[Double]] + check[Set[Double]] + } + + test("Test out Map[Long, Set[Int]]") { + primitiveOrderedBufferSupplier[Map[Long, Set[Int]]] + check[Map[Long, Set[Int]]] + val c = List(Map(9223372036854775807L -> Set[Int]()), Map(-1L -> Set[Int](-2043106012))) + checkManyExplicit(c.map { i => (i, i) }) + checkMany[Map[Long, Set[Int]]] + } + + test("Test out Map[Long, Long]") { + primitiveOrderedBufferSupplier[Map[Long, Long]] + check[Map[Long, Long]] + } + test("Test out HashMap[Long, Long]") { + import scala.collection.immutable.HashMap + implicit val isa = Arbitrary(implicitly[Arbitrary[List[(Long, Long)]]].arbitrary.map(HashMap(_: _*))) + primitiveOrderedBufferSupplier[HashMap[Long, Long]] + check[HashMap[Long, Long]] + } + test("Test out ListMap[Long, Long]") { + import scala.collection.immutable.ListMap + implicit val isa = Arbitrary(implicitly[Arbitrary[List[(Long, Long)]]].arbitrary.map(ListMap(_: _*))) + primitiveOrderedBufferSupplier[ListMap[Long, Long]] + check[ListMap[Long, Long]] + } + + test("Test out comparing Maps(3->2, 2->3) and Maps(2->3, 3->2) ") { + val a = Map(3 -> 2, 2 -> 3) + val b = Map(2 -> 3, 3 -> 2) + checkWithInputs(a, b) + checkAreSame(a, b) + } + + test("Test out comparing Set(\"asdf\", \"jkl\") and Set(\"jkl\", \"asdf\")") { + val a = Set("asdf", "jkl") + val b = Set("jkl", "asdf") + checkWithInputs(a, b) + checkAreSame(a, b) + } + + test("Test out Double") { + + check[Double] + } + + test("Test out Byte") { + + check[Byte] + } + + test("Test out String") { + primitiveOrderedBufferSupplier[String] + + check[String] + checkMany[String] + } + + test("Test known hard String Case") { + val a = "6" + val b = "곆" + val ord = Ordering.String + assert(rawCompare(a, b) === clamp(ord.compare(a, b)), "Raw and in memory compares match.") + + val c = List("榴㉕⊟풠湜ᙬ覹ꜻ裧뚐⠂覝쫨塢䇺楠谭픚ᐌ轮뺷Ⱟ洦擄黏著탅ﮓꆋ숷梸傠ァ蹵窥轲闇涡飽ꌳ䝞慙擃", + "堒凳媨쉏떽㶥⾽샣井ㆠᇗ裉깴辫࠷᤭塈䎙寫㸉ᶴ䰄똇䡷䥞㷗䷱赫懓䷏剆祲ᝯ졑쐯헢鷴ӕ秔㽰ퟡ㏉鶖奚㙰银䮌ᕗ膾买씋썴행䣈丶偝쾕鐗쇊ኋ넥︇瞤䋗噯邧⹆♣ἷ铆玼⪷沕辤ᠥ⥰箼䔄◗", + "騰쓢堷뛭ᣣﰩ嚲ﲯ㤑ᐜ檊೦⠩奯ᓩ윇롇러ᕰెꡩ璞﫼᭵礀閮䈦椄뾪ɔ믻䖔᪆嬽フ鶬曭꣍ᆏ灖㐸뗋ㆃ녵ퟸ겵晬礙㇩䫓ᘞ昑싨", + "좃ఱ䨻綛糔唄࿁劸酊᫵橻쩳괊筆ݓ淤숪輡斋靑耜঄骐冠㝑⧠떅漫곡祈䵾ᳺ줵됵↲搸虂㔢Ꝅ芆٠풐쮋炞哙⨗쾄톄멛癔짍避쇜畾㣕剼⫁়╢ꅢ澛氌ᄚ㍠ꃫᛔ匙㜗詇閦單錖⒅瘧崥", + "獌癚畇") + checkManyExplicit(c.map { i => (i, i) }) + + val c2 = List("聸", "") + checkManyExplicit(c2.map { i => (i, i) }) + } + + test("Test out Option[Int]") { + val oser = primitiveOrderedBufferSupplier[Option[Int]] + + assert(oser.staticSize === None, "can't get the size statically") + check[Option[Int]] + checkMany[Option[Int]] + } + + test("Test out Option[String]") { + primitiveOrderedBufferSupplier[Option[String]] + + check[Option[String]] + checkMany[Option[String]] + } + + test("Test Either[Int, Option[Int]]") { + val oser = primitiveOrderedBufferSupplier[Either[Int, Option[Int]]] + assert(oser.staticSize === None, "can't get the size statically") + check[Either[Int, Option[Int]]] + } + test("Test Either[Int, String]") { + val oser = primitiveOrderedBufferSupplier[Either[Int, String]] + assert(oser.staticSize === None, "can't get the size statically") + assert(Some(Serialization.toBytes[Either[Int, String]](Left(1)).length) === oser.dynamicSize(Left(1)), + "serialization size matches dynamic size") + check[Either[Int, String]] + } + test("Test Either[Int, Int]") { + val oser = primitiveOrderedBufferSupplier[Either[Int, Int]] + assert(oser.staticSize === Some(5), "can get the size statically") + check[Either[Int, Int]] + } + test("Test Either[String, Int]") { + primitiveOrderedBufferSupplier[Either[String, Int]] + check[Either[String, Int]] + } + test("Test Either[String, String]") { + primitiveOrderedBufferSupplier[Either[String, String]] + check[Either[String, String]] + } + + test("Test out Option[Option[Int]]") { + primitiveOrderedBufferSupplier[Option[Option[Int]]] + + check[Option[Option[Int]]] + } + + test("test product like TestCC") { + checkMany[(Int, Long, Option[Int], Double, Option[String])] + } + + test("test specific tuple aa1") { + primitiveOrderedBufferSupplier[(String, Option[Int], String)] + + checkMany[(String, Option[Int], String)] + } + + test("test specific tuple 2") { + check[(String, Option[Int], String)] + } + + test("test specific tuple 3") { + val c = List(("", None, ""), + ("a", Some(1), "b")) + checkManyExplicit(c.map { i => (i, i) }) + } + + test("Test out TestCC") { + import TestCC._ + primitiveOrderedBufferSupplier[TestCC] + check[TestCC] + checkMany[TestCC] + } + + test("Test out (Int, Int)") { + primitiveOrderedBufferSupplier[(Int, Int)] + check[(Int, Int)] + } + + test("Test out (String, Option[Int], String)") { + primitiveOrderedBufferSupplier[(String, Option[Int], String)] + check[(String, Option[Int], String)] + } + + test("Test out MyData") { + import MyData._ + primitiveOrderedBufferSupplier[MyData] + check[MyData] + } + + test("Test out MacroOpaqueContainer") { + // This will test for things which our macros can't view themselves, so need to use an implicit to let the user provide instead. + // by itself should just work from its own implicits + implicitly[OrderedSerialization[MacroOpaqueContainer]] + + // Put inside a tuple2 to test that + primitiveOrderedBufferSupplier[(MacroOpaqueContainer, MacroOpaqueContainer)] + check[(MacroOpaqueContainer, MacroOpaqueContainer)] + check[Option[MacroOpaqueContainer]] + check[List[MacroOpaqueContainer]] + } +} diff --git a/scalding-macros/src/test/scala/com/twitter/scalding/macros/MacrosUnitTests.scala b/scalding-macros/src/test/scala/com/twitter/scalding/macros/MacrosUnitTests.scala index 7fdb83c9ae..aad48410ce 100644 --- a/scalding-macros/src/test/scala/com/twitter/scalding/macros/MacrosUnitTests.scala +++ b/scalding-macros/src/test/scala/com/twitter/scalding/macros/MacrosUnitTests.scala @@ -18,7 +18,7 @@ package com.twitter.scalding.macros import cascading.tuple.{ Tuple => CTuple, TupleEntry } import org.scalatest.{ Matchers, WordSpec } - +import scala.language.experimental.{ macros => smacros } import com.twitter.scalding._ import com.twitter.scalding.macros._ import com.twitter.scalding.macros.impl._ diff --git a/scalding-macros/src/test/scala/com/twitter/scalding/macros/TraversableHelperLaws.scala b/scalding-macros/src/test/scala/com/twitter/scalding/macros/TraversableHelperLaws.scala new file mode 100644 index 0000000000..5b54e7d6b4 --- /dev/null +++ b/scalding-macros/src/test/scala/com/twitter/scalding/macros/TraversableHelperLaws.scala @@ -0,0 +1,50 @@ +/* +Copyright 2014 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.macros + +import org.scalatest.{ FunSuite, ShouldMatchers } +import org.scalatest.prop.Checkers +import org.scalatest.prop.PropertyChecks + +import com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.TraversableHelpers._ + +class TraversableHelperLaws extends FunSuite with PropertyChecks with ShouldMatchers { + test("Iterator ordering should be Iterable ordering") { + forAll { (l1: List[Int], l2: List[Int]) => + assert(iteratorCompare[Int](l1.iterator, l2.iterator) === + Ordering[Iterable[Int]].compare(l1, l2), "Matches scala's Iterable compare") + } + } + test("Iterator equiv should be Iterable ordering") { + forAll { (l1: List[Int], l2: List[Int]) => + assert(iteratorEquiv[Int](l1.iterator, l2.iterator) === + Ordering[Iterable[Int]].equiv(l1, l2), "Matches scala's Iterable compare") + } + } + test("sortedCompare matches sort followed by compare List[Int]") { + forAll(minSuccessful(1000)) { (l1: List[Int], l2: List[Int]) => + assert(sortedCompare[Int](l1, l2) === + Ordering[Iterable[Int]].compare(l1.sorted, l2.sorted), "Matches scala's Iterable compare") + } + } + test("sortedCompare matches sort followed by compare Set[Int]") { + forAll(minSuccessful(1000)) { (l1: Set[Int], l2: Set[Int]) => + assert(sortedCompare[Int](l1, l2) === + Ordering[Iterable[Int]].compare(l1.toList.sorted, l2.toList.sorted), "Matches scala's Iterable compare") + } + } +} From d03f9217a0a9843fbfdb87bc5def7cc256158c40 Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Thu, 26 Feb 2015 12:16:50 -1000 Subject: [PATCH 037/177] Address Alex and Ian's comments --- .../scala/com/twitter/scalding/Config.scala | 2 +- .../scalding/serialization/Boxed.scala | 8 +- .../CascadingBinaryComparator.scala | 22 +- .../scalding/serialization/Hasher.scala | 13 +- .../serialization/JavaStreamEnrichments.scala | 43 ++- .../scalding/serialization/KryoHadoop.scala | 4 +- .../twitter/scalding/serialization/Laws.scala | 18 +- ...rHashUtils.scala => MurmurHashUtils.scala} | 2 +- .../serialization/OrderedSerialization.scala | 24 +- .../serialization/PositionInputStream.scala | 6 +- .../scalding/serialization/Reader.scala | 6 +- .../serialization/Serialization.scala | 28 +- .../serialization/Serialization2.scala | 7 +- .../StringOrderedSerialization.scala | 10 +- .../serialization/WrappedSerialization.scala | 6 +- .../scalding/serialization/Writer.scala | 6 +- .../com/twitter/scalding/LookupJoinTest.scala | 266 +++++++++--------- .../JavaStreamEnrichmentsProperties.scala | 6 +- 18 files changed, 280 insertions(+), 197 deletions(-) rename scalding-core/src/main/scala/com/twitter/scalding/serialization/{MurmerHashUtils.scala => MurmurHashUtils.scala} (99%) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala index dd6777269b..4142b0cb95 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala @@ -119,7 +119,7 @@ trait Config { def getRequireOrderedSerialization: Boolean = get(ScaldingRequireOrderedSerialization) - .map(java.lang.Boolean.parseBoolean) + .map(_.toBoolean) .getOrElse(false) def getCascadingSerializationTokens: Map[Int, String] = diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Boxed.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Boxed.scala index ff7d15e1d3..b1c31ca28d 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Boxed.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Boxed.scala @@ -32,16 +32,13 @@ trait Boxed[+K] { } class BoxedDefaultSerialization extends KSerializer[Boxed[_]] { - def write(kryo: Kryo, output: Output, t: Boxed[_]) { + override def write(kryo: Kryo, output: Output, t: Boxed[_]) { sys.error(s"Kryo should never be used to serialize a boxed instance: $t") } - def read(kryo: Kryo, input: Input, t: Class[Boxed[_]]): Boxed[_] = { + override def read(kryo: Kryo, input: Input, t: Class[Boxed[_]]): Boxed[_] = sys.error("Kryo should never be used to serialize a boxed instance, class: $t") - } } -// TODO: Make more of these with a script - @DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed0[K](override val get: K) extends Boxed[K] @@ -342,7 +339,6 @@ class Boxed98[K](override val get: K) extends Boxed[K] @DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed99[K](override val get: K) extends Boxed[K] -// TODO this could be any general bijection case class BoxedOrderedSerialization[K](box: K => Boxed[K], ord: OrderedSerialization[K]) extends OrderedSerialization[Boxed[K]] { diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/CascadingBinaryComparator.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/CascadingBinaryComparator.scala index 885b1d76b9..20135a729d 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/CascadingBinaryComparator.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/CascadingBinaryComparator.scala @@ -42,7 +42,6 @@ object CascadingBinaryComparator { * groupBy/cogroups are using a CascadingBinaryComparator */ def checkForOrderedSerialization(fd: FlowDef): Try[Unit] = { - // Get the asScala enrichments locally? import collection.JavaConverters._ import cascading.pipe._ import com.twitter.scalding.RichPipe @@ -53,21 +52,30 @@ object CascadingBinaryComparator { def check(s: Splice): Try[Unit] = { val m = s.getKeySelectors.asScala - if (m.isEmpty) Failure(new Exception(s"Splice must have KeySelectors: $s")) + + def error(s: String): Try[Unit] = + Failure(new RuntimeException("Cannot verify OrderedSerialization: " + s)) + + if (m.isEmpty) error(s"Splice must have KeySelectors: $s") else { reduce(m.map { case (pipename, fields) => - Try { - if (fields.getComparators()(0).isInstanceOf[com.twitter.scalding.serialization.CascadingBinaryComparator[_]]) - () - else sys.error(s"pipe: $s, fields: $fields, comparators: ${fields.getComparators.toList}") - } + /* + * Scalding typed-API ALWAYS puts the key into field position 0. + * If OrderedSerialization is enabled, this must be a CascadingBinaryComparator + */ + if (fields.getComparators()(0).isInstanceOf[CascadingBinaryComparator[_]]) + Success(()) + else error(s"pipe: $s, fields: $fields, comparators: ${fields.getComparators.toList}") }) } } val allPipes: Set[Pipe] = fd.getTails.asScala.map(p => RichPipe(p).upstreamPipes).flatten.toSet reduce(allPipes.iterator.map { + /* + * There are only two cascading primitives used by scalding that do key-sorting: + */ case gb: GroupBy => check(gb) case cg: CoGroup => check(cg) case _ => Success(()) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Hasher.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Hasher.scala index eab23cee78..06e352da4f 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Hasher.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Hasher.scala @@ -28,7 +28,7 @@ trait Hasher[@specialized(Boolean, Byte, Short, Int, Long, Float, Double) -T] { } object Hasher { - import MurmerHashUtils._ + import MurmurHashUtils._ final val seed = 0xf7ca7fd2 @inline @@ -43,9 +43,16 @@ object Hasher { def hash(i: Unit) = 0 } implicit val boolean: Hasher[Boolean] = new Hasher[Boolean] { - // Here we use the two largest mersenne primes + /** + * Here we use the two large primes as the hash codes. + * We use primes because we want the probability of collision when + * we mod with some size (to fit into hash-buckets stored in an array) + * to be low. The choice of prime numbers means that they have no factors + * in common with any size, but they could have the same remainder. + * We actually just use the exact same values as Java here. + */ @inline - def hash(i: Boolean) = if (i) Int.MaxValue else ((1 << 19) - 1) + def hash(i: Boolean) = if (i) 1231 else 1237 } implicit val byte: Hasher[Byte] = new Hasher[Byte] { @inline diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/JavaStreamEnrichments.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/JavaStreamEnrichments.scala index c5ef9256bf..1fa0ca3233 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/JavaStreamEnrichments.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/JavaStreamEnrichments.scala @@ -15,16 +15,46 @@ limitations under the License. */ package com.twitter.scalding.serialization -import java.io.{ ByteArrayInputStream, ByteArrayOutputStream, InputStream, OutputStream, EOFException } +import java.io._ object JavaStreamEnrichments { def eof: Nothing = throw new EOFException() + /** + * Note this is only recommended for testing. + * You may want to use ByteArrayInputOutputStream for performance critical concerns + */ implicit class RichByteArrayOutputStream(val baos: ByteArrayOutputStream) extends AnyVal { def toInputStream: ByteArrayInputStream = new ByteArrayInputStream(baos.toByteArray) } - def sizeBytes(i: Int): Int = + /** + * enrichment to treat an Array like an OutputStream + */ + implicit class RichByteArray(val bytes: Array[Byte]) extends AnyVal { + def wrapAsOutputStream: ArrayWrappingOutputStream = wrapAsOutputStreamAt(0) + def wrapAsOutputStreamAt(pos: Int): ArrayWrappingOutputStream = + new ArrayWrappingOutputStream(bytes, pos) + } + /** + * Wraps an Array so that you can write into it as a stream without reallocations + * or copying at the end. Useful if you know an upper bound on the number of bytes + * you will write + */ + class ArrayWrappingOutputStream(val buffer: Array[Byte], initPos: Int) extends OutputStream { + require(buffer.length <= initPos, + s"Initial position cannot be more than length: $initPos > ${buffer.length}") + private[this] var pos = initPos + def position: Int = pos + override def write(b: Int) { buffer(pos) = b.toByte; pos += 1 } + override def write(b: Array[Byte], off: Int, len: Int) { + Array.copy(b, off, buffer, pos, len) + pos += len + } + } + + def posVarIntSize(i: Int): Int = { + require(i >= 0, s"negative numbers not allowed: $i") if (i < ((1 << 8) - 1)) 1 else { if (i < ((1 << 16) - 1)) { @@ -33,6 +63,7 @@ object JavaStreamEnrichments { 7 } } + } /** * This has a lot of methods from DataInputStream without @@ -140,7 +171,7 @@ object JavaStreamEnrichments { * 3 bytes for 256 - 65535, * 7 bytes for 65536 - Int.MaxValue */ - final def readSize: Int = { + final def readPosVarInt: Int = { val c1 = readUnsignedByte if (c1 < ((1 << 8) - 1)) c1 else { @@ -155,7 +186,7 @@ object JavaStreamEnrichments { def go(c: Long): Unit = { val skipped = s.skip(c) if (skipped == c) () - else if (skipped == 0L) throw new Exception(s"count, c, skipped = ${(count, c, skipped)}") + else if (skipped == 0L) throw new IOException(s"could not skipFully: count, c, skipped = ${(count, c, skipped)}") else go(c - skipped) } if (count != 0L) go(count) else () @@ -180,8 +211,8 @@ object JavaStreamEnrichments { * 3 bytes for 256 - 65535, * 7 bytes for 65536 - Int.MaxValue */ - def writeSize(i: Int): Unit = { - require(i >= 0, s"sizes must be non-negative: ${i}") + def writePosVarInt(i: Int): Unit = { + require(i >= 0, s"must be non-negative: ${i}") if (i < ((1 << 8) - 1)) s.write(i.toByte) else { s.write(-1: Byte) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala index fd6cce090b..ac0ee43722 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala @@ -38,7 +38,7 @@ import com.twitter.scalding.Args import com.twitter.chill.algebird._ import com.twitter.chill.config.Config -import com.twitter.chill.{SingletonSerializer, ScalaKryoInstantiator, KryoInstantiator} +import com.twitter.chill.{ SingletonSerializer, ScalaKryoInstantiator, KryoInstantiator } class KryoHadoop(config: Config) extends KryoInstantiator { /** @@ -59,7 +59,7 @@ class KryoHadoop(config: Config) extends KryoInstantiator { newK.register(classOf[DateRange], new DateRangeSerializer()) newK.register(classOf[Args], new ArgsSerializer) // Some of the monoids from Algebird that we use: - newK.register(classOf[com.twitter.algebird.AveragedValue], new AveragedValueSerializer) + newK.register(classOf[com.twitter.algebird.AveragedValue], new AveragedValueSerializer) newK.register(classOf[com.twitter.algebird.DecayedValue], new DecayedValueSerializer) newK.register(classOf[com.twitter.algebird.HyperLogLogMonoid], new HLLMonoidSerializer) newK.register(classOf[com.twitter.algebird.Moments], new MomentsSerializer) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Laws.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Laws.scala index 6ecdcb996e..a173171480 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Laws.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Laws.scala @@ -15,17 +15,13 @@ limitations under the License. */ package com.twitter.scalding.serialization +/** + * This is a simple trait for describing laws on single parameter + * type classes (Serialization, Monoid, Ordering, etc...) + */ sealed trait Law[T] { def name: String - def satisfied(a: T, b: T, c: T): Boolean -} - -case class Law1[T](override val name: String, check: T => Boolean) extends Law[T] { - def satisfied(a: T, b: T, c: T) = check(a) -} -case class Law2[T](override val name: String, check: (T, T) => Boolean) extends Law[T] { - def satisfied(a: T, b: T, c: T) = check(a, b) -} -case class Law3[T](override val name: String, check: (T, T, T) => Boolean) extends Law[T] { - def satisfied(a: T, b: T, c: T) = check(a, b, c) } +case class Law1[T](override val name: String, check: T => Boolean) extends Law[T] +case class Law2[T](override val name: String, check: (T, T) => Boolean) extends Law[T] +case class Law3[T](override val name: String, check: (T, T, T) => Boolean) extends Law[T] diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/MurmerHashUtils.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/MurmurHashUtils.scala similarity index 99% rename from scalding-core/src/main/scala/com/twitter/scalding/serialization/MurmerHashUtils.scala rename to scalding-core/src/main/scala/com/twitter/scalding/serialization/MurmurHashUtils.scala index 736b4421a6..0ae55c1a43 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/MurmerHashUtils.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/MurmurHashUtils.scala @@ -25,7 +25,7 @@ import scala.util.hashing.MurmurHash3 // Taking a few functions from: // https://guava-libraries.googlecode.com/git/guava/src/com/google/common/hash/Murmur3_32HashFunction.java -object MurmerHashUtils { +object MurmurHashUtils { final val seed = 0xf7ca7fd2 private final val C1: Int = 0xcc9e2d51 private final val C2: Int = 0x1b873593 diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/OrderedSerialization.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/OrderedSerialization.scala index 04ff78c368..f2b2e71c58 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/OrderedSerialization.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/OrderedSerialization.scala @@ -115,11 +115,31 @@ object OrderedSerialization { def orderingTransitive[T](implicit ordb: OrderedSerialization[T]): Law3[T] = Law3("transitivity", { (a: T, b: T, c: T) => - !(ordb.lteq(a, b) && ordb.lteq(b, c)) || ordb.lteq(a, c) + if (ordb.lteq(a, b) && ordb.lteq(b, c)) { ordb.lteq(a, c) } + else true }) + /** + * ordering must be antisymmetric. If this is not so, sort-based partitioning + * will be broken + */ + def orderingAntisymmetry[T](implicit ordb: OrderedSerialization[T]): Law2[T] = + Law2("antisymmetry", + { (a: T, b: T) => + if (ordb.lteq(a, b) && ordb.lteq(b, a)) { ordb.equiv(a, b) } + else true + }) + /** + * ordering must be total. If this is not so, sort-based partitioning + * will be broken + */ + def orderingTotality[T](implicit ordb: OrderedSerialization[T]): Law2[T] = + Law2("totality", { (a: T, b: T) => (ordb.lteq(a, b) || ordb.lteq(b, a)) }) def allLaws[T: OrderedSerialization]: Iterable[Law[T]] = - Serialization.allLaws ++ List(compareBinaryMatchesCompare[T], orderingTransitive[T]) + Serialization.allLaws ++ List(compareBinaryMatchesCompare[T], + orderingTransitive[T], + orderingAntisymmetry[T], + orderingTotality[T]) } /** diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/PositionInputStream.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/PositionInputStream.scala index e419c722cc..59880124e6 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/PositionInputStream.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/PositionInputStream.scala @@ -47,7 +47,11 @@ class PositionInputStream(val wraps: InputStream) extends InputStream { if (result >= 0) pos += 1 result } - override def read(bytes: Array[Byte]): Int = read(bytes, 0, bytes.length) + override def read(bytes: Array[Byte]): Int = { + val count = wraps.read(bytes) + if (count > 0) pos += count + count + } override def read(bytes: Array[Byte], off: Int, len: Int): Int = { val count = wraps.read(bytes, off, len) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Reader.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Reader.scala index 92062b4fd0..0e1a4e6199 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Reader.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Reader.scala @@ -62,7 +62,7 @@ object Reader { } implicit val string: Reader[String] = new Reader[String] { def read(is: InputStream) = { - val size = is.readSize + val size = is.readPosVarInt val bytes = new Array[Byte](size) is.readFully(bytes) new String(bytes, "UTF-8") @@ -94,7 +94,7 @@ object Reader { new Reader[Array[T]] { val readerT = implicitly[Reader[T]] def read(is: InputStream) = { - val size = is.readSize + val size = is.readPosVarInt val res = new Array[T](size) @annotation.tailrec def go(p: Int): Unit = @@ -113,7 +113,7 @@ object Reader { val readerT = implicitly[Reader[T]] def read(is: InputStream): C = { val builder = cbf() - val size = is.readSize + val size = is.readPosVarInt builder.sizeHint(size) @annotation.tailrec def go(idx: Int): Unit = diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization.scala index 03fb695edb..fcc419d21f 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization.scala @@ -27,7 +27,9 @@ import scala.util.hashing.Hashing * need to do key sorting for partitioning. * * This serialization typeclass must serialize equivalent objects - * identically to be lawful. Given that constraint, we can always + * identically to be lawful. Serialization should be the same + * on all JVMs run at any time, in other words, Serialization is a + * pure function. Given that constraint, we can always * get an Equiv and Hashing from a Serialization (by doing byte-wise * equivalence or byte-wise hashing). * @@ -41,6 +43,9 @@ trait Serialization[T] extends Equiv[T] with Hashing[T] with Serializable { def write(out: OutputStream, t: T): Try[Unit] /** * If all items have a static size, this returns Some, else None + * NOTE: lawful implementations that return Some here much return + * Some on dynamicSize so callers don't need to check both when + * they have an instance. */ def staticSize: Option[Int] = None /** @@ -69,15 +74,28 @@ object Serialization { def write[T](out: OutputStream, t: T)(implicit ser: Serialization[T]): Try[Unit] = ser.write(out, t) - def toBytes[T: Serialization](t: T): Array[Byte] = { - val baos = new ByteArrayOutputStream - write(baos, t).get // this should only throw on OOM - baos.toByteArray + def toBytes[T](t: T)(implicit ser: Serialization[T]): Array[Byte] = { + ser.dynamicSize(t) match { + case None => + val baos = new ByteArrayOutputStream + write(baos, t).get // this should only throw on OOM + baos.toByteArray + case Some(size) => + // If we know the size, we can just write directly into a fixed + // size byte array + val bytes = new Array[Byte](size) + val os = bytes.wrapAsOutputStream + write(os, t).get // this should only throw on OOM + bytes + } } def fromBytes[T: Serialization](b: Array[Byte]): Try[T] = read(new ByteArrayInputStream(b)) + /** + * This copies more than needed, but it is only for testing + */ private def roundTrip[T](t: T)(implicit ser: Serialization[T]): T = { val baos = new ByteArrayOutputStream ser.write(baos, t).get // should never throw on a ByteArrayOutputStream diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization2.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization2.scala index e87edb7958..2fdb7c9989 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization2.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization2.scala @@ -22,7 +22,12 @@ import scala.util.{ Failure, Success, Try } class Serialization2[A, B](val serA: Serialization[A], val serB: Serialization[B]) extends Serialization[(A, B)] { private val MAX_PRIME = Int.MaxValue // turns out MaxValue is a prime, which we want below - override def hash(x: (A, B)) = serA.hash(x._1) + MAX_PRIME * serB.hash(x._2) + override def hash(x: (A, B)) = { + import MurmurHashUtils._ + val h1 = mixH1(seed, serA.hash(x._1)) + val h2 = mixH1(h1, serB.hash(x._2)) + fmix(h2, 2) + } override def equiv(x: (A, B), y: (A, B)): Boolean = serA.equiv(x._1, y._1) && serB.equiv(x._2, y._2) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/StringOrderedSerialization.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/StringOrderedSerialization.scala index 1e64e94459..e6f4fc883b 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/StringOrderedSerialization.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/StringOrderedSerialization.scala @@ -86,21 +86,21 @@ class StringOrderedSerialization extends OrderedSerialization[String] { override def hash(s: String) = s.hashCode override def compare(a: String, b: String) = a.compareTo(b) override def read(in: InputStream) = try { - val byteString = new Array[Byte](in.readSize) + val byteString = new Array[Byte](in.readPosVarInt) in.readFully(byteString) Success(new String(byteString, "UTF-8")) } catch { case NonFatal(e) => Failure(e) } override def write(b: OutputStream, s: String) = try { val bytes = s.getBytes("UTF-8") - b.writeSize(bytes.length) + b.writePosVarInt(bytes.length) b.writeBytes(bytes) Serialization.successUnit } catch { case NonFatal(e) => Failure(e) } override def compareBinary(lhs: InputStream, rhs: InputStream) = try { - val leftSize = lhs.readSize - val rightSize = rhs.readSize + val leftSize = lhs.readPosVarInt + val rightSize = rhs.readPosVarInt val seekingLeft = PositionInputStream(lhs) val seekingRight = PositionInputStream(rhs) @@ -112,5 +112,7 @@ class StringOrderedSerialization extends OrderedSerialization[String] { seekingLeft.seekToPosition(leftStart + leftSize) seekingRight.seekToPosition(rightStart + rightSize) res + } catch { + case NonFatal(e) => OrderedSerialization.CompareFailure(e) } } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/WrappedSerialization.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/WrappedSerialization.scala index 1bda3f179c..ee5eccb56e 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/WrappedSerialization.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/WrappedSerialization.scala @@ -49,10 +49,12 @@ class WrappedSerialization[T] extends HSerialization[T] with Configurable { .asInstanceOf[Option[Serialization[T]]] def getSerializer(c: Class[T]): Serializer[T] = - new BinarySerializer(getSerialization(c).getOrElse(sys.error(s"Class: ${c} not found"))) + new BinarySerializer(getSerialization(c) + .getOrElse(sys.error(s"Serialization for class: ${c} not found"))) def getDeserializer(c: Class[T]): Deserializer[T] = - new BinaryDeserializer(getSerialization(c).getOrElse(sys.error(s"Class: ${c} not found"))) + new BinaryDeserializer(getSerialization(c) + .getOrElse(sys.error(s"Serialization for class: ${c} not found"))) } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Writer.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Writer.scala index 4db85ef171..27436ed26c 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Writer.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Writer.scala @@ -62,7 +62,7 @@ object Writer { implicit val string: Writer[String] = new Writer[String] { def write(os: OutputStream, s: String) = { val bytes = s.getBytes("UTF-8") - os.writeSize(bytes.length) + os.writePosVarInt(bytes.length) os.writeBytes(bytes) } } @@ -103,7 +103,7 @@ object Writer { val writerT = implicitly[Writer[T]] def write(os: OutputStream, a: Array[T]) = { val size = a.length - os.writeSize(size) + os.writePosVarInt(size) @annotation.tailrec def go(p: Int): Unit = if (p == size) () @@ -118,7 +118,7 @@ object Writer { val writerT = implicitly[Writer[T]] def write(os: OutputStream, c: C) = { val size = c.size - os.writeSize(size) + os.writePosVarInt(size) c.foreach { t: T => writerT.write(os, t) } diff --git a/scalding-core/src/test/scala/com/twitter/scalding/LookupJoinTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/LookupJoinTest.scala index abfe81dca4..754c7f3633 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/LookupJoinTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/LookupJoinTest.scala @@ -28,7 +28,7 @@ object LookupJoinedTest { (0 until sz).view.map { _ => (rng.nextInt(maxTime), rng.nextInt(maxKey), rng.nextInt) } - .groupBy { case (t, k, v) => (t, k)} + .groupBy { case (t, k, v) => (t, k) } .mapValues(_.headOption.toList) .values .flatten @@ -36,188 +36,182 @@ object LookupJoinedTest { } } +class LookupJoinerJob(args: Args) extends Job(args) { - class LookupJoinerJob(args: Args) extends Job(args) { + import TDsl._ - import TDsl._ + val in0 = TypedTsv[(Int, Int, Int)]("input0") + val in1 = TypedTsv[(Int, Int, Int)]("input1") - val in0 = TypedTsv[(Int, Int, Int)]("input0") - val in1 = TypedTsv[(Int, Int, Int)]("input1") - - LookupJoin(TypedPipe.from(in0).map { case (t, k, v) => (t, (k, v))}, - TypedPipe.from(in1).map { case (t, k, v) => (t, (k, v))}) - .map { + LookupJoin(TypedPipe.from(in0).map { case (t, k, v) => (t, (k, v)) }, + TypedPipe.from(in1).map { case (t, k, v) => (t, (k, v)) }) + .map { case (t, (k, (v, opt))) => (t.toString, k.toString, v.toString, opt.toString) } - .write(TypedTsv[(String, String, String, String)]("output")) + .write(TypedTsv[(String, String, String, String)]("output")) - LookupJoin.rightSumming(TypedPipe.from(in0).map { case (t, k, v) => (t, (k, v))}, - TypedPipe.from(in1).map { case (t, k, v) => (t, (k, v))}) - .map { + LookupJoin.rightSumming(TypedPipe.from(in0).map { case (t, k, v) => (t, (k, v)) }, + TypedPipe.from(in1).map { case (t, k, v) => (t, (k, v)) }) + .map { case (t, (k, (v, opt))) => (t.toString, k.toString, v.toString, opt.toString) } - .write(TypedTsv[(String, String, String, String)]("output2")) - } + .write(TypedTsv[(String, String, String, String)]("output2")) +} - class LookupJoinedTest extends WordSpec with Matchers { +class LookupJoinedTest extends WordSpec with Matchers { - import Dsl._ - import LookupJoinedTest.genList + import Dsl._ + import LookupJoinedTest.genList - def lookupJoin[T: Ordering, K, V, W](in0: Iterable[(T, K, V)], in1: Iterable[(T, K, W)]) = { - val serv = in1.groupBy(_._2) - def lookup(t: T, k: K): Option[W] = { - val ord = Ordering.by { tkw: (T, K, W) => tkw._1} - serv.get(k).flatMap { in1s => - in1s.filter { case (t1, _, _) => Ordering[T].lt(t1, t)} - .reduceOption(ord.max(_, _)) - .map { + def lookupJoin[T: Ordering, K, V, W](in0: Iterable[(T, K, V)], in1: Iterable[(T, K, W)]) = { + val serv = in1.groupBy(_._2) + def lookup(t: T, k: K): Option[W] = { + val ord = Ordering.by { tkw: (T, K, W) => tkw._1 } + serv.get(k).flatMap { in1s => + in1s.filter { case (t1, _, _) => Ordering[T].lt(t1, t) } + .reduceOption(ord.max(_, _)) + .map { _._3 } - } } - in0.map { case (t, k, v) => (t.toString, k.toString, v.toString, lookup(t, k).toString)} } + in0.map { case (t, k, v) => (t.toString, k.toString, v.toString, lookup(t, k).toString) } + } - def lookupSumJoin[T: Ordering, K, V, W: Semigroup](in0: Iterable[(T, K, V)], in1: Iterable[(T, K, W)]) = { - implicit val ord: Ordering[(T, K, W)] = Ordering.by { - _._1 - } - val serv = in1.groupBy(_._2).mapValues { - _.toList - .sorted - .scanLeft(None: Option[(T, K, W)]) { (old, newer) => - old.map { case (_, _, w) => (newer._1, newer._2, Semigroup.plus(w, newer._3))} + def lookupSumJoin[T: Ordering, K, V, W: Semigroup](in0: Iterable[(T, K, V)], in1: Iterable[(T, K, W)]) = { + implicit val ord: Ordering[(T, K, W)] = Ordering.by { + _._1 + } + val serv = in1.groupBy(_._2).mapValues { + _.toList + .sorted + .scanLeft(None: Option[(T, K, W)]) { (old, newer) => + old.map { case (_, _, w) => (newer._1, newer._2, Semigroup.plus(w, newer._3)) } .orElse(Some(newer)) } - .filter { + .filter { _.isDefined } - .map { + .map { _.get } - }.toMap // Force the map - - def lookup(t: T, k: K): Option[W] = { - val ord = Ordering.by { tkw: (T, K, W) => tkw._1} - serv.get(k).flatMap { in1s => - in1s.filter { case (t1, _, _) => Ordering[T].lt(t1, t)} - .reduceOption(ord.max(_, _)) - .map { + }.toMap // Force the map + + def lookup(t: T, k: K): Option[W] = { + val ord = Ordering.by { tkw: (T, K, W) => tkw._1 } + serv.get(k).flatMap { in1s => + in1s.filter { case (t1, _, _) => Ordering[T].lt(t1, t) } + .reduceOption(ord.max(_, _)) + .map { _._3 } - } } - in0.map { case (t, k, v) => (t.toString, k.toString, v.toString, lookup(t, k).toString)} } + in0.map { case (t, k, v) => (t.toString, k.toString, v.toString, lookup(t, k).toString) } + } - "A LookupJoinerJob" should { - "correctly lookup" in { - val MAX_KEY = 100 - val VAL_COUNT = 10000 - val in0 = genList(Int.MaxValue, MAX_KEY, VAL_COUNT) - val in1 = genList(Int.MaxValue, MAX_KEY, VAL_COUNT) - JobTest(new LookupJoinerJob(_)) - .source(TypedTsv[(Int, Int, Int)]("input0"), in0) - .source(TypedTsv[(Int, Int, Int)]("input1"), in1) - .sink[(String, String, String, String)]( - TypedTsv[(String, String, String, String)]("output")) { outBuf => - outBuf.toSet should equal (lookupJoin(in0, in1).toSet) - in0.size should equal (outBuf.size) - } - .sink[(String, String, String, String)]( - TypedTsv[(String, String, String, String)]("output2")) { outBuf => - outBuf.toSet should equal(lookupSumJoin(in0, in1).toSet) - in0.size should equal(outBuf.size) - } - .run - //.runHadoop - .finish - } + "A LookupJoinerJob" should { + "correctly lookup" in { + val MAX_KEY = 100 + val VAL_COUNT = 10000 + val in0 = genList(Int.MaxValue, MAX_KEY, VAL_COUNT) + val in1 = genList(Int.MaxValue, MAX_KEY, VAL_COUNT) + JobTest(new LookupJoinerJob(_)) + .source(TypedTsv[(Int, Int, Int)]("input0"), in0) + .source(TypedTsv[(Int, Int, Int)]("input1"), in1) + .sink[(String, String, String, String)]( + TypedTsv[(String, String, String, String)]("output")) { outBuf => + outBuf.toSet should equal (lookupJoin(in0, in1).toSet) + in0.size should equal (outBuf.size) + } + .sink[(String, String, String, String)]( + TypedTsv[(String, String, String, String)]("output2")) { outBuf => + outBuf.toSet should equal(lookupSumJoin(in0, in1).toSet) + in0.size should equal(outBuf.size) + } + .run + //.runHadoop + .finish } } +} - class WindowLookupJoinerJob(args: Args) extends Job(args) { +class WindowLookupJoinerJob(args: Args) extends Job(args) { - import TDsl._ + import TDsl._ - val in0 = TypedTsv[(Int, Int, Int)]("input0") - val in1 = TypedTsv[(Int, Int, Int)]("input1") - val window = args("window").toInt + val in0 = TypedTsv[(Int, Int, Int)]("input0") + val in1 = TypedTsv[(Int, Int, Int)]("input1") + val window = args("window").toInt - def gate(left: Int, right: Int) = - (left.toLong - right.toLong) < window + def gate(left: Int, right: Int) = + (left.toLong - right.toLong) < window - LookupJoin.withWindow(TypedPipe.from(in0).map { case (t, k, v) => (t, (k, v))}, - TypedPipe.from(in1).map { case (t, k, v) => (t, (k, v))})(gate _) - .map { + LookupJoin.withWindow(TypedPipe.from(in0).map { case (t, k, v) => (t, (k, v)) }, + TypedPipe.from(in1).map { case (t, k, v) => (t, (k, v)) })(gate _) + .map { case (t, (k, (v, opt))) => (t.toString, k.toString, v.toString, opt.toString) } - .write(TypedTsv[(String, String, String, String)]("output")) - } + .write(TypedTsv[(String, String, String, String)]("output")) +} - class WindowLookupJoinedTest extends WordSpec with Matchers { +class WindowLookupJoinedTest extends WordSpec with Matchers { - import Dsl._ - import LookupJoinedTest.genList + import Dsl._ + import LookupJoinedTest.genList - def windowLookupJoin[K, V, W](in0: Iterable[(Int, K, V)], in1: Iterable[(Int, K, W)], win: Int) = { - val serv = in1.groupBy(_._2) - // super inefficient, but easy to verify: - def lookup(t: Int, k: K): Option[W] = { - val ord = Ordering.by { tkw: (Int, K, W) => tkw._1} - serv.get(k).flatMap { in1s => - in1s.filter { - case (t1, _, _) => - (t1 < t) && ((t.toLong - t1.toLong) < win) - } - .reduceOption(ord.max(_, _)) - .map { + def windowLookupJoin[K, V, W](in0: Iterable[(Int, K, V)], in1: Iterable[(Int, K, W)], win: Int) = { + val serv = in1.groupBy(_._2) + // super inefficient, but easy to verify: + def lookup(t: Int, k: K): Option[W] = { + val ord = Ordering.by { tkw: (Int, K, W) => tkw._1 } + serv.get(k).flatMap { in1s => + in1s.filter { + case (t1, _, _) => + (t1 < t) && ((t.toLong - t1.toLong) < win) + } + .reduceOption(ord.max(_, _)) + .map { _._3 } - } } - in0.map { case (t, k, v) => (t.toString, k.toString, v.toString, lookup(t, k).toString)} } + in0.map { case (t, k, v) => (t.toString, k.toString, v.toString, lookup(t, k).toString) } + } - "A WindowLookupJoinerJob" should { - //Set up the job: - "correctly lookup" in { - val MAX_KEY = 10 - val MAX_TIME = 10000 - val sz:Int = 10000; - val in0 = genList(MAX_TIME, MAX_KEY, 10000) - val in1 = genList(MAX_TIME, MAX_KEY, 10000) - JobTest(new WindowLookupJoinerJob(_)) - .arg("window", "100") - .source(TypedTsv[(Int, Int, Int)]("input0"), in0) - .source(TypedTsv[(Int, Int, Int)]("input1"), in1) - .sink[(String, String, String, String)]( - TypedTsv[(String, String, String, String)]("output")) { outBuf => - val results = outBuf.toList.sorted - val correct = windowLookupJoin(in0, in1, 100).toList.sorted - def some(it: List[(String, String, String, String)]) = - it.filter(_._4.startsWith("Some")) - - def none(it: List[(String, String, String, String)]) = - it.filter(_._4.startsWith("None")) - - some(results) shouldBe (some(correct)) - none(results) shouldBe (none(correct)) - in0.size should equal (outBuf.size) - } - .run - //.runHadoop - .finish - } + "A WindowLookupJoinerJob" should { + //Set up the job: + "correctly lookup" in { + val MAX_KEY = 10 + val MAX_TIME = 10000 + val sz: Int = 10000; + val in0 = genList(MAX_TIME, MAX_KEY, 10000) + val in1 = genList(MAX_TIME, MAX_KEY, 10000) + JobTest(new WindowLookupJoinerJob(_)) + .arg("window", "100") + .source(TypedTsv[(Int, Int, Int)]("input0"), in0) + .source(TypedTsv[(Int, Int, Int)]("input1"), in1) + .sink[(String, String, String, String)]( + TypedTsv[(String, String, String, String)]("output")) { outBuf => + val results = outBuf.toList.sorted + val correct = windowLookupJoin(in0, in1, 100).toList.sorted + def some(it: List[(String, String, String, String)]) = + it.filter(_._4.startsWith("Some")) + + def none(it: List[(String, String, String, String)]) = + it.filter(_._4.startsWith("None")) + + some(results) shouldBe (some(correct)) + none(results) shouldBe (none(correct)) + in0.size should equal (outBuf.size) + } + .run + //.runHadoop + .finish } } - - - - - +} diff --git a/scalding-core/src/test/scala/com/twitter/scalding/serialization/JavaStreamEnrichmentsProperties.scala b/scalding-core/src/test/scala/com/twitter/scalding/serialization/JavaStreamEnrichmentsProperties.scala index 9f29d5f10a..8c95db542f 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/serialization/JavaStreamEnrichmentsProperties.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/serialization/JavaStreamEnrichmentsProperties.scala @@ -62,16 +62,16 @@ object JavaStreamEnrichmentsProperties extends Properties("JavaStreamEnrichments writeRead(implicitly[Arbitrary[T]].arbitrary, w, r) property("Can (read/write)Size") = writeRead(Gen.chooseNum(0, Int.MaxValue), - { (i: Int, os) => os.writeSize(i) }, { _.readSize }) + { (i: Int, os) => os.writePosVarInt(i) }, { _.readPosVarInt }) property("Can (read/write)Float") = writeRead( { (i: Float, os) => os.writeFloat(i) }, { _.readFloat }) property("Can (read/write)Array[Byte]") = writeRead( // Use list because Array has a shitty toString - { (b: List[Byte], os) => os.writeSize(b.size); os.writeBytes(b.toArray) }, + { (b: List[Byte], os) => os.writePosVarInt(b.size); os.writeBytes(b.toArray) }, { is => - val bytes = new Array[Byte](is.readSize) + val bytes = new Array[Byte](is.readPosVarInt) is.readFully(bytes) bytes.toList }) From 08a714ce600d5f117dd365543223eba50177f727 Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Thu, 26 Feb 2015 14:23:50 -0800 Subject: [PATCH 038/177] Add row L1 normalization logic --- .../com/twitter/scalding/mathematics/Matrix2.scala | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix2.scala b/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix2.scala index 7f2be86a29..13972368bd 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix2.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix2.scala @@ -142,6 +142,19 @@ sealed trait Matrix2[R, C, V] extends Serializable { MatrixLiteral(result, SizeHint.asDiagonal(this.sizeHint.setRowsToCols)) * matD } + /** + * Row L1 normalization (can only be called for Double) + * After this operation, the sum(|x|) alone each row will be 1. + */ + def rowL1Normalize(implicit ev: =:=[V, Double], mj: MatrixJoiner2): Matrix2[R, C, Double] = { + val matD = this.asInstanceOf[Matrix2[R, C, Double]] + lazy val result = MatrixLiteral(matD.toTypedPipe.map { case (r, c, x) => (r, c, x.abs) }, this.sizeHint) + .sumColVectors + .toTypedPipe + .map { case (r, c, x) => (r, r, 1 / x) } // diagonal + inverse + MatrixLiteral(result, SizeHint.asDiagonal(this.sizeHint.setRowsToCols)) * matD + } + def getRow(index: R): Matrix2[Unit, C, V] = MatrixLiteral( toTypedPipe From b6c7a83e458c8f6766f53d11d6c590702fe4e128 Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Thu, 26 Feb 2015 14:24:02 -0800 Subject: [PATCH 039/177] Add test for row L1 normalization --- .../scalding/mathematics/Matrix2Test.scala | 29 +++++++++++++++++++ 1 file changed, 29 insertions(+) diff --git a/scalding-core/src/test/scala/com/twitter/scalding/mathematics/Matrix2Test.scala b/scalding-core/src/test/scala/com/twitter/scalding/mathematics/Matrix2Test.scala index f3faee0c1b..541abb80d8 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/mathematics/Matrix2Test.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/mathematics/Matrix2Test.scala @@ -221,6 +221,21 @@ class Matrix2Cosine(args: Args) extends Job(args) { cosine.write(TypedTsv[(Int, Int, Double)]("cosine")) } +class Matrix2Normalize(args: Args) extends Job(args) { + + import Matrix2._ + import cascading.pipe.Pipe + import cascading.tuple.Fields + import com.twitter.scalding.TDsl._ + + val p1: Pipe = Tsv("mat1", ('x1, 'y1, 'v1)).read + val tp1 = p1.toTypedPipe[(Int, Int, Double)](('x1, 'y1, 'v1)) + val mat1 = MatrixLiteral(tp1, NoClue) + + val matL1Norm = mat1.rowL1Normalize + matL1Norm.write(TypedTsv[(Int, Int, Double)]("normalized")) +} + class Scalar2Ops(args: Args) extends Job(args) { import Matrix2._ @@ -433,6 +448,20 @@ class Matrix2Test extends WordSpec with Matchers { } } + "A Matrix2 Normalize job" should { + TUtil.printStack { + JobTest(new Matrix2Normalize(_)) + .source(Tsv("mat1", ('x1, 'y1, 'v1)), List((1, 1, 4.0), (1, 2, 1.0), (2, 2, 1.0), (3, 1, 1.0), (3, 2, 3.0), (3, 3, 4.0))) + .typedSink(TypedTsv[(Int, Int, Double)]("normalized")) { ob => + "correctly compute cosine similarity" in { + toSparseMat(ob) shouldBe Map((1, 1) -> 0.8, (1, 2) -> 0.2, (2, 2) -> 1.0, (3, 1) -> 0.125, (3, 2) -> 0.375, (3, 3) -> 0.5) + } + } + .runHadoop + .finish + } + } + "A Matrix2 Scalar2Ops job" should { TUtil.printStack { JobTest(new Scalar2Ops(_)) From 0f05fbfbe97d0a3e4ea802c6bcee67997f03bf6b Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Thu, 26 Feb 2015 16:10:22 -0800 Subject: [PATCH 040/177] Change type from Double to V --- .../scala/com/twitter/scalding/mathematics/Matrix2.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix2.scala b/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix2.scala index 13972368bd..77e1e59aac 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix2.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix2.scala @@ -133,7 +133,7 @@ sealed trait Matrix2[R, C, V] extends Serializable { * Row L2 normalization (can only be called for Double) * After this operation, the sum(|x|^2) along each row will be 1. */ - def rowL2Normalize(implicit ev: =:=[V, Double], mj: MatrixJoiner2): Matrix2[R, C, Double] = { + def rowL2Normalize(implicit num: Numeric[V], mj: MatrixJoiner2): Matrix2[R, C, Double] = { val matD = this.asInstanceOf[Matrix2[R, C, Double]] lazy val result = MatrixLiteral(matD.toTypedPipe.map { case (r, c, x) => (r, c, x * x) }, this.sizeHint) .sumColVectors @@ -146,9 +146,9 @@ sealed trait Matrix2[R, C, V] extends Serializable { * Row L1 normalization (can only be called for Double) * After this operation, the sum(|x|) alone each row will be 1. */ - def rowL1Normalize(implicit ev: =:=[V, Double], mj: MatrixJoiner2): Matrix2[R, C, Double] = { - val matD = this.asInstanceOf[Matrix2[R, C, Double]] - lazy val result = MatrixLiteral(matD.toTypedPipe.map { case (r, c, x) => (r, c, x.abs) }, this.sizeHint) + def rowL1Normalize(implicit num: Numeric[V], mj: MatrixJoiner2): Matrix2[R, C, Double] = { + val matD = MatrixLiteral(this.asInstanceOf[Matrix2[R, C, V]].toTypedPipe.map{ case (r, c, x) => (r, c, num.toDouble(x).abs) }, this.sizeHint) + lazy val result = matD .sumColVectors .toTypedPipe .map { case (r, c, x) => (r, r, 1 / x) } // diagonal + inverse From eb0b4475f4c5be8b6668db6a2b8249a6d69a9ebe Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Thu, 26 Feb 2015 16:11:05 -0800 Subject: [PATCH 041/177] Add test for Long type --- .../scalding/mathematics/Matrix2Test.scala | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/scalding-core/src/test/scala/com/twitter/scalding/mathematics/Matrix2Test.scala b/scalding-core/src/test/scala/com/twitter/scalding/mathematics/Matrix2Test.scala index 541abb80d8..a8269b480e 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/mathematics/Matrix2Test.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/mathematics/Matrix2Test.scala @@ -232,8 +232,16 @@ class Matrix2Normalize(args: Args) extends Job(args) { val tp1 = p1.toTypedPipe[(Int, Int, Double)](('x1, 'y1, 'v1)) val mat1 = MatrixLiteral(tp1, NoClue) + // Now test for the case when value is Long type val matL1Norm = mat1.rowL1Normalize matL1Norm.write(TypedTsv[(Int, Int, Double)]("normalized")) + + val p2: Pipe = Tsv("mat2", ('x2, 'y2, 'v2)).read // test Long type as value is OK + val tp2 = p2.toTypedPipe[(Int, Int, Long)](('x2, 'y2, 'v2)) + val mat2 = MatrixLiteral(tp2, NoClue) + + val mat2L1Norm = mat2.rowL1Normalize + mat2L1Norm.write(TypedTsv[(Int, Int, Double)]("long_normalized")) } class Scalar2Ops(args: Args) extends Job(args) { @@ -452,11 +460,18 @@ class Matrix2Test extends WordSpec with Matchers { TUtil.printStack { JobTest(new Matrix2Normalize(_)) .source(Tsv("mat1", ('x1, 'y1, 'v1)), List((1, 1, 4.0), (1, 2, 1.0), (2, 2, 1.0), (3, 1, 1.0), (3, 2, 3.0), (3, 3, 4.0))) + .source(Tsv("mat2", ('x2, 'y2, 'v2)), List((1, 1, 4L), (1, 2, 1L), (2, 2, 1L), (3, 1, 1L), (3, 2, 3L), (3, 3, 4L))) .typedSink(TypedTsv[(Int, Int, Double)]("normalized")) { ob => - "correctly compute cosine similarity" in { + "correctly compute l1 normalization for matrix with double values" in { toSparseMat(ob) shouldBe Map((1, 1) -> 0.8, (1, 2) -> 0.2, (2, 2) -> 1.0, (3, 1) -> 0.125, (3, 2) -> 0.375, (3, 3) -> 0.5) } } + .typedSink(TypedTsv[(Int, Int, Double)]("long_normalized")){ ob => + "correctly compute l1 normalization for matrix with long values" in { + toSparseMat(ob) shouldBe Map((1, 1) -> 0.8, (1, 2) -> 0.2, (2, 2) -> 1.0, (3, 1) -> 0.125, (3, 2) -> 0.375, (3, 3) -> 0.5) + } + + } .runHadoop .finish } From 6385899207d4fce8f0cfca9f02403a25b3344b75 Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Thu, 26 Feb 2015 14:13:10 -1000 Subject: [PATCH 042/177] make staticSize/dynamicSize required (finds several bugs) --- .../com/twitter/scalding/serialization/Boxed.scala | 2 ++ .../serialization/OrderedSerialization.scala | 2 ++ .../scalding/serialization/Serialization.scala | 4 ++-- .../scalding/serialization/Serialization2.scala | 12 +++++++++++- .../serialization/StringOrderedSerialization.scala | 7 +++++++ .../serialization/SerializationProperties.scala | 2 ++ 6 files changed, 26 insertions(+), 3 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Boxed.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Boxed.scala index b1c31ca28d..edf2eb97ce 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Boxed.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Boxed.scala @@ -347,6 +347,8 @@ case class BoxedOrderedSerialization[K](box: K => Boxed[K], override def compareBinary(a: InputStream, b: InputStream) = ord.compareBinary(a, b) override def read(from: InputStream) = ord.read(from).map(box) override def write(into: OutputStream, bk: Boxed[K]) = ord.write(into, bk.get) + override def staticSize = ord.staticSize + override def dynamicSize(k: Boxed[K]) = ord.dynamicSize(k.get) } object Boxed { diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/OrderedSerialization.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/OrderedSerialization.scala index f2b2e71c58..708986a870 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/OrderedSerialization.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/OrderedSerialization.scala @@ -164,4 +164,6 @@ final case class DeserializingOrderedSerialization[T](serialization: Serializati catch { case NonFatal(e) => OrderedSerialization.CompareFailure(e) } + final override def staticSize = serialization.staticSize + final override def dynamicSize(t: T) = serialization.dynamicSize(t) } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization.scala index fcc419d21f..4de221d9f5 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization.scala @@ -47,12 +47,12 @@ trait Serialization[T] extends Equiv[T] with Hashing[T] with Serializable { * Some on dynamicSize so callers don't need to check both when * they have an instance. */ - def staticSize: Option[Int] = None + def staticSize: Option[Int] /** * returns Some if the size is cheap to calculate. * otherwise the caller should just serialize into an ByteArrayOutputStream */ - def dynamicSize(t: T): Option[Int] = None + def dynamicSize(t: T): Option[Int] } object Serialization { diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization2.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization2.scala index 2fdb7c9989..3acd4635bc 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization2.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization2.scala @@ -21,7 +21,6 @@ import java.io.{ ByteArrayInputStream, ByteArrayOutputStream, InputStream, Outpu import scala.util.{ Failure, Success, Try } class Serialization2[A, B](val serA: Serialization[A], val serB: Serialization[B]) extends Serialization[(A, B)] { - private val MAX_PRIME = Int.MaxValue // turns out MaxValue is a prime, which we want below override def hash(x: (A, B)) = { import MurmurHashUtils._ val h1 = mixH1(seed, serA.hash(x._1)) @@ -46,6 +45,17 @@ class Serialization2[A, B](val serA: Serialization[A], val serB: Serialization[B if (resA.isSuccess) serB.write(out, a._2) else resA } + + override val staticSize = for { + a <- serA.staticSize + b <- serB.staticSize + } yield a + b + + override def dynamicSize(t: (A, B)) = if (staticSize.isDefined) staticSize + else for { + a <- serA.dynamicSize(t._1) + b <- serB.dynamicSize(t._2) + } yield a + b } class OrderedSerialization2[A, B](val ordA: OrderedSerialization[A], diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/StringOrderedSerialization.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/StringOrderedSerialization.scala index e6f4fc883b..c8c527d551 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/StringOrderedSerialization.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/StringOrderedSerialization.scala @@ -115,4 +115,11 @@ class StringOrderedSerialization extends OrderedSerialization[String] { } catch { case NonFatal(e) => OrderedSerialization.CompareFailure(e) } + /** + * generally there is no way to see how big a utf-8 string is without serializing. + * We could scan looking for all ascii characters, but it's hard to see if + * we'd get the balance right. + */ + override def staticSize = None + override def dynamicSize(s: String) = None } diff --git a/scalding-core/src/test/scala/com/twitter/scalding/serialization/SerializationProperties.scala b/scalding-core/src/test/scala/com/twitter/scalding/serialization/SerializationProperties.scala index a73ea684dc..c382e3c554 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/serialization/SerializationProperties.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/serialization/SerializationProperties.scala @@ -50,6 +50,8 @@ object SerializationProperties extends Properties("SerializationProperties") { def compare(a: Int, b: Int) = java.lang.Integer.compare(a, b) def compareBinary(a: InputStream, b: InputStream) = readThenCompare(a, b)(this) + val staticSize = Some(4) + def dynamicSize(i: Int) = staticSize } implicit val stringOrdSer: OrderedSerialization[String] = new StringOrderedSerialization From 2ddb1aecd377559cc384b488a6525a873c6b9f2a Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Thu, 26 Feb 2015 16:32:27 -0800 Subject: [PATCH 043/177] Remove comment --- .../main/scala/com/twitter/scalding/mathematics/Matrix2.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix2.scala b/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix2.scala index 77e1e59aac..4bc7759d58 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix2.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix2.scala @@ -130,7 +130,7 @@ sealed trait Matrix2[R, C, V] extends Serializable { } /** - * Row L2 normalization (can only be called for Double) + * Row L2 normalization * After this operation, the sum(|x|^2) along each row will be 1. */ def rowL2Normalize(implicit num: Numeric[V], mj: MatrixJoiner2): Matrix2[R, C, Double] = { @@ -143,7 +143,7 @@ sealed trait Matrix2[R, C, V] extends Serializable { } /** - * Row L1 normalization (can only be called for Double) + * Row L1 normalization * After this operation, the sum(|x|) alone each row will be 1. */ def rowL1Normalize(implicit num: Numeric[V], mj: MatrixJoiner2): Matrix2[R, C, Double] = { From c73589f9bc40642beb6dbaf0a930e4b1bf7d8801 Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Fri, 27 Feb 2015 12:13:09 -1000 Subject: [PATCH 044/177] Fix a incorrect error check --- .../twitter/scalding/serialization/JavaStreamEnrichments.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/JavaStreamEnrichments.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/JavaStreamEnrichments.scala index 1fa0ca3233..36c282be0f 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/JavaStreamEnrichments.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/JavaStreamEnrichments.scala @@ -42,7 +42,7 @@ object JavaStreamEnrichments { * you will write */ class ArrayWrappingOutputStream(val buffer: Array[Byte], initPos: Int) extends OutputStream { - require(buffer.length <= initPos, + require(buffer.length >= initPos, s"Initial position cannot be more than length: $initPos > ${buffer.length}") private[this] var pos = initPos def position: Int = pos From 064c63373444b654da9acf22a8aa498806619617 Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Fri, 27 Feb 2015 16:41:19 -1000 Subject: [PATCH 045/177] use a Map rather than an Iterable for lookups --- .../scalding/serialization/WrappedSerialization.scala | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/WrappedSerialization.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/WrappedSerialization.scala index ee5eccb56e..b2c1e13aa9 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/WrappedSerialization.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/WrappedSerialization.scala @@ -32,7 +32,7 @@ class WrappedSerialization[T] extends HSerialization[T] with Configurable { import WrappedSerialization.ClassSerialization private var conf: Option[Configuration] = None - private var serializations: Iterable[ClassSerialization[_]] = Nil + private var serializations: Map[Class[_], Serialization[_]] = Map.empty override def getConf: Configuration = conf.get override def setConf(config: Configuration) { @@ -40,11 +40,10 @@ class WrappedSerialization[T] extends HSerialization[T] with Configurable { serializations = WrappedSerialization.getBinary(config) } - def accept(c: Class[_]): Boolean = - serializations.exists { case (cls, _) => cls == c } + def accept(c: Class[_]): Boolean = serializations.contains(c) def getSerialization(c: Class[T]): Option[Serialization[T]] = - serializations.collectFirst { case (cls, b) if cls == c => b } + serializations.get(c) // This cast should never fail since we matched the class .asInstanceOf[Option[Serialization[T]]] @@ -102,7 +101,7 @@ object WrappedSerialization { def setBinary(conf: Configuration, bufs: Iterable[ClassSerialization[_]]): Unit = rawSetBinary(bufs, { case (k, v) => conf.set(k, v) }) - def getBinary(conf: Configuration): Iterable[ClassSerialization[_]] = + def getBinary(conf: Configuration): Map[Class[_], Serialization[_]] = conf .iterator .asScala @@ -119,5 +118,5 @@ object WrappedSerialization { deser(conf.getClassByName(className)) case _ => sys.error(s"ill formed bufferables: ${clsbuf}") } - }.toList + }.toMap } From 2913583312ae8d51c540c07351698ce079b21f01 Mon Sep 17 00:00:00 2001 From: "adam.kozuch@gmail.com" Date: Sun, 1 Mar 2015 08:44:28 +0100 Subject: [PATCH 046/177] remove warnings --- .../com/twitter/scalding/CumulativeSum.scala | 19 ++++++++++++------- .../scalding/serialization/KryoHadoop.scala | 2 +- 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/CumulativeSum.scala b/scalding-core/src/main/scala/com/twitter/scalding/CumulativeSum.scala index f0555b650e..e0e617244e 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/CumulativeSum.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/CumulativeSum.scala @@ -37,11 +37,13 @@ object CumulativeSum { val pipe: TypedPipe[(K, (U, V))]) { /** Takes a sortable field and a monoid and returns the cumulative sum of that monoid **/ def cumulativeSum( - implicit sg: Semigroup[V], ordU: Ordering[U], ordK: Ordering[K]): SortedGrouped[K, (U, V)] = { + implicit sg: Semigroup[V], + ordU: Ordering[U], + ordK: Ordering[K]): SortedGrouped[K, (U, V)] = { pipe.group - .sortBy { case (u: U, _) => u } + .sortBy { case (u, _) => u } .scanLeft(Nil: List[(U, V)]) { - case (acc, (u: U, v: V)) => + case (acc, (u, v)) => acc match { case List((previousU, previousSum)) => List((u, sg.plus(previousSum, v))) case _ => List((u, v)) @@ -49,6 +51,7 @@ object CumulativeSum { } .flattenValues } + /** * An optimization of cumulativeSum for cases when a particular key has many * entries. Requires a sortable partitioning of U. @@ -63,7 +66,7 @@ object CumulativeSum { ordK: Ordering[K]): TypedPipe[(K, (U, V))] = { val sumPerS = pipe - .map { case (k, (u: U, v: V)) => (k, partition(u)) -> v } + .map { case (k, (u, v)) => (k, partition(u)) -> v } .sumByKey .map { case ((k, s), v) => (k, (s, v)) } .group @@ -77,17 +80,19 @@ object CumulativeSum { case _ => Some((None, v, s)) } } - .flatMap{ + .flatMap { case (k, maybeAcc) => for ( acc <- maybeAcc; previousSum <- acc._1 - ) yield { (k, acc._3) -> (None, previousSum) } + ) yield { + (k, acc._3) -> (None, previousSum) + } } val summands = pipe .map { - case (k, (u: U, v: V)) => + case (k, (u, v)) => (k, partition(u)) -> (Some(u), v) } ++ sumPerS diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala index 5dbe4241ab..5b9a391f78 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala @@ -61,7 +61,7 @@ class KryoHadoop(config: Config) extends KryoInstantiator { newK.register(classOf[DateRange], new DateRangeSerializer()) newK.register(classOf[Args], new ArgsSerializer) // Some of the monoids from Algebird that we use: - newK.register(classOf[com.twitter.algebird.AveragedValue], new AveragedValueSerializer) + newK.register(classOf[com.twitter.algebird.AveragedValue], new AveragedValueSerializer) newK.register(classOf[com.twitter.algebird.DecayedValue], new DecayedValueSerializer) newK.register(classOf[com.twitter.algebird.HyperLogLogMonoid], new HLLMonoidSerializer) newK.register(classOf[com.twitter.algebird.Moments], new MomentsSerializer) From 783a832ab5f64f9009a5e673ceb11db363c72a4f Mon Sep 17 00:00:00 2001 From: "adam.kozuch@gmail.com" Date: Sun, 1 Mar 2015 08:50:55 +0100 Subject: [PATCH 047/177] fix --- .../main/scala/com/twitter/scalding/CumulativeSum.scala | 7 ++----- .../com/twitter/scalding/serialization/KryoHadoop.scala | 4 ++-- 2 files changed, 4 insertions(+), 7 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/CumulativeSum.scala b/scalding-core/src/main/scala/com/twitter/scalding/CumulativeSum.scala index e0e617244e..2513854721 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/CumulativeSum.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/CumulativeSum.scala @@ -51,7 +51,6 @@ object CumulativeSum { } .flattenValues } - /** * An optimization of cumulativeSum for cases when a particular key has many * entries. Requires a sortable partitioning of U. @@ -80,14 +79,12 @@ object CumulativeSum { case _ => Some((None, v, s)) } } - .flatMap { + .flatMap{ case (k, maybeAcc) => for ( acc <- maybeAcc; previousSum <- acc._1 - ) yield { - (k, acc._3) -> (None, previousSum) - } + ) yield { (k, acc._3) -> (None, previousSum) } } val summands = pipe diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala index 5b9a391f78..6bef051a8d 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala @@ -40,7 +40,7 @@ import com.twitter.scalding.Args import com.twitter.chill.algebird._ import com.twitter.chill.config.Config -import com.twitter.chill.{SingletonSerializer, ScalaKryoInstantiator, KryoInstantiator} +import com.twitter.chill.{ SingletonSerializer, ScalaKryoInstantiator, KryoInstantiator } class KryoHadoop(config: Config) extends KryoInstantiator { /** @@ -61,7 +61,7 @@ class KryoHadoop(config: Config) extends KryoInstantiator { newK.register(classOf[DateRange], new DateRangeSerializer()) newK.register(classOf[Args], new ArgsSerializer) // Some of the monoids from Algebird that we use: - newK.register(classOf[com.twitter.algebird.AveragedValue], new AveragedValueSerializer) + newK.register(classOf[com.twitter.algebird.AveragedValue], new AveragedValueSerializer) newK.register(classOf[com.twitter.algebird.DecayedValue], new DecayedValueSerializer) newK.register(classOf[com.twitter.algebird.HyperLogLogMonoid], new HLLMonoidSerializer) newK.register(classOf[com.twitter.algebird.Moments], new MomentsSerializer) From 2e45155556493e7d8f3d23e648c77abc97167996 Mon Sep 17 00:00:00 2001 From: "adam.kozuch@gmail.com" Date: Sun, 1 Mar 2015 08:52:30 +0100 Subject: [PATCH 048/177] fix --- .../scala/com/twitter/scalding/serialization/KryoHadoop.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala index 6bef051a8d..5dbe4241ab 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala @@ -40,7 +40,7 @@ import com.twitter.scalding.Args import com.twitter.chill.algebird._ import com.twitter.chill.config.Config -import com.twitter.chill.{ SingletonSerializer, ScalaKryoInstantiator, KryoInstantiator } +import com.twitter.chill.{SingletonSerializer, ScalaKryoInstantiator, KryoInstantiator} class KryoHadoop(config: Config) extends KryoInstantiator { /** From 8500ebd571f2f3b513a481b9a9bd8db8c907b08a Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Mon, 2 Mar 2015 16:10:29 -1000 Subject: [PATCH 049/177] Merge with develop --- .../ordered_serialization/ProductLike.scala | 6 +++--- .../TreeOrderedBuf.scala | 10 +++++----- .../providers/ByteBufferOrderedBuf.scala | 10 +++++----- .../providers/StringOrderedBuf.scala | 8 ++++---- .../providers/TraversablesOrderedBuf.scala | 20 +++++++++---------- .../runtime_helpers/TraversableHelpers.scala | 4 ++-- .../macros/MacroOrderingProperties.scala | 4 +++- 7 files changed, 32 insertions(+), 30 deletions(-) diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/ProductLike.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/ProductLike.scala index 17573c740e..3975ffa529 100644 --- a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/ProductLike.scala +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/ProductLike.scala @@ -55,14 +55,14 @@ object ProductLike { val target = freshT("target") q""" val $target = $element.$accessorSymbol - _root_.com.twitter.scalding.serialization.MurmerHashUtils.mixH1($currentHash, ${tBuf.hash(target)}) + _root_.com.twitter.scalding.serialization.MurmurHashUtils.mixH1($currentHash, ${tBuf.hash(target)}) """ } q""" - var $currentHash: Int = _root_.com.twitter.scalding.serialization.MurmerHashUtils.seed + var $currentHash: Int = _root_.com.twitter.scalding.serialization.MurmurHashUtils.seed ..${hashUpdates} - _root_.com.twitter.scalding.serialization.MurmerHashUtils.fmix($currentHash, ${elementData.size}) + _root_.com.twitter.scalding.serialization.MurmurHashUtils.fmix($currentHash, ${elementData.size}) """ } diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/TreeOrderedBuf.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/TreeOrderedBuf.scala index b2f4c5916a..3afb410abf 100644 --- a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/TreeOrderedBuf.scala +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/TreeOrderedBuf.scala @@ -137,7 +137,7 @@ object TreeOrderedBuf { (if ($tempLen.isDefined) { // Avoid a closure here while we are geeking out val innerLen = $tempLen.get - val $lensLen = sizeBytes(innerLen) + val $lensLen = posVarIntSize(innerLen) Some(innerLen + $lensLen) } else None): Option[Int] } @@ -169,7 +169,7 @@ object TreeOrderedBuf { val $baos = new _root_.java.io.ByteArrayOutputStream ${t.put(baos, element)} val $len = $baos.size - $outerbaos.writeSize($len) + $outerbaos.writePosVarInt($len) $baos.writeTo($outerbaos) """ @@ -179,7 +179,7 @@ object TreeOrderedBuf { */ def withLenCalc(lenC: Tree) = q""" val $len = $lenC - $outerbaos.writeSize($len) + $outerbaos.writePosVarInt($len) ${t.put(outerbaos, element)} """ @@ -211,14 +211,14 @@ object TreeOrderedBuf { def readLength(inputStream: TermName) = { t.length(q"e") match { case const: ConstantLengthCalculation[_] => q"${const.toInt}" - case _ => q"$inputStream.readSize" + case _ => q"$inputStream.readPosVarInt" } } def discardLength(inputStream: TermName) = { t.length(q"e") match { case const: ConstantLengthCalculation[_] => q"()" - case _ => q"$inputStream.readSize" + case _ => q"$inputStream.readPosVarInt" } } diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/ByteBufferOrderedBuf.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/ByteBufferOrderedBuf.scala index 5d52749915..3bf0608b32 100644 --- a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/ByteBufferOrderedBuf.scala +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/ByteBufferOrderedBuf.scala @@ -47,8 +47,8 @@ object ByteBufferOrderedBuf { val incr = freshT("incr") val state = freshT("state") q""" - val $lenA: Int = $inputStreamA.readSize - val $lenB: Int = $inputStreamB.readSize + val $lenA: Int = $inputStreamA.readPosVarInt + val $lenB: Int = $inputStreamB.readPosVarInt val $queryLength = _root_.scala.math.min($lenA, $lenB) var $incr = 0 @@ -67,7 +67,7 @@ object ByteBufferOrderedBuf { } override def put(inputStream: ctx.TermName, element: ctx.TermName) = q""" - $inputStream.writeSize($element.remaining) + $inputStream.writePosVarInt($element.remaining) $inputStream.writeBytes($element.array, $element.arrayOffset + $element.position, $element.remaining) """ @@ -75,7 +75,7 @@ object ByteBufferOrderedBuf { val lenA = freshT("lenA") val bytes = freshT("bytes") q""" - val $lenA = $inputStream.readSize + val $lenA = $inputStream.readPosVarInt val $bytes = new Array[Byte]($lenA) $inputStream.readFully($bytes) _root_.java.nio.ByteBuffer.wrap($bytes) @@ -88,7 +88,7 @@ object ByteBufferOrderedBuf { val tmpLen = freshT("tmpLen") FastLengthCalculation(c)(q""" val $tmpLen = $element.remaining - sizeBytes($tmpLen) + $tmpLen + posVarIntSize($tmpLen) + $tmpLen """) } diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/StringOrderedBuf.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/StringOrderedBuf.scala index ab549feba0..7556ab9d89 100644 --- a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/StringOrderedBuf.scala +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/StringOrderedBuf.scala @@ -42,8 +42,8 @@ object StringOrderedBuf { val lenB = freshT("lenB") q""" - val $lenA = $inputStreamA.readSize - val $lenB = $inputStreamB.readSize + val $lenA = $inputStreamA.readPosVarInt + val $lenB = $inputStreamB.readPosVarInt _root_.com.twitter.scalding.serialization.StringOrderedSerialization.binaryIntCompare($lenA, $inputStreamA, $lenB, @@ -59,7 +59,7 @@ object StringOrderedBuf { q""" val $bytes = $element.getBytes("UTF-8") val $len = $bytes.length - $inputStream.writeSize($len) + $inputStream.writePosVarInt($len) if($len > 0) { $inputStream.write($bytes) } @@ -69,7 +69,7 @@ object StringOrderedBuf { val len = freshT("len") val strBytes = freshT("strBytes") q""" - val $len = $inputStream.readSize + val $len = $inputStream.readPosVarInt if($len > 0) { val $strBytes = new Array[Byte]($len) $inputStream.readFully($strBytes) diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/TraversablesOrderedBuf.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/TraversablesOrderedBuf.scala index 36e5b84daa..e7d47c9cff 100644 --- a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/TraversablesOrderedBuf.scala +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/TraversablesOrderedBuf.scala @@ -135,7 +135,7 @@ object TraversablesOrderedBuf { case DoSort => q""" val $len = $element.size - $inputStream.writeSize($len) + $inputStream.writePosVarInt($len) if($len > 0) { val $asArray = $element.toArray[${innerBuf.tpe}] @@ -152,7 +152,7 @@ object TraversablesOrderedBuf { case NoSort => q""" val $len: Int = $element.size - $inputStream.writeSize($len) + $inputStream.writePosVarInt($len) $element.foreach { case $innerElement => ${innerBuf.put(inputStream, innerElement)} } @@ -167,22 +167,22 @@ object TraversablesOrderedBuf { maybeSort match { case NoSort => q""" - var $currentHash: Int = _root_.com.twitter.scalding.serialization.MurmerHashUtils.seed + var $currentHash: Int = _root_.com.twitter.scalding.serialization.MurmurHashUtils.seed var $len = 0 $element.foreach { t => val $target = t $currentHash = - _root_.com.twitter.scalding.serialization.MurmerHashUtils.mixH1($currentHash, ${innerBuf.hash(target)}) + _root_.com.twitter.scalding.serialization.MurmurHashUtils.mixH1($currentHash, ${innerBuf.hash(target)}) // go ahead and compute the length so we don't traverse twice for lists $len += 1 } - _root_.com.twitter.scalding.serialization.MurmerHashUtils.fmix($currentHash, $len) + _root_.com.twitter.scalding.serialization.MurmurHashUtils.fmix($currentHash, $len) """ case DoSort => // We actually don't sort here, which would be expensive, but combine with a commutative operation // so the order that we see items won't matter. For this we use XOR q""" - var $currentHash: Int = _root_.com.twitter.scalding.serialization.MurmerHashUtils.seed + var $currentHash: Int = _root_.com.twitter.scalding.serialization.MurmurHashUtils.seed var $len = 0 $element.foreach { t => val $target = t @@ -190,7 +190,7 @@ object TraversablesOrderedBuf { $len += 1 } // Might as well be fancy when we mix in the length - _root_.com.twitter.scalding.serialization.MurmerHashUtils.fmix($currentHash, $len) + _root_.com.twitter.scalding.serialization.MurmurHashUtils.fmix($currentHash, $len) """ } } @@ -222,7 +222,7 @@ object TraversablesOrderedBuf { """ } q""" - val $len: Int = $inputStream.readSize + val $len: Int = $inputStream.readPosVarInt if($len > 0) { if($len == 1) { val $firstVal: $innerType = ${innerBuf.get(inputStream)} @@ -262,7 +262,7 @@ object TraversablesOrderedBuf { innerBuf.length(q"$element.head") match { case const: ConstantLengthCalculation[_] => FastLengthCalculation(c)(q"""{ - sizeBytes($element.size) + $element.size * ${const.toInt} + posVarIntSize($element.size) + $element.size * ${const.toInt} }""") case m: MaybeLengthCalculation[_] => val maybeRes = freshT("maybeRes") @@ -274,7 +274,7 @@ object TraversablesOrderedBuf { val maybeRes = ${m.asInstanceOf[MaybeLengthCalculation[c.type]].t} maybeRes match { case _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.ConstLen(constSize) => - val sizeOverhead = sizeBytes($element.size) + val sizeOverhead = posVarIntSize($element.size) _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.DynamicLen(constSize * $element.size + sizeOverhead) // todo maybe we should support this case diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/runtime_helpers/TraversableHelpers.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/runtime_helpers/TraversableHelpers.scala index 8939204847..4cbb5ebba4 100644 --- a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/runtime_helpers/TraversableHelpers.scala +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/runtime_helpers/TraversableHelpers.scala @@ -22,8 +22,8 @@ object TraversableHelpers { import com.twitter.scalding.serialization.JavaStreamEnrichments._ final def rawCompare(inputStreamA: InputStream, inputStreamB: InputStream)(consume: (InputStream, InputStream) => Int): Int = { - val lenA = inputStreamA.readSize - val lenB = inputStreamB.readSize + val lenA = inputStreamA.readPosVarInt + val lenB = inputStreamB.readPosVarInt val minLen = math.min(lenA, lenB) var incr = 0 diff --git a/scalding-macros/src/test/scala/com/twitter/scalding/macros/MacroOrderingProperties.scala b/scalding-macros/src/test/scala/com/twitter/scalding/macros/MacroOrderingProperties.scala index 0a1b11eb7c..ec37d7eecc 100644 --- a/scalding-macros/src/test/scala/com/twitter/scalding/macros/MacroOrderingProperties.scala +++ b/scalding-macros/src/test/scala/com/twitter/scalding/macros/MacroOrderingProperties.scala @@ -99,7 +99,9 @@ object MacroOpaqueContainer { override def write(b: OutputStream, s: MacroOpaqueContainer) = intOrderedSerialization.write(b, s.myField) override def compareBinary(lhs: InputStream, rhs: InputStream) = intOrderedSerialization.compareBinary(lhs, rhs) - override def staticSize = Some(4) + override val staticSize = Some(4) + + override def dynamicSize(i: MacroOpaqueContainer) = staticSize } implicit def arbitraryMacroOpaqueContainer: Arbitrary[MacroOpaqueContainer] = Arbitrary { From 58b5465e4f73b8b7a19f916f829943795550f581 Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Tue, 3 Mar 2015 11:30:22 -0800 Subject: [PATCH 050/177] Address Oscar's review --- .../com/twitter/scalding/mathematics/Matrix2.scala | 6 +++--- .../twitter/scalding/mathematics/Matrix2Test.scala | 12 ++++++------ 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix2.scala b/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix2.scala index 4bc7759d58..4797d51a5e 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix2.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix2.scala @@ -134,8 +134,8 @@ sealed trait Matrix2[R, C, V] extends Serializable { * After this operation, the sum(|x|^2) along each row will be 1. */ def rowL2Normalize(implicit num: Numeric[V], mj: MatrixJoiner2): Matrix2[R, C, Double] = { - val matD = this.asInstanceOf[Matrix2[R, C, Double]] - lazy val result = MatrixLiteral(matD.toTypedPipe.map { case (r, c, x) => (r, c, x * x) }, this.sizeHint) + val matD = MatrixLiteral(this.toTypedPipe.map{ case (r, c, x) => (r, c, num.toDouble(x).abs) }, this.sizeHint) + lazy val result = MatrixLiteral(this.toTypedPipe.map { case (r, c, x) => (r, c, num.toDouble(x) * num.toDouble(x)) }, this.sizeHint) .sumColVectors .toTypedPipe .map { case (r, c, x) => (r, r, 1 / scala.math.sqrt(x)) } // diagonal + inverse @@ -147,7 +147,7 @@ sealed trait Matrix2[R, C, V] extends Serializable { * After this operation, the sum(|x|) alone each row will be 1. */ def rowL1Normalize(implicit num: Numeric[V], mj: MatrixJoiner2): Matrix2[R, C, Double] = { - val matD = MatrixLiteral(this.asInstanceOf[Matrix2[R, C, V]].toTypedPipe.map{ case (r, c, x) => (r, c, num.toDouble(x).abs) }, this.sizeHint) + val matD = MatrixLiteral(this.toTypedPipe.map{ case (r, c, x) => (r, c, num.toDouble(x).abs) }, this.sizeHint) lazy val result = matD .sumColVectors .toTypedPipe diff --git a/scalding-core/src/test/scala/com/twitter/scalding/mathematics/Matrix2Test.scala b/scalding-core/src/test/scala/com/twitter/scalding/mathematics/Matrix2Test.scala index a8269b480e..39e880b958 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/mathematics/Matrix2Test.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/mathematics/Matrix2Test.scala @@ -228,16 +228,16 @@ class Matrix2Normalize(args: Args) extends Job(args) { import cascading.tuple.Fields import com.twitter.scalding.TDsl._ - val p1: Pipe = Tsv("mat1", ('x1, 'y1, 'v1)).read - val tp1 = p1.toTypedPipe[(Int, Int, Double)](('x1, 'y1, 'v1)) + val tp1 = TypedPipe.from(TypedTsv[(Int, Int, Double)]("mat1")) val mat1 = MatrixLiteral(tp1, NoClue) // Now test for the case when value is Long type val matL1Norm = mat1.rowL1Normalize matL1Norm.write(TypedTsv[(Int, Int, Double)]("normalized")) - val p2: Pipe = Tsv("mat2", ('x2, 'y2, 'v2)).read // test Long type as value is OK - val tp2 = p2.toTypedPipe[(Int, Int, Long)](('x2, 'y2, 'v2)) + //val p2: Pipe = Tsv("mat2", ('x2, 'y2, 'v2)).read // test Long type as value is OK + val tp2 = TypedPipe.from(TypedTsv[(Int, Int, Long)]("mat2")) + //val tp2 = p2.toTypedPipe[(Int, Int, Long)](('x2, 'y2, 'v2)) val mat2 = MatrixLiteral(tp2, NoClue) val mat2L1Norm = mat2.rowL1Normalize @@ -459,8 +459,8 @@ class Matrix2Test extends WordSpec with Matchers { "A Matrix2 Normalize job" should { TUtil.printStack { JobTest(new Matrix2Normalize(_)) - .source(Tsv("mat1", ('x1, 'y1, 'v1)), List((1, 1, 4.0), (1, 2, 1.0), (2, 2, 1.0), (3, 1, 1.0), (3, 2, 3.0), (3, 3, 4.0))) - .source(Tsv("mat2", ('x2, 'y2, 'v2)), List((1, 1, 4L), (1, 2, 1L), (2, 2, 1L), (3, 1, 1L), (3, 2, 3L), (3, 3, 4L))) + .source(TypedTsv[(Int, Int, Double)]("mat1"), List((1, 1, 4.0), (1, 2, 1.0), (2, 2, 1.0), (3, 1, 1.0), (3, 2, 3.0), (3, 3, 4.0))) + .source(TypedTsv[(Int, Int, Long)]("mat2"), List((1, 1, 4L), (1, 2, 1L), (2, 2, 1L), (3, 1, 1L), (3, 2, 3L), (3, 3, 4L))) .typedSink(TypedTsv[(Int, Int, Double)]("normalized")) { ob => "correctly compute l1 normalization for matrix with double values" in { toSparseMat(ob) shouldBe Map((1, 1) -> 0.8, (1, 2) -> 0.2, (2, 2) -> 1.0, (3, 1) -> 0.125, (3, 2) -> 0.375, (3, 3) -> 0.5) From 920caf3171ea400274b0026e681ecd1dabd0a20d Mon Sep 17 00:00:00 2001 From: Avi Bryant Date: Tue, 3 Mar 2015 15:15:44 -1000 Subject: [PATCH 051/177] add make(dest) to TypedPipe --- .../com/twitter/scalding/typed/TypedPipe.scala | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipe.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipe.scala index 0b2f918dc0..75d4e32dac 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipe.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipe.scala @@ -539,6 +539,20 @@ trait TypedPipe[+T] extends Serializable { writeExecution(dest) .map(_ => TypedPipe.from(dest)) + /** + * If you want to writeThrough to a specific location if it doesn't already exist, + * and otherwise just read from it going forward, use this. + */ + def make[U >: T](dest: Source with TypedSink[T] with TypedSource[U]): Execution[TypedPipe[U]] = + Execution.getMode.flatMap { mode => + try { + dest.validateTaps(mode) + Execution.from(TypedPipe.from(dest)) + } catch { + case ivs: InvalidSourceException => writeThrough(dest) + } + } + /** Just keep the keys, or ._1 (if this type is a Tuple2) */ def keys[K](implicit ev: <:<[T, (K, Any)]): TypedPipe[K] = // avoid capturing ev in the closure: From 511a1231b80f0247b69bc782c13b38fe42d65ec3 Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Tue, 3 Mar 2015 17:20:35 -1000 Subject: [PATCH 052/177] Add scalameter benchmark, 4x faster than Kryo for Ints --- project/Build.scala | 31 +++++++----- .../com/twitter/scalding/Serialization.scala | 49 +++++++++++++++++++ 2 files changed, 69 insertions(+), 11 deletions(-) create mode 100644 scalding-benchmarks/src/test/scala/com/twitter/scalding/Serialization.scala diff --git a/project/Build.scala b/project/Build.scala index cd5c003c82..7f4821ae8b 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -20,26 +20,27 @@ object ScaldingBuild extends Build { } def isScala210x(scalaVersion: String) = scalaBinaryVersion(scalaVersion) == "2.10" - val scalaTestVersion = "2.2.2" - val scalaCheckVersion = "1.12.1" - val hadoopVersion = "1.2.1" val algebirdVersion = "0.9.0" + val avroVersion = "1.7.4" val bijectionVersion = "0.7.2" + val cascadingAvroVersion = "2.1.2" val chillVersion = "0.5.2" - val slf4jVersion = "1.6.6" - val parquetVersion = "1.6.0rc4" val dfsDatastoresVersion = "1.3.4" + val elephantbirdVersion = "4.6" + val hadoopLzoVersion = "0.4.16" + val hadoopVersion = "1.2.1" val hbaseVersion = "0.94.10" val hravenVersion = "0.9.13" val jacksonVersion = "2.4.2" - val protobufVersion = "2.4.1" - val elephantbirdVersion = "4.6" - val hadoopLzoVersion = "0.4.16" - val thriftVersion = "0.5.0" - val cascadingAvroVersion = "2.1.2" - val avroVersion = "1.7.4" val json4SVersion = "3.2.11" + val parquetVersion = "1.6.0rc4" + val protobufVersion = "2.4.1" + val scalaCheckVersion = "1.12.1" + val scalaTestVersion = "2.2.2" + val scalameterVersion = "0.6" val scroogeVersion = "3.17.0" + val slf4jVersion = "1.6.6" + val thriftVersion = "0.5.0" val printDependencyClasspath = taskKey[Unit]("Prints location of the dependencies") @@ -244,6 +245,14 @@ object ScaldingBuild extends Build { lazy val cascadingJDBCVersion = System.getenv.asScala.getOrElse("SCALDING_CASCADING_JDBC_VERSION", "2.6.0") + lazy val scaldingBenchmarks = module("benchmarks").settings( + libraryDependencies ++= Seq( + "com.storm-enroute" %% "scalameter" % scalameterVersion % "test" + ), + testFrameworks += new TestFramework("org.scalameter.ScalaMeterFramework"), + parallelExecution in Test := false + ).dependsOn(scaldingCore, scaldingMacros) + lazy val scaldingCore = module("core").settings( libraryDependencies ++= Seq( "cascading" % "cascading-core" % cascadingVersion, diff --git a/scalding-benchmarks/src/test/scala/com/twitter/scalding/Serialization.scala b/scalding-benchmarks/src/test/scala/com/twitter/scalding/Serialization.scala new file mode 100644 index 0000000000..d0c6fc3ee1 --- /dev/null +++ b/scalding-benchmarks/src/test/scala/com/twitter/scalding/Serialization.scala @@ -0,0 +1,49 @@ +package com.twitter.scalding.benchmarks + +import scala.language.experimental.macros +import com.twitter.scalding.serialization._ +import com.twitter.chill.KryoPool + +import org.scalameter.api._ + +trait LowerPriorityImplicit { + implicit def ordBuf[T]: OrderedSerialization[T] = + macro com.twitter.scalding.macros.impl.OrderedSerializationProviderImpl[T] +} + +object SerializationBenchmark extends PerformanceTest.Quickbenchmark with LowerPriorityImplicit { + val sizes = Gen.range("size")(300000, 1500000, 300000) + + val ranges: Gen[List[Int]] = for { + size <- sizes + } yield (0 until size).toList + + def roundTrip[T:Serialization](ts: Iterator[T]): Unit = + ts.map { t => + Serialization.fromBytes(Serialization.toBytes(t)).get + }.foreach(_ => ()) + + def kryoRoundTrip[T](k: KryoPool, ts: Iterator[T]): Unit = + ts.map { t => k.fromBytes(k.toBytesWithClass(t)) } + .foreach(_ => ()) + + performance of "Serialization" in { + measure method "typeclass" in { + using(ranges) in { l => roundTrip(l.iterator) } + } + measure method "kryo" in { + val kryo = KryoPool.withByteArrayOutputStream(1, + com.twitter.scalding.Config.default.getKryo.get) + + using(ranges) in { l => kryoRoundTrip(kryo, l.iterator) } + } + + /** + * TODO: + * 1) simple case class + * 2) case class with some nesting and collections + * 3) sorting of an Array[Array[Byte]] using OrderedSerialization vs Array[T] + * 4) fastest binary sorting for strings (byte-by-byte, longs, etc...) + */ + } +} From b5475dae1391c2adaa70d1a6ceb1a66415a6d477 Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Wed, 4 Mar 2015 16:41:36 -1000 Subject: [PATCH 053/177] Add sorting benchmarks --- project/Build.scala | 3 +- .../com/twitter/scalding/Serialization.scala | 121 ++++++++++++++++-- .../providers/StringOrderedBuf.scala | 38 +++++- 3 files changed, 142 insertions(+), 20 deletions(-) diff --git a/project/Build.scala b/project/Build.scala index 7f4821ae8b..d3e4a80a88 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -247,7 +247,8 @@ object ScaldingBuild extends Build { lazy val scaldingBenchmarks = module("benchmarks").settings( libraryDependencies ++= Seq( - "com.storm-enroute" %% "scalameter" % scalameterVersion % "test" + "com.storm-enroute" %% "scalameter" % scalameterVersion % "test", + "org.scalacheck" %% "scalacheck" % scalaCheckVersion % "test" ), testFrameworks += new TestFramework("org.scalameter.ScalaMeterFramework"), parallelExecution in Test := false diff --git a/scalding-benchmarks/src/test/scala/com/twitter/scalding/Serialization.scala b/scalding-benchmarks/src/test/scala/com/twitter/scalding/Serialization.scala index d0c6fc3ee1..3367f6980c 100644 --- a/scalding-benchmarks/src/test/scala/com/twitter/scalding/Serialization.scala +++ b/scalding-benchmarks/src/test/scala/com/twitter/scalding/Serialization.scala @@ -1,24 +1,64 @@ package com.twitter.scalding.benchmarks -import scala.language.experimental.macros -import com.twitter.scalding.serialization._ import com.twitter.chill.KryoPool - +import com.twitter.scalding.serialization._ +import java.io.ByteArrayInputStream +import org.scalacheck.{ Gen => scGen, Arbitrary } // We use scalacheck Gens to generate random scalameter gens. import org.scalameter.api._ +import scala.collection.generic.CanBuildFrom +import scala.language.experimental.macros +import scala.util.Sorting trait LowerPriorityImplicit { - implicit def ordBuf[T]: OrderedSerialization[T] = - macro com.twitter.scalding.macros.impl.OrderedSerializationProviderImpl[T] + implicit def ordBuf[T]: OrderedSerialization[T] = macro com.twitter.scalding.macros.impl.OrderedSerializationProviderImpl[T] } object SerializationBenchmark extends PerformanceTest.Quickbenchmark with LowerPriorityImplicit { val sizes = Gen.range("size")(300000, 1500000, 300000) + val smallSizes = Gen.range("size")(30000, 150000, 30000) + + /** + * This tends to create ascii strings + */ + def asciiStringGen: scGen[String] = scGen.parameterized { p => + val thisSize = p.rng.nextInt(p.size + 1) + scGen.const(new String(Array.fill(thisSize)(p.rng.nextInt(128).toByte))) + } + def charStringGen: scGen[String] = + scGen.listOf(scGen.choose(0.toChar, Char.MaxValue)).map(_.mkString) + + // Biases to ascii 80% of the time + def stringGen: scGen[String] = scGen.frequency((4, asciiStringGen), (1, charStringGen)) + + implicit def stringArb: Arbitrary[String] = Arbitrary(stringGen) + + def collection[T, C[_]](size: Gen[Int])(implicit arbT: Arbitrary[T], cbf: CanBuildFrom[Nothing, T, C[T]]): Gen[C[T]] = + collection[T, C](size, arbT.arbitrary)(cbf) - val ranges: Gen[List[Int]] = for { - size <- sizes - } yield (0 until size).toList + def collection[T, C[_]](size: Gen[Int], item: scGen[T])(implicit cbf: CanBuildFrom[Nothing, T, C[T]]): Gen[C[T]] = + size.map { s => + val builder = cbf() + builder.sizeHint(s) + // Initialize a fixed random number generator + val rng = new scala.util.Random("scalding".hashCode) + val p = scGen.Parameters.default.withRng(rng) - def roundTrip[T:Serialization](ts: Iterator[T]): Unit = + def get(attempt: Int): T = + if (attempt > 1000) sys.error("Failed to generate after 100 tries") + else { + item(p) match { + case None => get(attempt + 1) + case Some(t) => t + } + } + + (0 until s).foreach { _ => + builder += get(0) + } + builder.result() + } + + def roundTrip[T: Serialization](ts: Iterator[T]): Unit = ts.map { t => Serialization.fromBytes(Serialization.toBytes(t)).get }.foreach(_ => ()) @@ -27,15 +67,70 @@ object SerializationBenchmark extends PerformanceTest.Quickbenchmark with LowerP ts.map { t => k.fromBytes(k.toBytesWithClass(t)) } .foreach(_ => ()) + def toArrayOrd[T](t: OrderedSerialization[T]): Ordering[Array[Byte]] = new Ordering[Array[Byte]] { + def compare(a: Array[Byte], b: Array[Byte]) = { + t.compareBinary(new ByteArrayInputStream(a), new ByteArrayInputStream(b)).unsafeToInt + } + } + def toArrayOrd[T](k: KryoPool, ord: Ordering[T]): Ordering[Array[Byte]] = new Ordering[Array[Byte]] { + def compare(a: Array[Byte], b: Array[Byte]) = + ord.compare(k.fromBytes(a).asInstanceOf[T], + k.fromBytes(b).asInstanceOf[T]) + } + performance of "Serialization" in { - measure method "typeclass" in { - using(ranges) in { l => roundTrip(l.iterator) } + measure method "typeclass: Int" in { + using(collection[Int, List](sizes)) in { l => roundTrip(l.iterator) } + } + measure method "kryo: Int" in { + val kryo = KryoPool.withByteArrayOutputStream(1, + com.twitter.scalding.Config.default.getKryo.get) + + using(collection[Int, List](sizes)) in { l => kryoRoundTrip(kryo, l.iterator) } + } + measure method "typeclass: String" in { + using(collection[String, List](smallSizes)) in { l => roundTrip(l.iterator) } + } + measure method "kryo: String" in { + val kryo = KryoPool.withByteArrayOutputStream(1, + com.twitter.scalding.Config.default.getKryo.get) + + using(collection[String, List](smallSizes)) in { l => kryoRoundTrip(kryo, l.iterator) } + } + measure method "typeclass: (Int, (Long, String))" in { + using(collection[(Int, (Long, String)), List](smallSizes)) in { l => roundTrip(l.iterator) } + } + measure method "kryo: (Int, (Long, String))" in { + val kryo = KryoPool.withByteArrayOutputStream(1, + com.twitter.scalding.Config.default.getKryo.get) + + using(collection[(Int, (Long, String)), List](smallSizes)) in { l => kryoRoundTrip(kryo, l.iterator) } + } + measure method "typeclass: (Int, Long, Short)" in { + using(collection[(Int, Long, Short), List](smallSizes)) in { l => roundTrip(l.iterator) } + } + measure method "kryo: (Int, Long, Short)" in { + val kryo = KryoPool.withByteArrayOutputStream(1, + com.twitter.scalding.Config.default.getKryo.get) + + using(collection[(Int, Long, Short), List](smallSizes)) in { l => kryoRoundTrip(kryo, l.iterator) } + } + measure method "sort typeclass: (Int, (Long, String))" in { + val ordSer = implicitly[OrderedSerialization[(Int, (Long, String))]] + using(collection[(Int, (Long, String)), List](smallSizes) + .map { items => + items.map { Serialization.toBytes(_) }.toArray + }) in { ary => Sorting.quickSort(ary)(toArrayOrd(ordSer)) } } - measure method "kryo" in { + measure method "sort kryo: (Int, (Long, String))" in { val kryo = KryoPool.withByteArrayOutputStream(1, com.twitter.scalding.Config.default.getKryo.get) - using(ranges) in { l => kryoRoundTrip(kryo, l.iterator) } + val ord = implicitly[Ordering[(Int, (Long, String))]] + using(collection[(Int, (Long, String)), List](smallSizes) + .map { items => + items.map { kryo.toBytesWithClass(_) }.toArray + }) in { ary => Sorting.quickSort(ary)(toArrayOrd(kryo, ord)) } } /** diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/StringOrderedBuf.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/StringOrderedBuf.scala index 7556ab9d89..afb1f72a2c 100644 --- a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/StringOrderedBuf.scala +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/StringOrderedBuf.scala @@ -55,14 +55,40 @@ object StringOrderedBuf { override def put(inputStream: ctx.TermName, element: ctx.TermName) = { val bytes = freshT("bytes") + val charLen = freshT("charLen") val len = freshT("len") q""" - val $bytes = $element.getBytes("UTF-8") - val $len = $bytes.length - $inputStream.writePosVarInt($len) - if($len > 0) { - $inputStream.write($bytes) - } + // Ascii is very common, so if the string is short, + // we check if it is ascii: + def isShortAscii(size: Int, str: String): Boolean = (size < 65) && { + var pos = 0 + var ascii: Boolean = true + while((pos < size) && ascii) { + ascii = (str.charAt(pos) < 128) + pos += 1 + } + ascii + } + + val $charLen = $element.length + if ($charLen == 0) { + $inputStream.writePosVarInt(0) + } + else if (isShortAscii($charLen, $element)) { + $inputStream.writePosVarInt($charLen) + val $bytes = new Array[Byte]($charLen) + // This deprecated gets ascii bytes out, but is incorrect + // for non-ascii data. + $element.getBytes(0, $charLen, $bytes, 0) + $inputStream.write($bytes) + } + else { + // Just use utf-8 + val $bytes = $element.getBytes("UTF-8") + val $len = $bytes.length + $inputStream.writePosVarInt($len) + $inputStream.write($bytes) + } """ } override def get(inputStream: ctx.TermName): ctx.Tree = { From 71530339e69f61274743e177695ab8ba251cdb94 Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Thu, 5 Mar 2015 11:09:58 -1000 Subject: [PATCH 054/177] Improve unsigned number comparisons based on benchmarks --- .../com/twitter/scalding/Serialization.scala | 150 +++++++++++++++++- .../serialization/JavaStreamEnrichments.scala | 12 +- .../serialization/UnsignedComparisons.scala | 37 ++--- 3 files changed, 170 insertions(+), 29 deletions(-) diff --git a/scalding-benchmarks/src/test/scala/com/twitter/scalding/Serialization.scala b/scalding-benchmarks/src/test/scala/com/twitter/scalding/Serialization.scala index 3367f6980c..cef0a44393 100644 --- a/scalding-benchmarks/src/test/scala/com/twitter/scalding/Serialization.scala +++ b/scalding-benchmarks/src/test/scala/com/twitter/scalding/Serialization.scala @@ -14,6 +14,8 @@ trait LowerPriorityImplicit { } object SerializationBenchmark extends PerformanceTest.Quickbenchmark with LowerPriorityImplicit { + import JavaStreamEnrichments._ + val sizes = Gen.range("size")(300000, 1500000, 300000) val smallSizes = Gen.range("size")(30000, 150000, 30000) @@ -78,7 +80,116 @@ object SerializationBenchmark extends PerformanceTest.Quickbenchmark with LowerP k.fromBytes(b).asInstanceOf[T]) } + val longArrayByte: Gen[Array[Byte]] = + collection[Byte, Array](sizes.map(s => (s / 8) * 8)) + + // This is here to make sure the compiler cannot optimize away reads + var effectInt: Int = 0 + var effectLong: Long = 0L + performance of "Serialization" in { + measure method "JavaStreamEnrichments.readInt" in { + using(longArrayByte) in { a => + val length = a.length + val is = new ByteArrayInputStream(a) + var ints = length / 4 + while (ints > 0) { + effectInt ^= is.readInt + ints -= 1 + } + } + } + measure method "JavaStreamEnrichments.readLong" in { + using(longArrayByte) in { a => + val length = a.length + val is = new ByteArrayInputStream(a) + var longs = length / 8 + while (longs > 0) { + effectLong ^= is.readLong + longs -= 1 + } + } + } + measure method "UnsignedComparisons.unsignedLongCompare" in { + using(collection[Long, Array](sizes)) in { a => + val max = a.length - 1 + var pos = 0 + while (pos < max) { + effectInt ^= UnsignedComparisons.unsignedLongCompare(a(pos), a(pos + 1)) + pos += 2 + } + } + } + measure method "normal long compare" in { + using(collection[Long, Array](sizes)) in { a => + val max = a.length - 1 + var pos = 0 + while (pos < max) { + effectInt ^= java.lang.Long.compare(a(pos), a(pos + 1)) + pos += 2 + } + } + } + measure method "UnsignedComparisons.unsignedInt" in { + using(collection[Int, Array](sizes)) in { a => + val max = a.length - 1 + var pos = 0 + while (pos < max) { + effectInt ^= UnsignedComparisons.unsignedIntCompare(a(pos), a(pos + 1)) + pos += 2 + } + } + } + measure method "normal int compare" in { + using(collection[Int, Array](sizes)) in { a => + val max = a.length - 1 + var pos = 0 + while (pos < max) { + effectInt ^= java.lang.Integer.compare(a(pos), a(pos + 1)) + pos += 2 + } + } + } + measure method "UnsignedComparisons.unsignedShort" in { + using(collection[Short, Array](sizes)) in { a => + val max = a.length - 1 + var pos = 0 + while (pos < max) { + effectInt ^= UnsignedComparisons.unsignedShortCompare(a(pos), a(pos + 1)) + pos += 2 + } + } + } + measure method "normal short compare" in { + using(collection[Short, Array](sizes)) in { a => + val max = a.length - 1 + var pos = 0 + while (pos < max) { + effectInt ^= java.lang.Short.compare(a(pos), a(pos + 1)) + pos += 2 + } + } + } + measure method "UnsignedComparisons.unsignedByte" in { + using(collection[Byte, Array](sizes)) in { a => + val max = a.length - 1 + var pos = 0 + while (pos < max) { + effectInt ^= UnsignedComparisons.unsignedByteCompare(a(pos), a(pos + 1)) + pos += 2 + } + } + } + measure method "normal byte compare" in { + using(collection[Byte, Array](sizes)) in { a => + val max = a.length - 1 + var pos = 0 + while (pos < max) { + effectInt ^= java.lang.Byte.compare(a(pos), a(pos + 1)) + pos += 2 + } + } + } measure method "typeclass: Int" in { using(collection[Int, List](sizes)) in { l => roundTrip(l.iterator) } } @@ -115,12 +226,47 @@ object SerializationBenchmark extends PerformanceTest.Quickbenchmark with LowerP using(collection[(Int, Long, Short), List](smallSizes)) in { l => kryoRoundTrip(kryo, l.iterator) } } + measure method "sort typeclass: Long" in { + val ordSer = implicitly[OrderedSerialization[Long]] + using(collection[Long, List](smallSizes) + .map { items => + items.map { Serialization.toBytes(_) }.toArray + }) in { ary => java.util.Arrays.sort(ary, toArrayOrd(ordSer)) } + } + measure method "sort kryo: Long" in { + val kryo = KryoPool.withByteArrayOutputStream(1, + com.twitter.scalding.Config.default.getKryo.get) + + val ord = implicitly[Ordering[Long]] + using(collection[Long, List](smallSizes) + .map { items => + items.map { kryo.toBytesWithClass(_) }.toArray + }) in { ary => java.util.Arrays.sort(ary, toArrayOrd(kryo, ord)) } + } + measure method "sort typeclass: String" in { + val ordSer = implicitly[OrderedSerialization[String]] + using(collection[String, List](smallSizes) + .map { items => + items.map { Serialization.toBytes(_) }.toArray + }) in { ary => java.util.Arrays.sort(ary, toArrayOrd(ordSer)) } + } + measure method "sort kryo: String" in { + val kryo = KryoPool.withByteArrayOutputStream(1, + com.twitter.scalding.Config.default.getKryo.get) + + val ord = implicitly[Ordering[String]] + using(collection[String, List](smallSizes) + .map { items => + items.map { kryo.toBytesWithClass(_) }.toArray + }) in { ary => java.util.Arrays.sort(ary, toArrayOrd(kryo, ord)) } + } + measure method "sort typeclass: (Int, (Long, String))" in { val ordSer = implicitly[OrderedSerialization[(Int, (Long, String))]] using(collection[(Int, (Long, String)), List](smallSizes) .map { items => items.map { Serialization.toBytes(_) }.toArray - }) in { ary => Sorting.quickSort(ary)(toArrayOrd(ordSer)) } + }) in { ary => java.util.Arrays.sort(ary, toArrayOrd(ordSer)) } } measure method "sort kryo: (Int, (Long, String))" in { val kryo = KryoPool.withByteArrayOutputStream(1, @@ -130,7 +276,7 @@ object SerializationBenchmark extends PerformanceTest.Quickbenchmark with LowerP using(collection[(Int, (Long, String)), List](smallSizes) .map { items => items.map { kryo.toBytesWithClass(_) }.toArray - }) in { ary => Sorting.quickSort(ary)(toArrayOrd(kryo, ord)) } + }) in { ary => java.util.Arrays.sort(ary, toArrayOrd(kryo, ord)) } } /** diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/JavaStreamEnrichments.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/JavaStreamEnrichments.scala index 36c282be0f..29d1097f46 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/JavaStreamEnrichments.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/JavaStreamEnrichments.scala @@ -135,16 +135,24 @@ object JavaStreamEnrichments { def readDouble: Double = java.lang.Double.longBitsToDouble(readLong) def readFloat: Float = java.lang.Float.intBitsToFloat(readInt) + /** + * This is the algorithm from DataInputStream + * it was also benchmarked against the approach + * used in readLong and found to be faster + */ def readInt: Int = { val c1 = s.read val c2 = s.read val c3 = s.read val c4 = s.read - // This is the algorithm from DataInputStream if ((c1 | c2 | c3 | c4) < 0) eof else ((c1 << 24) | (c2 << 16) | (c3 << 8) | c4) } + /* + * This is the algorithm from DataInputStream + * it was also benchmarked against the same approach used + * in readInt (buffer-less) and found to be faster. + */ def readLong: Long = { - // This is the algorithm from DataInputStream val buf = new Array[Byte](8) readFully(buf) (buf(0).toLong << 56) + diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/UnsignedComparisons.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/UnsignedComparisons.scala index a15a0664ec..86c0839c7d 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/UnsignedComparisons.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/UnsignedComparisons.scala @@ -18,32 +18,19 @@ package com.twitter.scalding.serialization object UnsignedComparisons { final def unsignedLongCompare(a: Long, b: Long): Int = if (a == b) 0 else { - // We only get into this block when the a != b, so it has to be the last - // block - val firstBitXor = (a ^ b) & (1L << 63) - // If both are on the same side of zero, normal compare works - if (firstBitXor == 0) java.lang.Long.compare(a, b) - else if (b >= 0) 1 + val xor = (a ^ b) + // If xor >= 0, then a and b are on the same side of zero + if (xor >= 0L) java.lang.Long.compare(a, b) + else if (b >= 0L) 1 else -1 } - final def unsignedIntCompare(a: Int, b: Int): Int = if (a == b) 0 else { - val firstBitXor = (a ^ b) & (1 << 31) - // If both are on the same side of zero, normal compare works - if (firstBitXor == 0) Integer.compare(a, b) - else if (b >= 0) 1 - else -1 - } - final def unsignedShortCompare(a: Short, b: Short): Int = if (a == b) 0 else { - // We have to convert to bytes to Int on JVM to do - // anything anyway, so might as well compare in that space - def fromShort(x: Short): Int = if (x < 0) x + (1 << 16) else x.toInt - Integer.compare(fromShort(a), fromShort(b)) - } - final def unsignedByteCompare(a: Byte, b: Byte): Int = if (a == b) 0 else { - // We have to convert to bytes to Int on JVM to do - // anything anyway, so might as well compare in that space - def fromByte(x: Byte): Int = if (x < 0) x + (1 << 8) else x.toInt - Integer.compare(fromByte(a), fromByte(b)) - } + final def unsignedIntCompare(a: Int, b: Int): Int = + java.lang.Long.compare(a.toLong & 0xFFFFFFFFL, b.toLong & 0xFFFFFFFFL) + + final def unsignedShortCompare(a: Short, b: Short): Int = + Integer.compare(a & 0xFFFF, b & 0xFFFF) + + final def unsignedByteCompare(a: Byte, b: Byte): Int = + Integer.compare(a & 0xFF, b & 0xFF) } From c915360d3a1623c5c7c82e7597681aba8591e081 Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Thu, 5 Mar 2015 13:32:10 -1000 Subject: [PATCH 055/177] Remove equality checking --- .../com/twitter/scalding/Serialization.scala | 17 +++ .../TreeOrderedBuf.scala | 105 +++++++----------- 2 files changed, 59 insertions(+), 63 deletions(-) diff --git a/scalding-benchmarks/src/test/scala/com/twitter/scalding/Serialization.scala b/scalding-benchmarks/src/test/scala/com/twitter/scalding/Serialization.scala index cef0a44393..4420dfe4eb 100644 --- a/scalding-benchmarks/src/test/scala/com/twitter/scalding/Serialization.scala +++ b/scalding-benchmarks/src/test/scala/com/twitter/scalding/Serialization.scala @@ -226,6 +226,23 @@ object SerializationBenchmark extends PerformanceTest.Quickbenchmark with LowerP using(collection[(Int, Long, Short), List](smallSizes)) in { l => kryoRoundTrip(kryo, l.iterator) } } + measure method "sort typeclass: Int" in { + val ordSer = implicitly[OrderedSerialization[Int]] + using(collection[Int, List](smallSizes) + .map { items => + items.map { Serialization.toBytes(_) }.toArray + }) in { ary => java.util.Arrays.sort(ary, toArrayOrd(ordSer)) } + } + measure method "sort kryo: Int" in { + val kryo = KryoPool.withByteArrayOutputStream(1, + com.twitter.scalding.Config.default.getKryo.get) + + val ord = implicitly[Ordering[Int]] + using(collection[Int, List](smallSizes) + .map { items => + items.map { kryo.toBytesWithClass(_) }.toArray + }) in { ary => java.util.Arrays.sort(ary, toArrayOrd(kryo, ord)) } + } measure method "sort typeclass: Long" in { val ordSer = implicitly[OrderedSerialization[Long]] using(collection[Long, List](smallSizes) diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/TreeOrderedBuf.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/TreeOrderedBuf.scala index 3afb410abf..a3bee56827 100644 --- a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/TreeOrderedBuf.scala +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/TreeOrderedBuf.scala @@ -17,7 +17,7 @@ package com.twitter.scalding.macros.impl.ordered_serialization import com.twitter.scalding._ import com.twitter.scalding.serialization.OrderedSerialization -import com.twitter.scalding.serialization.PositionInputStream +import com.twitter.scalding.serialization.JavaStreamEnrichments import java.io.InputStream import scala.reflect.macros.Context import scala.language.experimental.macros @@ -35,63 +35,36 @@ object CommonCompareBinary { * If the inputsteam supports mark/reset (such as those backed by Array[Byte]), * and the lengths are equal and longer than minSizeForFulBinaryCompare we first * check if they are byte-for-byte identical, which is a cheap way to avoid doing - * potentially complex logic in innerCmp - * - * If the above fails to show them equal, we apply innerCmp. Note innerCmp does - * not need to seek each stream to the end of the records, that is handled by - * this method after innerCmp returns + * potentially complex logic in binary comparators */ - final def compareBinaryPrelude(inputStreamA: InputStream, + final def earlyEqual(inputStreamA: InputStream, lenA: Int, inputStreamB: InputStream, - lenB: Int)(innerCmp: (InputStream, InputStream) => Int) = { - try { - // First up validate the lengths passed make sense - require(lenA >= 0, "Length was " + lenA + "which is < 0, invalid") - require(lenB >= 0, "Length was " + lenB + "which is < 0, invalid") - - val earlyEqual: Boolean = if (lenA > minSizeForFulBinaryCompare && - (lenA == lenB) && - inputStreamA.markSupported && - inputStreamB.markSupported) { + lenB: Int): Boolean = + (lenA > minSizeForFulBinaryCompare && + (lenA == lenB) && + inputStreamA.markSupported && + inputStreamB.markSupported) && { inputStreamA.mark(lenA) inputStreamB.mark(lenB) - @annotation.tailrec - def arrayBytesSame(pos: Int): Boolean = - (pos >= lenA) || - ((inputStreamA.readByte == inputStreamB.readByte) && - arrayBytesSame(pos + 1)) - - arrayBytesSame(0) || { - // rewind if they don't match for doing the full compare - inputStreamA.reset() - inputStreamB.reset() - false + var pos: Int = 0 + while (pos < lenA) { + val a = inputStreamA.read + val b = inputStreamB.read + pos += 1 + if (a != b) { + inputStreamA.reset() + inputStreamB.reset() + // yeah, return sucks, but trying to optimize here + return false + } + // a == b, but may be eof + if (a < 0) return JavaStreamEnrichments.eof } - } else false - - val r = if (earlyEqual) { - 0 - } else { - val bufferedStreamA = PositionInputStream(inputStreamA) - val initialPositionA = bufferedStreamA.position - val bufferedStreamB = PositionInputStream(inputStreamB) - val initialPositionB = bufferedStreamB.position - - val innerR = innerCmp(bufferedStreamA, bufferedStreamB) - - bufferedStreamA.seekToPosition(initialPositionA + lenA) - bufferedStreamB.seekToPosition(initialPositionB + lenB) - innerR + // we consumed all the bytes, and they were all equal + true } - - OrderedSerialization.resultFrom(r) - } catch { - case NonFatal(e) => OrderedSerialization.CompareFailure(e) - } - } - } object TreeOrderedBuf { import CompileTimeLengthTypes._ @@ -232,6 +205,8 @@ object TreeOrderedBuf { val inputStreamA = freshT("inputStreamA") val inputStreamB = freshT("inputStreamB") + val posStreamA = freshT("posStreamA") + val posStreamB = freshT("posStreamB") val lenA = freshT("lenA") val lenB = freshT("lenB") @@ -241,19 +216,23 @@ object TreeOrderedBuf { import _root_.com.twitter.scalding.serialization.JavaStreamEnrichments._ ..$lazyVariables - private[this] val innerBinaryCompare = { ($inputStreamA: _root_.java.io.InputStream, $inputStreamB: _root_.java.io.InputStream) => - ${t.compareBinary(inputStreamA, inputStreamB)} - } - - override def compareBinary($inputStreamA: _root_.java.io.InputStream, $inputStreamB: _root_.java.io.InputStream): _root_.com.twitter.scalding.serialization.OrderedSerialization.Result = { - - val $lenA = ${readLength(inputStreamA)} - val $lenB = ${readLength(inputStreamB)} - - com.twitter.scalding.macros.impl.ordered_serialization.CommonCompareBinary.compareBinaryPrelude($inputStreamA, - $lenA, - $inputStreamB, - $lenB)(innerBinaryCompare) + override def compareBinary($inputStreamA: _root_.java.io.InputStream, $inputStreamB: _root_.java.io.InputStream): _root_.com.twitter.scalding.serialization.OrderedSerialization.Result = + try _root_.com.twitter.scalding.serialization.OrderedSerialization.resultFrom { + val $lenA = ${readLength(inputStreamA)} + val $lenB = ${readLength(inputStreamB)} + val $posStreamA = _root_.com.twitter.scalding.serialization.PositionInputStream($inputStreamA) + val initialPositionA = $posStreamA.position + val $posStreamB = _root_.com.twitter.scalding.serialization.PositionInputStream($inputStreamB) + val initialPositionB = $posStreamB.position + + val innerR = ${t.compareBinary(posStreamA, posStreamB)} + + $posStreamA.seekToPosition(initialPositionA + $lenA) + $posStreamB.seekToPosition(initialPositionB + $lenB) + innerR + } catch { + case _root_.scala.util.control.NonFatal(e) => + _root_.com.twitter.scalding.serialization.OrderedSerialization.CompareFailure(e) } override def hash(passedInObjectToHash: $T): Int = { From 13144600c6095efcbd63ae51acdb357a0eb5f9d7 Mon Sep 17 00:00:00 2001 From: Avi Bryant Date: Thu, 5 Mar 2015 14:57:39 -1000 Subject: [PATCH 056/177] add validateTaps() for Local mode --- .../src/main/scala/com/twitter/scalding/FileSource.scala | 7 +++++++ .../main/scala/com/twitter/scalding/typed/TypedPipe.scala | 4 ++-- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala b/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala index e783525be8..40e95b14b0 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala @@ -195,6 +195,13 @@ abstract class FileSource extends SchemedSource with LocalSourceOverride { "[" + this.toString + "] No good paths in: " + hdfsPaths.toString) } } + + case Local(_) => { + val file = new java.io.File(localPath) + if (!file.exists) + throw new InvalidSourceException( + "[" + this.toString + "] Nothing at path: " + localPath) + } case _ => () } } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipe.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipe.scala index 75d4e32dac..8f5d8ae820 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipe.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipe.scala @@ -540,10 +540,10 @@ trait TypedPipe[+T] extends Serializable { .map(_ => TypedPipe.from(dest)) /** - * If you want to writeThrough to a specific location if it doesn't already exist, + * If you want to writeThrough to a specific file if it doesn't already exist, * and otherwise just read from it going forward, use this. */ - def make[U >: T](dest: Source with TypedSink[T] with TypedSource[U]): Execution[TypedPipe[U]] = + def make[U >: T](dest: FileSource with TypedSink[T] with TypedSource[U]): Execution[TypedPipe[U]] = Execution.getMode.flatMap { mode => try { dest.validateTaps(mode) From 4da5f90f61823d01583137889e38e68b55af43aa Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Thu, 5 Mar 2015 22:13:51 -0800 Subject: [PATCH 057/177] Remove abs --- .../main/scala/com/twitter/scalding/mathematics/Matrix2.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix2.scala b/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix2.scala index 4797d51a5e..66dc078272 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix2.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix2.scala @@ -134,7 +134,7 @@ sealed trait Matrix2[R, C, V] extends Serializable { * After this operation, the sum(|x|^2) along each row will be 1. */ def rowL2Normalize(implicit num: Numeric[V], mj: MatrixJoiner2): Matrix2[R, C, Double] = { - val matD = MatrixLiteral(this.toTypedPipe.map{ case (r, c, x) => (r, c, num.toDouble(x).abs) }, this.sizeHint) + val matD = MatrixLiteral(this.toTypedPipe.map{ case (r, c, x) => (r, c, num.toDouble(x)) }, this.sizeHint) lazy val result = MatrixLiteral(this.toTypedPipe.map { case (r, c, x) => (r, c, num.toDouble(x) * num.toDouble(x)) }, this.sizeHint) .sumColVectors .toTypedPipe From cdf91997a74676353c2ca2f2a0d7ee7cd175e6fb Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Fri, 6 Mar 2015 11:54:35 -1000 Subject: [PATCH 058/177] Add support for Char --- .../scalding/serialization/Hasher.scala | 12 +++- .../serialization/JavaStreamEnrichments.scala | 52 ++++++++++------- .../providers/PrimitiveOrderedBuf.scala | 43 ++++---------- .../macros/MacroOrderingProperties.scala | 56 ++++++++----------- 4 files changed, 75 insertions(+), 88 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Hasher.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Hasher.scala index 06e352da4f..52600598d5 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Hasher.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Hasher.scala @@ -22,7 +22,7 @@ import scala.util.hashing.MurmurHash3 * The specialization *should* mean that there is no boxing and if the JIT * does its work, Hasher should compose well (via collections, Tuple2, Option, Either) */ -trait Hasher[@specialized(Boolean, Byte, Short, Int, Long, Float, Double) -T] { +trait Hasher[@specialized(Boolean, Byte, Char, Short, Int, Long, Float, Double) -T] { @inline def hash(i: T): Int } @@ -58,6 +58,12 @@ object Hasher { @inline def hash(i: Byte) = hashInt(i.toInt) } + implicit val char: Hasher[Char] = new Hasher[Char] { + @inline + def hash(i: Char) = hashInt(i.toInt) + } + val character = char + implicit val short: Hasher[Short] = new Hasher[Short] { @inline def hash(i: Short) = hashInt(i.toInt) @@ -78,11 +84,11 @@ object Hasher { implicit val float: Hasher[Float] = new Hasher[Float] { @inline - def hash(i: Float) = hashInt(java.lang.Float.valueOf(i).intValue) + def hash(i: Float) = hashInt(java.lang.Float.floatToIntBits(i)) } implicit val double: Hasher[Double] = new Hasher[Double] { @inline - def hash(i: Double) = hashLong(i.longValue) + def hash(i: Double) = hashLong(java.lang.Double.doubleToLongBits(i)) } implicit val string: Hasher[String] = new Hasher[String] { @inline diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/JavaStreamEnrichments.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/JavaStreamEnrichments.scala index 29d1097f46..b12b297989 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/JavaStreamEnrichments.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/JavaStreamEnrichments.scala @@ -165,6 +165,13 @@ object JavaStreamEnrichments { (buf(7) & 255) } + def readChar: Char = { + val c1 = s.read + val c2 = s.read + // This is the algorithm from DataInputStream + if ((c1 | c2) < 0) eof else ((c1 << 8) | c2).toChar + } + def readShort: Short = { val c1 = s.read val c2 = s.read @@ -221,17 +228,15 @@ object JavaStreamEnrichments { */ def writePosVarInt(i: Int): Unit = { require(i >= 0, s"must be non-negative: ${i}") - if (i < ((1 << 8) - 1)) s.write(i.toByte) + if (i < ((1 << 8) - 1)) s.write(i) else { s.write(-1: Byte) if (i < ((1 << 16) - 1)) { - val b1 = (i >> 8).toByte - val b2 = (i & 0xFF).toByte - s.write(b1) - s.write(b2) + s.write(i >> 8) + s.write(i) } else { - s.write(-1: Byte) - s.write(-1: Byte) + s.write(-1) + s.write(-1) writeInt(i) } } @@ -242,26 +247,31 @@ object JavaStreamEnrichments { def writeFloat(f: Float): Unit = writeInt(java.lang.Float.floatToIntBits(f)) def writeLong(l: Long): Unit = { - s.write((l >>> 56).toByte) - s.write(((l >>> 48) & 0xFF).toByte) - s.write(((l >>> 40) & 0xFF).toByte) - s.write((l >>> 32).toByte) - s.write((l >>> 24).toByte) - s.write(((l >>> 16) & 0xFF).toByte) - s.write(((l >>> 8) & 0xFF).toByte) - s.write((l & 0xFF).toByte) + s.write((l >>> 56).toInt) + s.write((l >>> 48).toInt) + s.write((l >>> 40).toInt) + s.write((l >>> 32).toInt) + s.write((l >>> 24).toInt) + s.write((l >>> 16).toInt) + s.write((l >>> 8).toInt) + s.write(l.toInt) } def writeInt(i: Int): Unit = { - s.write((i >>> 24).toByte) - s.write(((i >>> 16) & 0xFF).toByte) - s.write(((i >>> 8) & 0xFF).toByte) - s.write((i & 0xFF).toByte) + s.write(i >>> 24) + s.write(i >>> 16) + s.write(i >>> 8) + s.write(i) + } + + def writeChar(sh: Char): Unit = { + s.write(sh >>> 8) + s.write(sh.toInt) } def writeShort(sh: Short): Unit = { - s.write(((sh >>> 8) & 0xFF).toByte) - s.write((sh & 0xFF).toByte) + s.write(sh >>> 8) + s.write(sh.toInt) } } } diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/PrimitiveOrderedBuf.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/PrimitiveOrderedBuf.scala index 6a04ab6149..c9dd1e0414 100644 --- a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/PrimitiveOrderedBuf.scala +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/PrimitiveOrderedBuf.scala @@ -28,6 +28,7 @@ object PrimitiveOrderedBuf { def dispatch(c: Context): PartialFunction[c.Type, TreeOrderedBuf[c.type]] = { case tpe if tpe =:= c.universe.typeOf[Byte] => PrimitiveOrderedBuf(c)(tpe, "Byte", "readByte", "writeByte", 1) case tpe if tpe =:= c.universe.typeOf[Short] => PrimitiveOrderedBuf(c)(tpe, "Short", "readShort", "writeShort", 2) + case tpe if tpe =:= c.universe.typeOf[Char] => PrimitiveOrderedBuf(c)(tpe, "Character", "readChar", "writeChar", 2) case tpe if tpe =:= c.universe.typeOf[Int] => PrimitiveOrderedBuf(c)(tpe, "Integer", "readInt", "writeInt", 4) case tpe if tpe =:= c.universe.typeOf[Long] => PrimitiveOrderedBuf(c)(tpe, "Long", "readLong", "writeLong", 8) case tpe if tpe =:= c.universe.typeOf[Float] => PrimitiveOrderedBuf(c)(tpe, "Float", "readFloat", "writeFloat", 4) @@ -40,10 +41,10 @@ object PrimitiveOrderedBuf { val bbPutter = newTermName(bbPutterStr) val javaType = newTermName(javaTypeStr) - def freshT(id: String = "Product") = newTermName(c.fresh(s"fresh_$id")) + def freshT(id: String) = newTermName(c.fresh(s"fresh_$id")) def genBinaryCompare(inputStreamA: TermName, inputStreamB: TermName): Tree = - if (Set("Float", "Double").contains(javaTypeStr)) { + if (Set("Float", "Double", "Character").contains(javaTypeStr)) { // These cannot be compared using byte-wise approach q"""_root_.java.lang.$javaType.compare($inputStreamA.$bbGetter, $inputStreamB.$bbGetter)""" } else { @@ -72,40 +73,20 @@ object PrimitiveOrderedBuf { }.get // there must be at least one item because no primitive is zero bytes } - def genCompareFn(compareInputA: TermName, compareInputB: TermName): Tree = { - val clamp = Set("Byte", "Short").contains(javaTypeStr) - val compareFn = if (clamp) { - val cmpTmpVal = freshT("cmpTmpVal") - - q""" - val $cmpTmpVal = _root_.java.lang.$javaType.compare($compareInputA, $compareInputB) - if($cmpTmpVal < 0) { - -1 - } else if($cmpTmpVal > 0) { - 1 - } else { - 0 - } - """ - } else { - q""" - _root_.java.lang.$javaType.compare($compareInputA, $compareInputB) - """ - } - compareFn - } - - // used in the hasher - val typeLowerCase = newTermName(javaTypeStr.toLowerCase) - new TreeOrderedBuf[c.type] { override val ctx: c.type = c override val tpe = outerType - override def compareBinary(inputStreamA: ctx.TermName, inputStreamB: ctx.TermName) = genBinaryCompare(inputStreamA, inputStreamB) - override def hash(element: ctx.TermName): ctx.Tree = q"_root_.com.twitter.scalding.serialization.Hasher.$typeLowerCase.hash($element)" + override def compareBinary(inputStreamA: ctx.TermName, inputStreamB: ctx.TermName) = + genBinaryCompare(inputStreamA, inputStreamB) + override def hash(element: ctx.TermName): ctx.Tree = { + // This calls out the correctly named item in Hasher + val typeLowerCase = newTermName(javaTypeStr.toLowerCase) + q"_root_.com.twitter.scalding.serialization.Hasher.$typeLowerCase.hash($element)" + } override def put(inputStream: ctx.TermName, element: ctx.TermName) = q"$inputStream.$bbPutter($element)" override def get(inputStream: ctx.TermName): ctx.Tree = q"$inputStream.$bbGetter" - override def compare(elementA: ctx.TermName, elementB: ctx.TermName): ctx.Tree = genCompareFn(elementA, elementB) + override def compare(elementA: ctx.TermName, elementB: ctx.TermName): ctx.Tree = + q"""_root_.java.lang.$javaType.compare($elementA, $elementB)""" override def length(element: Tree): CompileTimeLengthTypes[c.type] = ConstantLengthCalculation(c)(lenInBytes) override val lazyOuterVariables: Map[String, ctx.Tree] = Map.empty } diff --git a/scalding-macros/src/test/scala/com/twitter/scalding/macros/MacroOrderingProperties.scala b/scalding-macros/src/test/scala/com/twitter/scalding/macros/MacroOrderingProperties.scala index ec37d7eecc..8c7e79404a 100644 --- a/scalding-macros/src/test/scala/com/twitter/scalding/macros/MacroOrderingProperties.scala +++ b/scalding-macros/src/test/scala/com/twitter/scalding/macros/MacroOrderingProperties.scala @@ -228,35 +228,42 @@ class MacroOrderingProperties extends FunSuite with PropertyChecks with ShouldMa test("Test out Unit") { primitiveOrderedBufferSupplier[Unit] - check[Unit] checkMany[Unit] } + test("Test out Boolean") { + primitiveOrderedBufferSupplier[Boolean] + check[Boolean] + } + test("Test out Byte") { + check[Byte] + } + test("Test out Short") { + check[Short] + } + test("Test out Char") { + check[Char] + } test("Test out Int") { primitiveOrderedBufferSupplier[Int] - check[Int] checkMany[Int] } - + test("Test out Float") { + check[Float] + } test("Test out Long") { check[Long] } - - test("Test out Short") { - - check[Short] - } - - test("Test out Float") { - - check[Float] + test("Test out Double") { + check[Double] } + test("Test out String") { + primitiveOrderedBufferSupplier[String] - test("Test out Boolean") { - primitiveOrderedBufferSupplier[Boolean] - check[Boolean] + check[String] + checkMany[String] } test("Test out ByteBuffer") { @@ -390,23 +397,6 @@ class MacroOrderingProperties extends FunSuite with PropertyChecks with ShouldMa checkAreSame(a, b) } - test("Test out Double") { - - check[Double] - } - - test("Test out Byte") { - - check[Byte] - } - - test("Test out String") { - primitiveOrderedBufferSupplier[String] - - check[String] - checkMany[String] - } - test("Test known hard String Case") { val a = "6" val b = "곆" @@ -472,7 +462,7 @@ class MacroOrderingProperties extends FunSuite with PropertyChecks with ShouldMa } test("test product like TestCC") { - checkMany[(Int, Long, Option[Int], Double, Option[String])] + checkMany[(Int, Char, Long, Option[Int], Double, Option[String])] } test("test specific tuple aa1") { From 1f4fbae35407c5b34a94fc1876c6044bb784c58b Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Fri, 6 Mar 2015 14:51:35 -1000 Subject: [PATCH 059/177] Add boxed primitive support --- .../impl/OrderedBufferableProviderImpl.scala | 2 - .../providers/BooleanOrderedBuf.scala | 64 --------------- .../providers/PrimitiveOrderedBuf.scala | 78 +++++++++++++++---- .../macros/MacroOrderingProperties.scala | 48 ++++++++---- 4 files changed, 98 insertions(+), 94 deletions(-) delete mode 100644 scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/BooleanOrderedBuf.scala diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/OrderedBufferableProviderImpl.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/OrderedBufferableProviderImpl.scala index e4c9d60353..4085b5c6db 100644 --- a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/OrderedBufferableProviderImpl.scala +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/OrderedBufferableProviderImpl.scala @@ -38,13 +38,11 @@ object OrderedSerializationProviderImpl { val productDispatcher = ProductOrderedBuf.dispatch(c)(buildDispatcher) val stringDispatcher = StringOrderedBuf.dispatch(c) val traversablesDispatcher = TraversablesOrderedBuf.dispatch(c)(buildDispatcher) - val booleanDispatcher = BooleanOrderedBuf.dispatch(c) val unitDispatcher = UnitOrderedBuf.dispatch(c) val byteBufferDispatcher = ByteBufferOrderedBuf.dispatch(c) OrderedSerializationProviderImpl.normalizedDispatcher(c)(buildDispatcher) .orElse(primitiveDispatcher) - .orElse(booleanDispatcher) .orElse(unitDispatcher) .orElse(optionDispatcher) .orElse(eitherDispatcher) diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/BooleanOrderedBuf.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/BooleanOrderedBuf.scala deleted file mode 100644 index b237f27c4f..0000000000 --- a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/BooleanOrderedBuf.scala +++ /dev/null @@ -1,64 +0,0 @@ -/* - Copyright 2014 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.macros.impl.ordered_serialization.providers - -import scala.language.experimental.macros -import scala.reflect.macros.Context - -import com.twitter.scalding._ - -import com.twitter.scalding.macros.impl.ordered_serialization.{ CompileTimeLengthTypes, ProductLike, TreeOrderedBuf } -import CompileTimeLengthTypes._ - -import java.nio.ByteBuffer -import com.twitter.scalding.serialization.OrderedSerialization - -object BooleanOrderedBuf { - def dispatch(c: Context): PartialFunction[c.Type, TreeOrderedBuf[c.type]] = { - case tpe if tpe =:= c.universe.typeOf[Boolean] => BooleanOrderedBuf(c)(tpe) - } - - def apply(c: Context)(outerType: c.Type): TreeOrderedBuf[c.type] = { - import c.universe._ - - new TreeOrderedBuf[c.type] { - override val ctx: c.type = c - override val tpe = outerType - - override def compareBinary(inputStreamA: ctx.TermName, inputStreamB: ctx.TermName) = - q"_root_.java.lang.Byte.compare($inputStreamA.readByte, $inputStreamB.readByte)" - - override def hash(element: ctx.TermName): ctx.Tree = - q"_root_.com.twitter.scalding.serialization.Hasher.boolean.hash($element)" - - override def put(inputStream: ctx.TermName, element: ctx.TermName) = - q"$inputStream.writeByte(if($element) (1: Byte) else (0: Byte))" - - override def get(inputStreamA: ctx.TermName): ctx.Tree = - q"($inputStreamA.readByte == (1: Byte))" - - def compare(elementA: ctx.TermName, elementB: ctx.TermName): ctx.Tree = - q"_root_.java.lang.Boolean.compare($elementA, $elementB)" - - override def length(element: Tree): CompileTimeLengthTypes[c.type] = - ConstantLengthCalculation(c)(1) - - override val lazyOuterVariables: Map[String, ctx.Tree] = - Map.empty - } - } -} - diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/PrimitiveOrderedBuf.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/PrimitiveOrderedBuf.scala index c9dd1e0414..d224a6202d 100644 --- a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/PrimitiveOrderedBuf.scala +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/PrimitiveOrderedBuf.scala @@ -26,23 +26,55 @@ import com.twitter.scalding.serialization.OrderedSerialization object PrimitiveOrderedBuf { def dispatch(c: Context): PartialFunction[c.Type, TreeOrderedBuf[c.type]] = { - case tpe if tpe =:= c.universe.typeOf[Byte] => PrimitiveOrderedBuf(c)(tpe, "Byte", "readByte", "writeByte", 1) - case tpe if tpe =:= c.universe.typeOf[Short] => PrimitiveOrderedBuf(c)(tpe, "Short", "readShort", "writeShort", 2) - case tpe if tpe =:= c.universe.typeOf[Char] => PrimitiveOrderedBuf(c)(tpe, "Character", "readChar", "writeChar", 2) - case tpe if tpe =:= c.universe.typeOf[Int] => PrimitiveOrderedBuf(c)(tpe, "Integer", "readInt", "writeInt", 4) - case tpe if tpe =:= c.universe.typeOf[Long] => PrimitiveOrderedBuf(c)(tpe, "Long", "readLong", "writeLong", 8) - case tpe if tpe =:= c.universe.typeOf[Float] => PrimitiveOrderedBuf(c)(tpe, "Float", "readFloat", "writeFloat", 4) - case tpe if tpe =:= c.universe.typeOf[Double] => PrimitiveOrderedBuf(c)(tpe, "Double", "readDouble", "writeDouble", 8) + case tpe if tpe =:= c.universe.typeOf[Boolean] => + PrimitiveOrderedBuf(c)(tpe, "Boolean", 1, false) + case tpe if tpe =:= c.universe.typeOf[java.lang.Boolean] => + PrimitiveOrderedBuf(c)(tpe, "Boolean", 1, true) + case tpe if tpe =:= c.universe.typeOf[Byte] => + PrimitiveOrderedBuf(c)(tpe, "Byte", 1, false) + case tpe if tpe =:= c.universe.typeOf[java.lang.Byte] => + PrimitiveOrderedBuf(c)(tpe, "Byte", 1, true) + case tpe if tpe =:= c.universe.typeOf[Short] => + PrimitiveOrderedBuf(c)(tpe, "Short", 2, false) + case tpe if tpe =:= c.universe.typeOf[java.lang.Short] => + PrimitiveOrderedBuf(c)(tpe, "Short", 2, true) + case tpe if tpe =:= c.universe.typeOf[Char] => + PrimitiveOrderedBuf(c)(tpe, "Character", 2, false) + case tpe if tpe =:= c.universe.typeOf[java.lang.Character] => + PrimitiveOrderedBuf(c)(tpe, "Character", 2, true) + case tpe if tpe =:= c.universe.typeOf[Int] => + PrimitiveOrderedBuf(c)(tpe, "Integer", 4, false) + case tpe if tpe =:= c.universe.typeOf[java.lang.Integer] => + PrimitiveOrderedBuf(c)(tpe, "Integer", 4, true) + case tpe if tpe =:= c.universe.typeOf[Long] => + PrimitiveOrderedBuf(c)(tpe, "Long", 8, false) + case tpe if tpe =:= c.universe.typeOf[java.lang.Long] => + PrimitiveOrderedBuf(c)(tpe, "Long", 8, true) + case tpe if tpe =:= c.universe.typeOf[Float] => + PrimitiveOrderedBuf(c)(tpe, "Float", 4, false) + case tpe if tpe =:= c.universe.typeOf[java.lang.Float] => + PrimitiveOrderedBuf(c)(tpe, "Float", 4, true) + case tpe if tpe =:= c.universe.typeOf[Double] => + PrimitiveOrderedBuf(c)(tpe, "Double", 8, false) + case tpe if tpe =:= c.universe.typeOf[java.lang.Double] => + PrimitiveOrderedBuf(c)(tpe, "Double", 8, true) } - def apply(c: Context)(outerType: c.Type, javaTypeStr: String, bbGetterStr: String, bbPutterStr: String, lenInBytes: Int): TreeOrderedBuf[c.type] = { + def apply(c: Context)(outerType: c.Type, + javaTypeStr: String, + lenInBytes: Int, + boxed: Boolean): TreeOrderedBuf[c.type] = { import c.universe._ - val bbGetter = newTermName(bbGetterStr) - val bbPutter = newTermName(bbPutterStr) val javaType = newTermName(javaTypeStr) def freshT(id: String) = newTermName(c.fresh(s"fresh_$id")) + val shortName: String = Map("Integer" -> "Int", "Character" -> "Char") + .getOrElse(javaTypeStr, javaTypeStr) + + val bbGetter = newTermName("read" + shortName) + val bbPutter = newTermName("write" + shortName) + def genBinaryCompare(inputStreamA: TermName, inputStreamB: TermName): Tree = if (Set("Float", "Double", "Character").contains(javaTypeStr)) { // These cannot be compared using byte-wise approach @@ -73,6 +105,12 @@ object PrimitiveOrderedBuf { }.get // there must be at least one item because no primitive is zero bytes } + def accessor(e: c.TermName): c.Tree = { + val primitiveAccessor = newTermName(shortName.toLowerCase + "Value") + if (boxed) q"$e.$primitiveAccessor" + else q"$e" + } + new TreeOrderedBuf[c.type] { override val ctx: c.type = c override val tpe = outerType @@ -81,13 +119,23 @@ object PrimitiveOrderedBuf { override def hash(element: ctx.TermName): ctx.Tree = { // This calls out the correctly named item in Hasher val typeLowerCase = newTermName(javaTypeStr.toLowerCase) - q"_root_.com.twitter.scalding.serialization.Hasher.$typeLowerCase.hash($element)" + q"_root_.com.twitter.scalding.serialization.Hasher.$typeLowerCase.hash(${accessor(element)})" } - override def put(inputStream: ctx.TermName, element: ctx.TermName) = q"$inputStream.$bbPutter($element)" - override def get(inputStream: ctx.TermName): ctx.Tree = q"$inputStream.$bbGetter" + override def put(inputStream: ctx.TermName, element: ctx.TermName) = + q"$inputStream.$bbPutter(${accessor(element)})" + + override def get(inputStream: ctx.TermName): ctx.Tree = { + val unboxed = q"$inputStream.$bbGetter" + if (boxed) q"_root_.java.lang.$javaType.valueOf($unboxed)" else unboxed + } + override def compare(elementA: ctx.TermName, elementB: ctx.TermName): ctx.Tree = - q"""_root_.java.lang.$javaType.compare($elementA, $elementB)""" - override def length(element: Tree): CompileTimeLengthTypes[c.type] = ConstantLengthCalculation(c)(lenInBytes) + if (boxed) q"""$elementA.compareTo($elementB)""" + else q"""_root_.java.lang.$javaType.compare($elementA, $elementB)""" + + override def length(element: Tree): CompileTimeLengthTypes[c.type] = + ConstantLengthCalculation(c)(lenInBytes) + override val lazyOuterVariables: Map[String, ctx.Tree] = Map.empty } } diff --git a/scalding-macros/src/test/scala/com/twitter/scalding/macros/MacroOrderingProperties.scala b/scalding-macros/src/test/scala/com/twitter/scalding/macros/MacroOrderingProperties.scala index 8c7e79404a..0d439518db 100644 --- a/scalding-macros/src/test/scala/com/twitter/scalding/macros/MacroOrderingProperties.scala +++ b/scalding-macros/src/test/scala/com/twitter/scalding/macros/MacroOrderingProperties.scala @@ -136,6 +136,8 @@ class MacroOrderingProperties extends FunSuite with PropertyChecks with ShouldMa def gen[T: Arbitrary]: Gen[T] = implicitly[Arbitrary[T]].arbitrary + def arbMap[T: Arbitrary, U](fn: T => U): Arbitrary[U] = Arbitrary(gen[T].map(fn)) + def collectionArb[C[_], T: Arbitrary](implicit cbf: collection.generic.CanBuildFrom[Nothing, T, C[T]]): Arbitrary[C[T]] = Arbitrary { gen[List[T]].map { l => val builder = cbf() @@ -235,30 +237,50 @@ class MacroOrderingProperties extends FunSuite with PropertyChecks with ShouldMa primitiveOrderedBufferSupplier[Boolean] check[Boolean] } - test("Test out Byte") { - check[Byte] + test("Test out jl.Boolean") { + implicit val a = arbMap { b: Boolean => java.lang.Boolean.valueOf(b) } + check[java.lang.Boolean] } - test("Test out Short") { - check[Short] + test("Test out Byte") { check[Byte] } + test("Test out jl.Byte") { + implicit val a = arbMap { b: Byte => java.lang.Byte.valueOf(b) } + check[java.lang.Byte] } - - test("Test out Char") { - check[Char] + test("Test out Short") { check[Short] } + test("Test out jl.Short") { + implicit val a = arbMap { b: Short => java.lang.Short.valueOf(b) } + check[java.lang.Short] + } + test("Test out Char") { check[Char] } + test("Test out jl.Char") { + implicit val a = arbMap { b: Char => java.lang.Character.valueOf(b) } + check[java.lang.Character] } test("Test out Int") { primitiveOrderedBufferSupplier[Int] check[Int] checkMany[Int] } - test("Test out Float") { - check[Float] + test("Test out jl.Integer") { + implicit val a = arbMap { b: Int => java.lang.Integer.valueOf(b) } + check[java.lang.Integer] } - test("Test out Long") { - check[Long] + test("Test out Float") { check[Float] } + test("Test out jl.Float") { + implicit val a = arbMap { b: Float => java.lang.Float.valueOf(b) } + check[java.lang.Float] } - test("Test out Double") { - check[Double] + test("Test out Long") { check[Long] } + test("Test out jl.Long") { + implicit val a = arbMap { b: Long => java.lang.Long.valueOf(b) } + check[java.lang.Long] } + test("Test out Double") { check[Double] } + test("Test out jl.Double") { + implicit val a = arbMap { b: Double => java.lang.Double.valueOf(b) } + check[java.lang.Double] + } + test("Test out String") { primitiveOrderedBufferSupplier[String] From 6104de58a0c68c7f5d487abd5d40faf8b416f724 Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Mon, 9 Mar 2015 11:13:01 -1000 Subject: [PATCH 060/177] Make the buffer for dynamic sized bigger than 32 bytes --- .../macros/impl/ordered_serialization/TreeOrderedBuf.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/TreeOrderedBuf.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/TreeOrderedBuf.scala index a3bee56827..885e0a6253 100644 --- a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/TreeOrderedBuf.scala +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/TreeOrderedBuf.scala @@ -139,7 +139,8 @@ object TreeOrderedBuf { * directly serializing. */ val noLenCalc = q""" - val $baos = new _root_.java.io.ByteArrayOutputStream + // Start with pretty big buffers because reallocation will be expensive + val $baos = new _root_.java.io.ByteArrayOutputStream(256) ${t.put(baos, element)} val $len = $baos.size $outerbaos.writePosVarInt($len) From 3f8902de0c83e95b000450b3c370b2e979d5165d Mon Sep 17 00:00:00 2001 From: Tom Dyas Date: Thu, 5 Mar 2015 20:42:32 -0500 Subject: [PATCH 061/177] provide Args as an implicit The Checkpoint extension (com.twitter.scalding.commons.extensions.Checkpoint) wants an implicit Args provided (along with FlowDef and Mode). Args is similar to the already implicitly provided FlowDef and Mode in Job and should be made available implicitly as well. --- scalding-core/src/main/scala/com/twitter/scalding/Job.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Job.scala b/scalding-core/src/main/scala/com/twitter/scalding/Job.scala index 076fd3979b..7a2604cd77 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Job.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Job.scala @@ -116,6 +116,9 @@ class Job(val args: Args) extends FieldConversions with java.io.Serializable { implicit def iterableToRichPipe[T](iter: Iterable[T])(implicit set: TupleSetter[T], conv: TupleConverter[T]): RichPipe = RichPipe(toPipe(iter)(set, conv)) + // Provide args as an implicit val for extensions such as the Checkpoint extension. + implicit protected def _implicitJobArgs: Args = args + // Override this if you want to change how the mapred.job.name is written in Hadoop def name: String = Config.defaultFrom(mode).toMap.getOrElse("mapred.job.name", getClass.getName) From 50a6b71fac29eb73f3b1dd28f1a4dba317c76342 Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Fri, 13 Mar 2015 15:38:32 -1000 Subject: [PATCH 062/177] Address Alex's comments --- .../serialization/JavaStreamEnrichments.scala | 14 ++++++++++---- .../serialization/PositionInputStream.scala | 7 +++++-- .../serialization/WrappedSerialization.scala | 4 ++-- .../providers/StringOrderedBuf.scala | 4 ++++ .../scalding/macros/MacroOrderingProperties.scala | 11 ++--------- 5 files changed, 23 insertions(+), 17 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/JavaStreamEnrichments.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/JavaStreamEnrichments.scala index b12b297989..528807f824 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/JavaStreamEnrichments.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/JavaStreamEnrichments.scala @@ -20,6 +20,11 @@ import java.io._ object JavaStreamEnrichments { def eof: Nothing = throw new EOFException() + // We use this to avoid allocating a closure to make + // a lazy parameter to require + private def illegal(s: String): Nothing = + throw new IllegalArgumentException(s) + /** * Note this is only recommended for testing. * You may want to use ByteArrayInputOutputStream for performance critical concerns @@ -42,8 +47,9 @@ object JavaStreamEnrichments { * you will write */ class ArrayWrappingOutputStream(val buffer: Array[Byte], initPos: Int) extends OutputStream { - require(buffer.length >= initPos, - s"Initial position cannot be more than length: $initPos > ${buffer.length}") + if (buffer.length < initPos) { + illegal(s"Initial position cannot be more than length: $initPos > ${buffer.length}") + } private[this] var pos = initPos def position: Int = pos override def write(b: Int) { buffer(pos) = b.toByte; pos += 1 } @@ -54,7 +60,7 @@ object JavaStreamEnrichments { } def posVarIntSize(i: Int): Int = { - require(i >= 0, s"negative numbers not allowed: $i") + if (i < 0) illegal(s"negative numbers not allowed: $i") if (i < ((1 << 8) - 1)) 1 else { if (i < ((1 << 16) - 1)) { @@ -227,7 +233,7 @@ object JavaStreamEnrichments { * 7 bytes for 65536 - Int.MaxValue */ def writePosVarInt(i: Int): Unit = { - require(i >= 0, s"must be non-negative: ${i}") + if (i < 0) illegal(s"must be non-negative: ${i}") if (i < ((1 << 8) - 1)) s.write(i) else { s.write(-1: Byte) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/PositionInputStream.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/PositionInputStream.scala index 59880124e6..434adfc610 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/PositionInputStream.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/PositionInputStream.scala @@ -64,8 +64,11 @@ class PositionInputStream(val wraps: InputStream) extends InputStream { pos = markPos } + private def illegal(s: String): Nothing = + throw new IllegalArgumentException(s) + override def skip(n: Long): Long = { - require(n >= 0, "Must seek fowards") + if (n < 0) illegal("Must seek fowards") val count = skip(n) if (count > 0) pos += count count @@ -75,7 +78,7 @@ class PositionInputStream(val wraps: InputStream) extends InputStream { * This throws an exception if it can't set the position to what you give it. */ def seekToPosition(p: Long) { - require(p >= pos, s"Can't seek backwards, at position $pos, trying to goto $p") + if (p < pos) illegal(s"Can't seek backwards, at position $pos, trying to goto $p") wraps.skipFully(p - pos) pos = p } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/WrappedSerialization.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/WrappedSerialization.scala index b2c1e13aa9..566d701031 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/WrappedSerialization.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/WrappedSerialization.scala @@ -64,7 +64,7 @@ class BinarySerializer[T](buf: Serialization[T]) extends Serializer[T] { } def close(): Unit = { out = null } def serialize(t: T): Unit = { - require(out != null, "OutputStream is null") + if (out == null) throw new NullPointerException("OutputStream is null") buf.write(out, t).get } } @@ -74,7 +74,7 @@ class BinaryDeserializer[T](buf: Serialization[T]) extends Deserializer[T] { def open(i: InputStream): Unit = { is = i } def close(): Unit = { is = null } def deserialize(t: T): T = { - require(is != null, "InputStream is null") + if (is == null) throw new NullPointerException("InputStream is null") buf.read(is).get } } diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/StringOrderedBuf.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/StringOrderedBuf.scala index afb1f72a2c..d29dde6e4a 100644 --- a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/StringOrderedBuf.scala +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/StringOrderedBuf.scala @@ -84,6 +84,10 @@ object StringOrderedBuf { } else { // Just use utf-8 + // TODO: investigate faster ways to encode UTF-8, if + // the bug that makes string Charsets faster than using Charset instances. + // see for instance: + // http://psy-lob-saw.blogspot.com/2012/12/encode-utf-8-string-to-bytebuffer-faster.html val $bytes = $element.getBytes("UTF-8") val $len = $bytes.length $inputStream.writePosVarInt($len) diff --git a/scalding-macros/src/test/scala/com/twitter/scalding/macros/MacroOrderingProperties.scala b/scalding-macros/src/test/scala/com/twitter/scalding/macros/MacroOrderingProperties.scala index 0d439518db..49c7263691 100644 --- a/scalding-macros/src/test/scala/com/twitter/scalding/macros/MacroOrderingProperties.scala +++ b/scalding-macros/src/test/scala/com/twitter/scalding/macros/MacroOrderingProperties.scala @@ -81,7 +81,7 @@ object MyData { class MyData(override val _1: Int, override val _2: Option[Long]) extends Product2[Int, Option[Long]] { override def canEqual(that: Any): Boolean = that match { - case o: MyData => this._1 == o._1 && this._2 == o._2 + case o: MyData => true case _ => false } } @@ -186,13 +186,6 @@ class MacroOrderingProperties extends FunSuite with PropertyChecks with ShouldMa checkManyExplicit(i) } - def clamp(i: Int): Int = - i match { - case x if x < 0 => -1 - case x if x > 0 => 1 - case x => 0 - } - def checkWithInputs[T](a: T, b: T)(implicit obuf: OrderedSerialization[T]) { val rta = rt(a) // before we do anything ensure these don't throw val rtb = rt(b) // before we do anything ensure these don't throw @@ -423,7 +416,7 @@ class MacroOrderingProperties extends FunSuite with PropertyChecks with ShouldMa val a = "6" val b = "곆" val ord = Ordering.String - assert(rawCompare(a, b) === clamp(ord.compare(a, b)), "Raw and in memory compares match.") + assert(rawCompare(a, b) === ord.compare(a, b).signum, "Raw and in memory compares match.") val c = List("榴㉕⊟풠湜ᙬ覹ꜻ裧뚐⠂覝쫨塢䇺楠谭픚ᐌ轮뺷Ⱟ洦擄黏著탅ﮓꆋ숷梸傠ァ蹵窥轲闇涡飽ꌳ䝞慙擃", "堒凳媨쉏떽㶥⾽샣井ㆠᇗ裉깴辫࠷᤭塈䎙寫㸉ᶴ䰄똇䡷䥞㷗䷱赫懓䷏剆祲ᝯ졑쐯헢鷴ӕ秔㽰ퟡ㏉鶖奚㙰银䮌ᕗ膾买씋썴행䣈丶偝쾕鐗쇊ኋ넥︇瞤䋗噯邧⹆♣ἷ铆玼⪷沕辤ᠥ⥰箼䔄◗", From 37e03d1f76ede7d8a10ebaeb539d0689d77a286b Mon Sep 17 00:00:00 2001 From: Tom Dyas Date: Tue, 17 Mar 2015 18:06:35 -0400 Subject: [PATCH 063/177] call sourceConfInit when reading from taps in local mode sourceConfInit should be called even in local mode when accessing a Tap. --- scalding-core/src/main/scala/com/twitter/scalding/Mode.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Mode.scala b/scalding-core/src/main/scala/com/twitter/scalding/Mode.scala index e6c73f6f58..4547fb0abc 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Mode.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Mode.scala @@ -144,6 +144,7 @@ trait CascadingLocal extends Mode { config.toMap.foreach { case (k, v) => props.setProperty(k, v) } val fp = new LocalFlowProcess(props) ltap.retrieveSourceFields(fp) + ltap.sourceConfInit(fp, props) ltap.openForRead(fp) } } From e4bf0bac357269ff411079199843e0e8eb4678a1 Mon Sep 17 00:00:00 2001 From: Kevin Lin Date: Tue, 24 Mar 2015 17:25:42 -0700 Subject: [PATCH 064/177] Add distinctCount and distinctValues helper method to KeyedList. --- .../twitter/scalding/typed/KeyedList.scala | 13 +++++++ .../com/twitter/scalding/TypedPipeTest.scala | 35 +++++++++++++++++++ 2 files changed, 48 insertions(+) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/KeyedList.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/KeyedList.scala index aab6db8429..cbdf73d9b1 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/KeyedList.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/KeyedList.scala @@ -303,6 +303,19 @@ trait KeyedListLike[K, +T, +This[K, +T] <: KeyedListLike[K, T, This]] /** For each key, give the number of values */ def size: This[K, Long] = mapValues { x => 1L }.sum + + /** + * For each key, give the number of unique values. WARNING: May OOM. + * This assumes the values for each key can fit in memory. + */ + def distinctCount[B >: T]: This[K, Long] = toSet[B].mapValues(_.size) + + /** + * For each key, remove duplicate values. WARNING: May OOM. + * This assumes the values for each key can fit in memory. + */ + def distinctValues[B >: T]: This[K, B] = toSet[B].flattenValues + /** * AVOID THIS IF POSSIBLE * For each key, accumulate all the values into a List. WARNING: May OOM diff --git a/scalding-core/src/test/scala/com/twitter/scalding/TypedPipeTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/TypedPipeTest.scala index 3a345c051c..03d15487a7 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/TypedPipeTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/TypedPipeTest.scala @@ -194,6 +194,41 @@ class TypedPipeDistinctByTest extends WordSpec with Matchers { } } +class TypedPipeGroupedDistinctJob(args: Args) extends Job(args) { + val groupedTP = Tsv("inputFile").read.toTypedPipe[(Int, Int)](0, 1) + .group + + groupedTP + .distinctValues + .write(TypedTsv[(Int, Int)]("outputFile1")) + groupedTP + .distinctCount + .write(TypedTsv[(Int, Long)]("outputFile2")) +} + +class TypedPipeGroupedDistinctJobTest extends WordSpec with Matchers { + import Dsl._ + "A TypedPipeGroupedDistinctJob" should { + JobTest(new TypedPipeGroupedDistinctJob(_)) + .source(Tsv("inputFile"), List((0, 0), (0, 1), (0, 1), (1, 0), (1, 1))) + .sink[(Int, Int)](TypedTsv[(Int, Int)]("outputFile1")){ outputBuffer => + val outSet = outputBuffer.toSet + "correctly generate unique items" in { + outSet should have size 4 + } + } + .sink[(Int, Int)](TypedTsv[(Int, Long)]("outputFile2")){ outputBuffer => + val outMap = outputBuffer.toMap + "correctly count unique item sizes" in { + outMap(0) shouldBe 2 + outMap(1) shouldBe 2 + } + } + .run + .finish + } +} + class TypedPipeHashJoinJob(args: Args) extends Job(args) { TypedTsv[(Int, Int)]("inputFile0") .group From d90eead939778c9fe885df1671e6f1c234f7677e Mon Sep 17 00:00:00 2001 From: Kevin Lin Date: Wed, 25 Mar 2015 11:30:14 -0700 Subject: [PATCH 065/177] distinctSize. --- .../src/main/scala/com/twitter/scalding/typed/KeyedList.scala | 2 +- .../src/test/scala/com/twitter/scalding/TypedPipeTest.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/KeyedList.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/KeyedList.scala index cbdf73d9b1..229fbdf5e2 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/KeyedList.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/KeyedList.scala @@ -308,7 +308,7 @@ trait KeyedListLike[K, +T, +This[K, +T] <: KeyedListLike[K, T, This]] * For each key, give the number of unique values. WARNING: May OOM. * This assumes the values for each key can fit in memory. */ - def distinctCount[B >: T]: This[K, Long] = toSet[B].mapValues(_.size) + def distinctSize[B >: T]: This[K, Long] = toSet[B].mapValues(_.size) /** * For each key, remove duplicate values. WARNING: May OOM. diff --git a/scalding-core/src/test/scala/com/twitter/scalding/TypedPipeTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/TypedPipeTest.scala index 03d15487a7..10ff9e67bf 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/TypedPipeTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/TypedPipeTest.scala @@ -202,7 +202,7 @@ class TypedPipeGroupedDistinctJob(args: Args) extends Job(args) { .distinctValues .write(TypedTsv[(Int, Int)]("outputFile1")) groupedTP - .distinctCount + .distinctSize .write(TypedTsv[(Int, Long)]("outputFile2")) } From 5e9200af7178697ed6b4cb24c62760c5ec1b7dc3 Mon Sep 17 00:00:00 2001 From: Kevin Lin Date: Thu, 26 Mar 2015 10:47:12 -0700 Subject: [PATCH 066/177] Directly use type T. --- .../src/main/scala/com/twitter/scalding/typed/KeyedList.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/KeyedList.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/KeyedList.scala index 229fbdf5e2..2620e8e7f7 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/KeyedList.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/KeyedList.scala @@ -308,7 +308,7 @@ trait KeyedListLike[K, +T, +This[K, +T] <: KeyedListLike[K, T, This]] * For each key, give the number of unique values. WARNING: May OOM. * This assumes the values for each key can fit in memory. */ - def distinctSize[B >: T]: This[K, Long] = toSet[B].mapValues(_.size) + def distinctSize: This[K, Long] = toSet[T].mapValues(_.size) /** * For each key, remove duplicate values. WARNING: May OOM. From 3c3f5dab691422ffc623b850107045ba3a10fd1e Mon Sep 17 00:00:00 2001 From: Kevin Lin Date: Thu, 26 Mar 2015 12:27:25 -0700 Subject: [PATCH 067/177] Also update distinctValues. --- .../src/main/scala/com/twitter/scalding/typed/KeyedList.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/KeyedList.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/KeyedList.scala index 2620e8e7f7..8f812ef471 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/KeyedList.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/KeyedList.scala @@ -314,7 +314,7 @@ trait KeyedListLike[K, +T, +This[K, +T] <: KeyedListLike[K, T, This]] * For each key, remove duplicate values. WARNING: May OOM. * This assumes the values for each key can fit in memory. */ - def distinctValues[B >: T]: This[K, B] = toSet[B].flattenValues + def distinctValues: This[K, T] = toSet[T].flattenValues /** * AVOID THIS IF POSSIBLE From 8f1a95e5241993e1c7c28f9a7a8e96733005726e Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Fri, 27 Mar 2015 16:05:10 -0700 Subject: [PATCH 068/177] Convert AnyVal to AnyRef for date in order to work with serialization --- .../src/test/scala/com/twitter/scalding/KryoTest.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/scalding-core/src/test/scala/com/twitter/scalding/KryoTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/KryoTest.scala index aabae9197a..b59071a4d3 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/KryoTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/KryoTest.scala @@ -150,7 +150,11 @@ class KryoTest extends WordSpec with Matchers { val myDate: RichDate = "1999-12-30T14" val simpleDate: java.util.Date = myDate.value val myDateRange = DateRange("2012-01-02", "2012-06-09") - singleRT(myDate) shouldBe myDate + + // to work with serialization + val newMyDate = myDate.asInstanceOf[AnyRef] + + singleRT(newMyDate) shouldBe myDate singleRT(simpleDate) shouldBe simpleDate singleRT(myDateRange) shouldBe myDateRange } From a050c7de328d0dd4da0d865a600d867f6590645e Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Fri, 27 Mar 2015 16:05:41 -0700 Subject: [PATCH 069/177] Change RichDate to value class --- .../main/scala/com/twitter/scalding/RichDate.scala | 12 +----------- 1 file changed, 1 insertion(+), 11 deletions(-) diff --git a/scalding-date/src/main/scala/com/twitter/scalding/RichDate.scala b/scalding-date/src/main/scala/com/twitter/scalding/RichDate.scala index a06a91d815..cbc1284165 100644 --- a/scalding-date/src/main/scala/com/twitter/scalding/RichDate.scala +++ b/scalding-date/src/main/scala/com/twitter/scalding/RichDate.scala @@ -66,7 +66,7 @@ object RichDate { /** * A value class wrapper for milliseconds since the epoch */ -case class RichDate(val timestamp: Long) extends Ordered[RichDate] { +case class RichDate(val timestamp: Long) extends AnyVal with Ordered[RichDate] { // these are mutable, don't keep them around def value: Date = new java.util.Date(timestamp) @@ -79,14 +79,6 @@ case class RichDate(val timestamp: Long) extends Ordered[RichDate] { override def compare(that: RichDate): Int = Ordering[Long].compare(timestamp, that.timestamp) - //True of the other is a RichDate with equal value, or a Date equal to value - override def equals(that: Any) = - that match { - case d: Date => d.getTime == timestamp - case RichDate(ts) => ts == timestamp - case _ => false - } - /** * Use String.format to format the date, as opposed to toString with uses SimpleDateFormat */ @@ -97,8 +89,6 @@ case class RichDate(val timestamp: Long) extends Ordered[RichDate] { * to make them equal. this is the same as what java does (and only sane thing): * http://grepcode.com/file/repository.grepcode.com/java/root/jdk/openjdk/6-b14/java/util/Date.java#989 */ - override def hashCode = - (timestamp.toInt) ^ ((timestamp >> 32).toInt) def toCalendar(implicit tz: TimeZone) = { val cal = Calendar.getInstance(tz) From 76728c306403cbd9141a4461a5d0cdfc803df942 Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Fri, 27 Mar 2015 16:06:35 -0700 Subject: [PATCH 070/177] Delete the test for equals method and hashCode method for RichDate --- .../src/test/scala/com/twitter/scalding/DateTest.scala | 5 ----- 1 file changed, 5 deletions(-) diff --git a/scalding-date/src/test/scala/com/twitter/scalding/DateTest.scala b/scalding-date/src/test/scala/com/twitter/scalding/DateTest.scala index 543300d7df..79723d7952 100644 --- a/scalding-date/src/test/scala/com/twitter/scalding/DateTest.scala +++ b/scalding-date/src/test/scala/com/twitter/scalding/DateTest.scala @@ -66,11 +66,6 @@ class DateTest extends WordSpec { val rd2: RichDate = " 2010-10-02 T 00:00:01 " assert(rd1 === rd2) } - "Have same equals & hashCode as Date (crazy?)" in { - val rd1: RichDate = "2011-10-20" - assert(rd1 === rd1.value) - assert(rd1.hashCode === rd1.value.hashCode) - } "be well ordered" in { val rd1: RichDate = "2011-10-20" val rd2: RichDate = "2011-10-21" From 9265c1473b007371ff1388db8d17a1a28cf62543 Mon Sep 17 00:00:00 2001 From: Tom Dyas Date: Wed, 1 Apr 2015 00:25:50 -0400 Subject: [PATCH 071/177] import hygiene: remove unused imports Remove many unused imports. --- .../com/twitter/scalding/Serialization.scala | 1 - .../scalding/commons/extensions/Checkpoint.scala | 5 +---- .../scalding/commons/source/DailySources.scala | 2 -- .../commons/source/FixedPathSources.scala | 2 -- .../scalding/commons/source/HourlySources.scala | 2 -- .../commons/source/LongThriftTransformer.scala | 1 - .../scalding/commons/source/LzoCodecSource.scala | 1 - .../scalding/commons/source/LzoTraits.scala | 3 --- .../scalding/commons/source/PailSource.scala | 6 +----- .../scalding/commons/source/TsvWithHeader.scala | 4 +--- .../commons/VersionedKeyValSourceTest.scala | 3 --- .../main/scala/com/twitter/scalding/Config.scala | 9 ++------- .../scala/com/twitter/scalding/Execution.scala | 2 +- .../scala/com/twitter/scalding/FlowState.scala | 5 ++--- .../com/twitter/scalding/GroupBuilder.scala | 7 +------ .../twitter/scalding/IntegralComparator.scala | 8 ++++---- .../com/twitter/scalding/IterableSource.scala | 7 ------- .../main/scala/com/twitter/scalding/Job.scala | 14 ++++---------- .../scala/com/twitter/scalding/JobStats.scala | 2 -- .../com/twitter/scalding/JoinAlgorithms.scala | 8 -------- .../scala/com/twitter/scalding/MemoryTap.scala | 2 +- .../main/scala/com/twitter/scalding/Mode.scala | 10 ++++------ .../scala/com/twitter/scalding/Operations.scala | 5 ----- .../com/twitter/scalding/PartitionSource.scala | 7 ------- .../com/twitter/scalding/ReduceOperations.scala | 3 +-- .../scala/com/twitter/scalding/RichPipe.scala | 7 ------- .../main/scala/com/twitter/scalding/Source.scala | 16 +++++----------- .../com/twitter/scalding/StreamOperations.scala | 4 ---- .../com/twitter/scalding/TemplateSource.scala | 7 ------- .../com/twitter/scalding/TimePathedSource.scala | 3 --- .../main/scala/com/twitter/scalding/Tool.scala | 16 +++++++--------- .../com/twitter/scalding/TupleConversions.scala | 9 --------- .../scala/com/twitter/scalding/TuplePacker.scala | 2 -- .../com/twitter/scalding/TupleUnpacker.scala | 3 --- .../com/twitter/scalding/BlockJoinTest.scala | 2 -- .../scala/com/twitter/scalding/CascadeTest.scala | 1 - .../scala/com/twitter/scalding/CoGroupTest.scala | 1 - .../scala/com/twitter/scalding/ConfigTest.scala | 2 -- .../com/twitter/scalding/CumulitiveSumTest.scala | 2 -- .../com/twitter/scalding/DistinctByTest.scala | 3 --- .../scalding/ExecutionAppProperties.scala | 2 -- .../scala/com/twitter/scalding/KryoTest.scala | 3 +-- .../twitter/scalding/PartitionSourceTest.scala | 1 - .../twitter/scalding/ReduceOperationsTest.scala | 1 - .../com/twitter/scalding/ScanLeftTest.scala | 1 - .../com/twitter/scalding/SideEffectTest.scala | 2 -- .../com/twitter/scalding/SkewJoinTest.scala | 4 ---- .../twitter/scalding/TemplateSourceTest.scala | 3 --- .../twitter/scalding/TestTapFactoryTest.scala | 1 - .../twitter/scalding/TypedDelimitedTest.scala | 1 - .../scala/com/twitter/scalding/DateOps.scala | 2 -- .../com/twitter/scalding/DateProperties.scala | 1 - .../com/twitter/scalding/GlobifierOps.scala | 1 - .../twitter/scalding/GlobifierProperties.scala | 2 -- .../platform/HadoopPlatformJobTest.scala | 2 -- .../twitter/scalding/platform/LocalCluster.scala | 16 ++-------------- .../ReducerEstimatorTest.scala | 3 +-- .../scala/com/twitter/scalding/TypedJson.scala | 3 --- .../com/twitter/scalding/ILoopCompat.scala | 2 +- .../com/twitter/scalding/ScaldingShell.scala | 2 +- .../scala/com/twitter/scalding/ShellPipe.scala | 2 -- 61 files changed, 41 insertions(+), 211 deletions(-) diff --git a/scalding-benchmarks/src/test/scala/com/twitter/scalding/Serialization.scala b/scalding-benchmarks/src/test/scala/com/twitter/scalding/Serialization.scala index 4420dfe4eb..de4ef75323 100644 --- a/scalding-benchmarks/src/test/scala/com/twitter/scalding/Serialization.scala +++ b/scalding-benchmarks/src/test/scala/com/twitter/scalding/Serialization.scala @@ -7,7 +7,6 @@ import org.scalacheck.{ Gen => scGen, Arbitrary } // We use scalacheck Gens to g import org.scalameter.api._ import scala.collection.generic.CanBuildFrom import scala.language.experimental.macros -import scala.util.Sorting trait LowerPriorityImplicit { implicit def ordBuf[T]: OrderedSerialization[T] = macro com.twitter.scalding.macros.impl.OrderedSerializationProviderImpl[T] diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/extensions/Checkpoint.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/extensions/Checkpoint.scala index d27c85be57..a6d7666941 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/extensions/Checkpoint.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/extensions/Checkpoint.scala @@ -19,12 +19,9 @@ package com.twitter.scalding.commons.extensions import com.twitter.scalding._ import com.twitter.scalding.Dsl._ -import java.io.File import cascading.flow.FlowDef import cascading.pipe.Pipe -import cascading.tuple.{ Fields, TupleEntry } -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{ FileSystem, Path } +import cascading.tuple.Fields import org.slf4j.{ Logger, LoggerFactory => LogManager } /** diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/DailySources.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/DailySources.scala index c7bee55b4e..a33cb85a55 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/DailySources.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/DailySources.scala @@ -19,8 +19,6 @@ package com.twitter.scalding.commons.source import com.google.protobuf.Message import com.twitter.bijection.Injection import com.twitter.chill.Externalizer -import com.twitter.elephantbird.cascading2.scheme._ -import com.twitter.elephantbird.util.{ ThriftUtils, TypeRef } import com.twitter.scalding._ import com.twitter.scalding.source._ diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/FixedPathSources.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/FixedPathSources.scala index a7aad7ee18..eaca1d9863 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/FixedPathSources.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/FixedPathSources.scala @@ -18,8 +18,6 @@ package com.twitter.scalding.commons.source import com.google.protobuf.Message import com.twitter.scalding._ -import com.twitter.scalding.Dsl._ -import java.io.Serializable import org.apache.thrift.TBase abstract class FixedPathLzoThrift[T <: TBase[_, _]: Manifest](path: String*) diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/HourlySources.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/HourlySources.scala index 64d29df656..568dce0609 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/HourlySources.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/HourlySources.scala @@ -21,9 +21,7 @@ import com.google.protobuf.Message import com.twitter.bijection.Injection import com.twitter.chill.Externalizer import com.twitter.scalding._ -import com.twitter.scalding.Dsl._ import com.twitter.scalding.source._ -import java.io.Serializable import org.apache.thrift.TBase abstract class HourlySuffixLzoCodec[T](prefix: String, dateRange: DateRange)(implicit @transient suppliedInjection: Injection[T, Array[Byte]]) diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LongThriftTransformer.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LongThriftTransformer.scala index 80d25fe45f..dda71a26a4 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LongThriftTransformer.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LongThriftTransformer.scala @@ -21,7 +21,6 @@ import cascading.tuple.Fields import com.twitter.elephantbird.mapreduce.io.ThriftWritable import com.twitter.elephantbird.util.{ ThriftUtils, TypeRef } import com.twitter.scalding._ -import com.twitter.scalding.Dsl._ import org.apache.hadoop.io.{ LongWritable, Writable } import org.apache.thrift.TBase diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoCodecSource.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoCodecSource.scala index c97a612644..74d2ff6924 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoCodecSource.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoCodecSource.scala @@ -17,7 +17,6 @@ limitations under the License. package com.twitter.scalding.commons.source import com.twitter.chill.Externalizer -import com.twitter.scalding._ import com.twitter.bijection.Injection /** diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoTraits.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoTraits.scala index d69cfcb1ac..eeb28fc929 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoTraits.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoTraits.scala @@ -16,10 +16,7 @@ limitations under the License. package com.twitter.scalding.commons.source -import collection.mutable.ListBuffer - import cascading.pipe.Pipe -import cascading.scheme.local.{ TextDelimited => CLTextDelimited, TextLine => CLTextLine } import cascading.scheme.Scheme import org.apache.thrift.TBase diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/PailSource.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/PailSource.scala index 3f700b3573..7b60eedc77 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/PailSource.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/PailSource.scala @@ -19,15 +19,11 @@ package com.twitter.scalding.commons.source import scala.reflect.ClassTag import com.backtype.cascading.tap.PailTap -import com.backtype.hadoop.pail.{ Pail, PailStructure } -import cascading.pipe.Pipe -import cascading.scheme.Scheme +import com.backtype.hadoop.pail.PailStructure import cascading.tap.Tap import com.twitter.bijection.Injection -import com.twitter.chill.Externalizer import com.twitter.scalding._ import java.util.{ List => JList } -import org.apache.hadoop.mapred.{ JobConf, OutputCollector, RecordReader } import scala.collection.JavaConverters._ /** diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/TsvWithHeader.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/TsvWithHeader.scala index 46dd53744a..7d369fd4aa 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/TsvWithHeader.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/TsvWithHeader.scala @@ -22,9 +22,7 @@ import cascading.tuple.Fields import com.google.common.base.Charsets import com.google.common.io.Files import com.twitter.scalding._ -import com.twitter.scalding.Dsl._ -import java.io.{ BufferedWriter, File, FileOutputStream, IOException, OutputStreamWriter, Serializable } -import org.apache.hadoop.conf.Configuration +import java.io.{ BufferedWriter, File, FileOutputStream, IOException, OutputStreamWriter } import org.apache.hadoop.fs.{ FileSystem, Path } /** diff --git a/scalding-commons/src/test/scala/com/twitter/scalding/commons/VersionedKeyValSourceTest.scala b/scalding-commons/src/test/scala/com/twitter/scalding/commons/VersionedKeyValSourceTest.scala index 5b51fb4f72..5a41d19bdf 100644 --- a/scalding-commons/src/test/scala/com/twitter/scalding/commons/VersionedKeyValSourceTest.scala +++ b/scalding-commons/src/test/scala/com/twitter/scalding/commons/VersionedKeyValSourceTest.scala @@ -23,11 +23,8 @@ import com.backtype.hadoop.datastores.VersionedStore import org.apache.hadoop.mapred.JobConf // Use the scalacheck generators -import org.scalacheck.Gen import scala.collection.mutable.Buffer -import TDsl._ - class TypedWriteIncrementalJob(args: Args) extends Job(args) { import RichPipeEx._ val pipe = TypedPipe.from(TypedTsv[Int]("input")) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala index 4142b0cb95..29137a2747 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala @@ -18,19 +18,14 @@ package com.twitter.scalding import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.serializer.{ Serialization => HSerialization } import com.twitter.chill.KryoInstantiator -import com.twitter.chill.config.{ ScalaMapConfig, ScalaAnyRefMapConfig, ConfiguredInstantiator } -import com.twitter.scalding.reducer_estimation.ReducerEstimator +import com.twitter.chill.config.{ ScalaMapConfig, ConfiguredInstantiator } import cascading.pipe.assembly.AggregateBy -import cascading.flow.{ FlowStepStrategy, FlowProps } +import cascading.flow.FlowProps import cascading.property.AppProps import cascading.tuple.collect.SpillableProps import java.security.MessageDigest -import java.util.UUID - -import org.apache.hadoop.mapred.JobConf -import org.slf4j.LoggerFactory import scala.collection.JavaConverters._ import scala.util.{ Failure, Success, Try } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala b/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala index 5715e03a0c..4ab2364d17 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala @@ -20,7 +20,7 @@ import com.twitter.algebird.{ Monoid, Monad } import com.twitter.scalding.cascading_interop.FlowListenerPromise import com.twitter.scalding.Dsl.flowDefToRichFlowDef -import scala.concurrent.{ Await, Future, Promise, ExecutionContext => ConcurrentExecutionContext } +import scala.concurrent.{ Await, Future, ExecutionContext => ConcurrentExecutionContext } import scala.util.{ Failure, Success, Try } import cascading.flow.{ FlowDef, Flow } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/FlowState.scala b/scalding-core/src/main/scala/com/twitter/scalding/FlowState.scala index ccc7ab32a5..89f306524c 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/FlowState.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/FlowState.scala @@ -15,10 +15,9 @@ limitations under the License. */ package com.twitter.scalding -import cascading.pipe.Pipe import cascading.flow.FlowDef -import java.util.{ Map => JMap, WeakHashMap } -import scala.collection.JavaConverters._ +import java.util.WeakHashMap + /** * Immutable state that we attach to the Flow using the FlowStateMap */ diff --git a/scalding-core/src/main/scala/com/twitter/scalding/GroupBuilder.scala b/scalding-core/src/main/scala/com/twitter/scalding/GroupBuilder.scala index 0b6d40409c..e1c7d518e6 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/GroupBuilder.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/GroupBuilder.scala @@ -17,14 +17,9 @@ package com.twitter.scalding import cascading.pipe._ import cascading.pipe.assembly._ import cascading.operation._ -import cascading.operation.aggregator._ -import cascading.operation.filter._ import cascading.tuple.Fields -import cascading.tuple.{ Tuple => CTuple, TupleEntry } +import cascading.tuple.TupleEntry -import scala.collection.JavaConverters._ -import scala.annotation.tailrec -import scala.math.Ordering import scala.{ Range => ScalaRange } /** diff --git a/scalding-core/src/main/scala/com/twitter/scalding/IntegralComparator.scala b/scalding-core/src/main/scala/com/twitter/scalding/IntegralComparator.scala index a95de95bea..29b888854f 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/IntegralComparator.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/IntegralComparator.scala @@ -14,12 +14,12 @@ See the License for the specific language governing permissions and limitations under the License. */ -package com.twitter.scalding; +package com.twitter.scalding -import cascading.tuple.Hasher; +import cascading.tuple.Hasher -import java.io.Serializable; -import java.util.Comparator; +import java.io.Serializable +import java.util.Comparator /* * Handles numerical hashing properly diff --git a/scalding-core/src/main/scala/com/twitter/scalding/IterableSource.scala b/scalding-core/src/main/scala/com/twitter/scalding/IterableSource.scala index 2ae8045aed..9a575bd9dd 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/IterableSource.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/IterableSource.scala @@ -17,9 +17,6 @@ package com.twitter.scalding import com.twitter.maple.tap.MemorySourceTap -import cascading.flow.FlowProcess -import cascading.scheme.local.{ TextDelimited => CLTextDelimited } -import cascading.scheme.Scheme import cascading.tap.Tap import cascading.tuple.Tuple import cascading.tuple.Fields @@ -27,10 +24,6 @@ import cascading.scheme.NullScheme import java.io.{ InputStream, OutputStream } -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._ diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Job.scala b/scalding-core/src/main/scala/com/twitter/scalding/Job.scala index 7a2604cd77..37ac298b52 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Job.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Job.scala @@ -16,27 +16,21 @@ limitations under the License. package com.twitter.scalding import com.twitter.algebird.monad.Reader -import com.twitter.chill.config.{ ScalaAnyRefMapConfig, ConfiguredInstantiator } -import cascading.pipe.assembly.AggregateBy -import cascading.flow.{ Flow, FlowDef, FlowProps, FlowListener, FlowStep, FlowStepListener, FlowSkipStrategy, FlowStepStrategy } +import cascading.flow.{ Flow, FlowDef, FlowListener, FlowStep, FlowStepListener, FlowSkipStrategy, FlowStepStrategy } import cascading.pipe.Pipe import cascading.property.AppProps -import cascading.tuple.collect.SpillableProps import cascading.stats.CascadingStats -import com.twitter.scalding.reducer_estimation.EstimatorConfig import org.apache.hadoop.io.serializer.{ Serialization => HSerialization } -import org.apache.hadoop.mapred.JobConf -import org.slf4j.LoggerFactory -//For java -> scala implicits on collections +// For java -> scala implicits on collections import scala.collection.JavaConversions._ import scala.concurrent.{ Future, Promise } import scala.util.Try -import java.io.{ BufferedWriter, File, FileOutputStream, OutputStreamWriter } -import java.util.{ Calendar, UUID, List => JList } +import java.io.{ BufferedWriter, FileOutputStream, OutputStreamWriter } +import java.util.{ List => JList } import java.util.concurrent.{ Executors, TimeUnit, ThreadFactory, Callable, TimeoutException } import java.util.concurrent.atomic.AtomicInteger diff --git a/scalding-core/src/main/scala/com/twitter/scalding/JobStats.scala b/scalding-core/src/main/scala/com/twitter/scalding/JobStats.scala index 5529f76816..dac1f1a720 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/JobStats.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/JobStats.scala @@ -15,9 +15,7 @@ limitations under the License. */ package com.twitter.scalding -import java.io.{ File, OutputStream } import scala.collection.JavaConverters._ -import cascading.flow.Flow import cascading.stats.{ CascadeStats, CascadingStats, FlowStats } import scala.util.{ Failure, Try } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/JoinAlgorithms.scala b/scalding-core/src/main/scala/com/twitter/scalding/JoinAlgorithms.scala index 19d60874f6..c43e135d63 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/JoinAlgorithms.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/JoinAlgorithms.scala @@ -15,17 +15,9 @@ limitations under the License. */ package com.twitter.scalding -import cascading.tap._ -import cascading.scheme._ import cascading.pipe._ -import cascading.pipe.assembly._ import cascading.pipe.joiner._ -import cascading.flow._ -import cascading.operation._ -import cascading.operation.aggregator._ -import cascading.operation.filter._ import cascading.tuple._ -import cascading.cascade._ import java.util.Random // this one is serializable, scala.util.Random is not import scala.collection.JavaConverters._ diff --git a/scalding-core/src/main/scala/com/twitter/scalding/MemoryTap.scala b/scalding-core/src/main/scala/com/twitter/scalding/MemoryTap.scala index 5793e87aa4..8889b56dcc 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/MemoryTap.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/MemoryTap.scala @@ -21,7 +21,7 @@ import cascading.tuple._ import scala.collection.JavaConversions._ import cascading.scheme.Scheme import cascading.flow.FlowProcess -import collection.mutable.{ Buffer, MutableList } +import collection.mutable.Buffer class MemoryTap[In, Out](val scheme: Scheme[Properties, In, Out, _, _], val tupleBuffer: Buffer[Tuple]) extends Tap[Properties, In, Out](scheme) { diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Mode.scala b/scalding-core/src/main/scala/com/twitter/scalding/Mode.scala index 4547fb0abc..71e539fc3d 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Mode.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Mode.scala @@ -16,18 +16,17 @@ limitations under the License. package com.twitter.scalding import java.io.File -import java.util.{ Map => JMap, UUID, Properties } +import java.util.{ UUID, Properties } import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{ FileSystem, Path } import org.apache.hadoop.mapred.JobConf -import cascading.flow.{ FlowConnector, FlowDef, Flow } +import cascading.flow.FlowConnector import cascading.flow.hadoop.HadoopFlowProcess import cascading.flow.hadoop.HadoopFlowConnector import cascading.flow.local.LocalFlowConnector import cascading.flow.local.LocalFlowProcess -import cascading.pipe.Pipe import cascading.property.AppProps import cascading.tap.Tap import cascading.tuple.Tuple @@ -38,10 +37,9 @@ import scala.collection.JavaConverters._ import scala.collection.mutable.Buffer import scala.collection.mutable.{ Map => MMap } import scala.collection.mutable.{ Set => MSet } -import scala.collection.mutable.{ Iterable => MIterable } -import scala.util.{ Failure, Success, Try } +import scala.util.{ Failure, Success } -import org.slf4j.{ Logger, LoggerFactory } +import org.slf4j.LoggerFactory case class ModeException(message: String) extends RuntimeException(message) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Operations.scala b/scalding-core/src/main/scala/com/twitter/scalding/Operations.scala index 8225324a91..7e5a1a9507 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Operations.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Operations.scala @@ -19,14 +19,9 @@ package com.twitter.scalding { import cascading.tuple._ import cascading.flow._ import cascading.pipe.assembly.AggregateBy - import cascading.pipe._ import com.twitter.chill.MeatLocker import scala.collection.JavaConverters._ - import org.apache.hadoop.conf.Configuration - - import com.esotericsoftware.kryo.Kryo; - import com.twitter.algebird.{ Semigroup, SummingCache } import com.twitter.scalding.mathematics.Poisson import serialization.Externalizer diff --git a/scalding-core/src/main/scala/com/twitter/scalding/PartitionSource.scala b/scalding-core/src/main/scala/com/twitter/scalding/PartitionSource.scala index 84d8b4f133..fac95fefdd 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/PartitionSource.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/PartitionSource.scala @@ -15,13 +15,6 @@ limitations under the License. */ package com.twitter.scalding -import org.apache.hadoop.mapred.JobConf -import org.apache.hadoop.mapred.RecordReader -import org.apache.hadoop.mapred.OutputCollector - -import cascading.scheme.hadoop.{ TextDelimited => CHTextDelimited } -import cascading.scheme.hadoop.TextLine.Compress -import cascading.scheme.Scheme import cascading.tap.hadoop.Hfs import cascading.tap.hadoop.{ PartitionTap => HPartitionTap } import cascading.tap.local.FileTap diff --git a/scalding-core/src/main/scala/com/twitter/scalding/ReduceOperations.scala b/scalding-core/src/main/scala/com/twitter/scalding/ReduceOperations.scala index 40f66917c6..851dd08c80 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/ReduceOperations.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/ReduceOperations.scala @@ -16,10 +16,9 @@ limitations under the License. package com.twitter.scalding import cascading.tuple.Fields -import cascading.tuple.{ Tuple => CTuple, TupleEntry } +import cascading.tuple.{ Tuple => CTuple } import com.twitter.algebird.{ - Monoid, Semigroup, Ring, AveragedValue, diff --git a/scalding-core/src/main/scala/com/twitter/scalding/RichPipe.scala b/scalding-core/src/main/scala/com/twitter/scalding/RichPipe.scala index 4827478d65..1d3d29b2fa 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/RichPipe.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/RichPipe.scala @@ -15,18 +15,11 @@ limitations under the License. */ package com.twitter.scalding -import cascading.tap._ -import cascading.scheme._ import cascading.pipe._ -import cascading.pipe.assembly._ -import cascading.pipe.joiner._ import cascading.flow._ import cascading.operation._ -import cascading.operation.aggregator._ import cascading.operation.filter._ import cascading.tuple._ -import cascading.cascade._ -import cascading.operation.Debug.Output import scala.util.Random diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Source.scala b/scalding-core/src/main/scala/com/twitter/scalding/Source.scala index b7f98a5057..70ab2c4440 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Source.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Source.scala @@ -15,29 +15,23 @@ limitations under the License. */ package com.twitter.scalding -import java.io.{ File, InputStream, OutputStream } -import java.util.{ TimeZone, Calendar, Map => JMap, Properties } +import java.io.{ InputStream, OutputStream } +import java.util.{ Map => JMap, Properties } import cascading.flow.FlowDef import cascading.flow.FlowProcess -import cascading.flow.hadoop.HadoopFlowProcess -import cascading.flow.local.LocalFlowProcess import cascading.scheme.{ NullScheme, Scheme } import cascading.tap.hadoop.Hfs -import cascading.tap.{ MultiSourceTap, SinkMode } +import cascading.tap.SinkMode import cascading.tap.{ Tap, SinkTap } -import cascading.tap.local.FileTap import cascading.tuple.{ Fields, Tuple => CTuple, TupleEntry, TupleEntryCollector } import cascading.pipe.Pipe -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._ /** diff --git a/scalding-core/src/main/scala/com/twitter/scalding/StreamOperations.scala b/scalding-core/src/main/scala/com/twitter/scalding/StreamOperations.scala index e641bf1ad5..b7f2216b13 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/StreamOperations.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/StreamOperations.scala @@ -18,10 +18,6 @@ package com.twitter.scalding import cascading.tuple.Fields import cascading.tuple.{ Tuple => CTuple, TupleEntry } -import scala.collection.JavaConverters._ - -import Dsl._ //Get the conversion implicits - /** * Implements reductions on top of a simple abstraction for the Fields-API * We use the f-bounded polymorphism trick to return the type called Self diff --git a/scalding-core/src/main/scala/com/twitter/scalding/TemplateSource.scala b/scalding-core/src/main/scala/com/twitter/scalding/TemplateSource.scala index 9b6d889017..09d469ef50 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/TemplateSource.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/TemplateSource.scala @@ -15,13 +15,6 @@ limitations under the License. */ package com.twitter.scalding -import org.apache.hadoop.mapred.JobConf -import org.apache.hadoop.mapred.RecordReader -import org.apache.hadoop.mapred.OutputCollector - -import cascading.scheme.hadoop.{ TextDelimited => CHTextDelimited } -import cascading.scheme.hadoop.TextLine.Compress -import cascading.scheme.Scheme import cascading.tap.hadoop.Hfs import cascading.tap.hadoop.{ TemplateTap => HTemplateTap } import cascading.tap.local.FileTap diff --git a/scalding-core/src/main/scala/com/twitter/scalding/TimePathedSource.scala b/scalding-core/src/main/scala/com/twitter/scalding/TimePathedSource.scala index e2db997c0c..1379a66d65 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/TimePathedSource.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/TimePathedSource.scala @@ -18,9 +18,6 @@ package com.twitter.scalding import java.util.TimeZone import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileStatus -import org.apache.hadoop.fs.Path -import org.apache.hadoop.mapred.JobConf object TimePathedSource { val YEAR_MONTH_DAY = "/%1$tY/%1$tm/%1$td" diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Tool.scala b/scalding-core/src/main/scala/com/twitter/scalding/Tool.scala index d0bb880599..ad9af8449a 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Tool.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Tool.scala @@ -15,15 +15,13 @@ limitations under the License. */ package com.twitter.scalding -import org.apache.hadoop -import cascading.tuple.Tuple -import collection.mutable.{ ListBuffer, Buffer } +import org.apache.hadoop.conf.Configured +import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.util.{ GenericOptionsParser, Tool => HTool, ToolRunner } + import scala.annotation.tailrec -import scala.util.Try -import java.io.{ BufferedWriter, File, FileOutputStream, OutputStreamWriter } -import java.util.UUID -class Tool extends hadoop.conf.Configured with hadoop.util.Tool { +class Tool extends Configured with HTool { // This mutable state is not my favorite, but we are constrained by the Hadoop API: var rootJob: Option[(Args) => Job] = None @@ -53,7 +51,7 @@ class Tool extends hadoop.conf.Configured with hadoop.util.Tool { // and returns all the non-hadoop arguments. Should be called once if // you want to process hadoop arguments (like -libjars). protected def nonHadoopArgsFrom(args: Array[String]): Array[String] = { - (new hadoop.util.GenericOptionsParser(getConf, args)).getRemainingArgs + (new GenericOptionsParser(getConf, args)).getRemainingArgs } def parseModeArgs(args: Array[String]): (Mode, Args) = { @@ -125,7 +123,7 @@ class Tool extends hadoop.conf.Configured with hadoop.util.Tool { object Tool { def main(args: Array[String]) { try { - hadoop.util.ToolRunner.run(new hadoop.mapred.JobConf, new Tool, args) + ToolRunner.run(new JobConf, new Tool, args) } catch { case t: Throwable => { //re-throw the exception with extra info diff --git a/scalding-core/src/main/scala/com/twitter/scalding/TupleConversions.scala b/scalding-core/src/main/scala/com/twitter/scalding/TupleConversions.scala index 4e5caa7777..e52511d2af 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/TupleConversions.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/TupleConversions.scala @@ -15,14 +15,5 @@ limitations under the License. */ package com.twitter.scalding -import cascading.tuple.TupleEntry -import cascading.tuple.TupleEntryIterator -import cascading.tuple.{ Tuple => CTuple } -import cascading.tuple.Tuples - -import java.io.Serializable - -import scala.collection.JavaConverters._ - @deprecated("This trait does nothing now", "0.9.0") trait TupleConversions diff --git a/scalding-core/src/main/scala/com/twitter/scalding/TuplePacker.scala b/scalding-core/src/main/scala/com/twitter/scalding/TuplePacker.scala index efd6547f94..32262a8be0 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/TuplePacker.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/TuplePacker.scala @@ -15,8 +15,6 @@ limitations under the License. */ package com.twitter.scalding -import cascading.pipe._ -import cascading.pipe.joiner._ import cascading.tuple._ import java.lang.reflect.Method diff --git a/scalding-core/src/main/scala/com/twitter/scalding/TupleUnpacker.scala b/scalding-core/src/main/scala/com/twitter/scalding/TupleUnpacker.scala index 82fa09bb20..fbef771c81 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/TupleUnpacker.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/TupleUnpacker.scala @@ -15,12 +15,9 @@ limitations under the License. */ package com.twitter.scalding -import cascading.pipe._ -import cascading.pipe.joiner._ import cascading.tuple._ import scala.reflect.Manifest -import scala.collection.JavaConverters._ /** * Typeclass for objects which unpack an object into a tuple. diff --git a/scalding-core/src/test/scala/com/twitter/scalding/BlockJoinTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/BlockJoinTest.scala index c6353973cd..26a4f671d7 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/BlockJoinTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/BlockJoinTest.scala @@ -19,8 +19,6 @@ import org.scalatest.{ Matchers, WordSpec } import cascading.pipe.joiner._ -import java.lang.reflect.InvocationTargetException - import scala.collection.mutable.Buffer class InnerProductJob(args: Args) extends Job(args) { diff --git a/scalding-core/src/test/scala/com/twitter/scalding/CascadeTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/CascadeTest.scala index d0a280d7f1..819e30c62c 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/CascadeTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/CascadeTest.scala @@ -23,7 +23,6 @@ import scala.io.Source.fromFile import java.io.File import cascading.cascade.Cascade import cascading.flow.FlowSkipIfSinkNotStale -import cascading.tuple.Fields class Job1(args: Args) extends Job(args) { Tsv(args("input0"), ('line)).pipe.map[String, String]('line -> 'line)((x: String) => "job1:" + x).write(Tsv(args("output0"), fields = 'line)) diff --git a/scalding-core/src/test/scala/com/twitter/scalding/CoGroupTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/CoGroupTest.scala index c8b9cfc659..5d615be772 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/CoGroupTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/CoGroupTest.scala @@ -15,7 +15,6 @@ limitations under the License. */ package com.twitter.scalding -import cascading.pipe.joiner._ import org.scalatest.{ WordSpec, Matchers } class StarJoinJob(args: Args) extends Job(args) { diff --git a/scalding-core/src/test/scala/com/twitter/scalding/ConfigTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/ConfigTest.scala index ed7fa01708..a7543f991c 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/ConfigTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/ConfigTest.scala @@ -17,10 +17,8 @@ package com.twitter.scalding import org.scalatest.{ WordSpec, Matchers } import org.scalacheck.Arbitrary -import org.scalacheck.Arbitrary.arbitrary import org.scalacheck.Properties import org.scalacheck.Prop.forAll -import org.scalacheck.Gen._ import scala.util.Success diff --git a/scalding-core/src/test/scala/com/twitter/scalding/CumulitiveSumTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/CumulitiveSumTest.scala index 7f8fb5faf7..aa40e0eb91 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/CumulitiveSumTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/CumulitiveSumTest.scala @@ -2,8 +2,6 @@ package com.twitter.scalding import org.scalatest.WordSpec -import com.twitter.scalding._ - import com.twitter.scalding.typed.CumulativeSum._ class AddRankingWithCumulativeSum(args: Args) extends Job(args) { diff --git a/scalding-core/src/test/scala/com/twitter/scalding/DistinctByTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/DistinctByTest.scala index d5405b40ea..ed4be54953 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/DistinctByTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/DistinctByTest.scala @@ -17,11 +17,8 @@ package com.twitter.scalding import com.twitter.scalding.typed.CoGrouped.distinctBy -import org.scalacheck.Arbitrary -import org.scalacheck.Arbitrary.arbitrary import org.scalacheck.Properties import org.scalacheck.Prop.forAll -import org.scalacheck.Gen._ object DistinctByProps extends Properties("CoGrouped.DistinctBy") { diff --git a/scalding-core/src/test/scala/com/twitter/scalding/ExecutionAppProperties.scala b/scalding-core/src/test/scala/com/twitter/scalding/ExecutionAppProperties.scala index fdc8d6d028..2a7524b315 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/ExecutionAppProperties.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/ExecutionAppProperties.scala @@ -15,10 +15,8 @@ limitations under the License. */ package com.twitter.scalding -import org.scalacheck.Arbitrary import org.scalacheck.Properties import org.scalacheck.Prop.forAll -import org.scalacheck.Gen.choose import org.scalacheck.Prop._ // Be careful here in that Array[String] equality isn't contents based. its java referenced based. diff --git a/scalding-core/src/test/scala/com/twitter/scalding/KryoTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/KryoTest.scala index b59071a4d3..036edaa0c6 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/KryoTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/KryoTest.scala @@ -15,8 +15,6 @@ limitations under the License. */ package com.twitter.scalding -import com.twitter.scalding.serialization._ - import org.scalatest.{ Matchers, WordSpec } import java.io.{ ByteArrayOutputStream => BOS } @@ -39,6 +37,7 @@ import com.twitter.chill.hadoop.HadoopConfig import com.twitter.chill.hadoop.KryoSerialization import org.apache.hadoop.conf.Configuration + /* * This is just a test case for Kryo to deal with. It should * be outside KryoTest, otherwise the enclosing class, KryoTest diff --git a/scalding-core/src/test/scala/com/twitter/scalding/PartitionSourceTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/PartitionSourceTest.scala index 204f0f96fa..b9adb9a037 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/PartitionSourceTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/PartitionSourceTest.scala @@ -26,7 +26,6 @@ import cascading.tuple.Fields import cascading.tuple.TupleEntry import cascading.util.Util import cascading.tap.partition.Partition -import cascading.tap.partition.DelimitedPartition import com.twitter.scalding.{ PartitionedTsv => StandardPartitionedTsv, _ } diff --git a/scalding-core/src/test/scala/com/twitter/scalding/ReduceOperationsTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/ReduceOperationsTest.scala index 2a1c484b01..6dc773df65 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/ReduceOperationsTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/ReduceOperationsTest.scala @@ -16,7 +16,6 @@ limitations under the License. package com.twitter.scalding import org.scalatest.{ Matchers, WordSpec } -import com.twitter.scalding._ class SortWithTakeJob(args: Args) extends Job(args) { try { diff --git a/scalding-core/src/test/scala/com/twitter/scalding/ScanLeftTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/ScanLeftTest.scala index 21970aa1bf..154a20536c 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/ScanLeftTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/ScanLeftTest.scala @@ -1,7 +1,6 @@ package com.twitter.scalding import org.scalatest.{ Matchers, WordSpec } -import com.twitter.scalding._ /** * Simple Example: First group data by gender and then sort by height reverse order. diff --git a/scalding-core/src/test/scala/com/twitter/scalding/SideEffectTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/SideEffectTest.scala index 83ce7bfa30..54f3351e8c 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/SideEffectTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/SideEffectTest.scala @@ -15,8 +15,6 @@ limitations under the License. */ package com.twitter.scalding -import scala.annotation.tailrec -import cascading.pipe._ import org.scalatest.{ Matchers, WordSpec } /* diff --git a/scalding-core/src/test/scala/com/twitter/scalding/SkewJoinTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/SkewJoinTest.scala index eb6e0b6560..b018867de4 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/SkewJoinTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/SkewJoinTest.scala @@ -17,10 +17,6 @@ package com.twitter.scalding import org.scalatest.{ Matchers, WordSpec } -import cascading.pipe.joiner._ - -import java.lang.reflect.InvocationTargetException - import scala.collection.mutable.Buffer class SkewJoinJob(args: Args) extends Job(args) { diff --git a/scalding-core/src/test/scala/com/twitter/scalding/TemplateSourceTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/TemplateSourceTest.scala index 74809e6b4a..366b5c6676 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/TemplateSourceTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/TemplateSourceTest.scala @@ -21,9 +21,6 @@ import scala.io.{ Source => ScalaSource } import org.scalatest.{ Matchers, WordSpec } -import cascading.tap.SinkMode -import cascading.tuple.Fields - class TemplateTestJob(args: Args) extends Job(args) { try { Tsv("input", ('col1, 'col2)).read.write(TemplatedTsv("base", "%s", 'col1)) diff --git a/scalding-core/src/test/scala/com/twitter/scalding/TestTapFactoryTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/TestTapFactoryTest.scala index 152d822a2c..b071edabfc 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/TestTapFactoryTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/TestTapFactoryTest.scala @@ -2,7 +2,6 @@ package com.twitter.scalding import cascading.tap.Tap import cascading.tuple.{ Fields, Tuple } -import java.lang.IllegalArgumentException import scala.collection.mutable.Buffer import org.scalatest.{ Matchers, WordSpec } diff --git a/scalding-core/src/test/scala/com/twitter/scalding/TypedDelimitedTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/TypedDelimitedTest.scala index 4197e18e3f..16435b180f 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/TypedDelimitedTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/TypedDelimitedTest.scala @@ -16,7 +16,6 @@ limitations under the License. package com.twitter.scalding import org.scalatest.{ Matchers, WordSpec } -import com.twitter.scalding._ import com.twitter.scalding.source.DailySuffixTypedTsv class TypedTsvJob(args: Args) extends Job(args) { diff --git a/scalding-date/src/main/scala/com/twitter/scalding/DateOps.scala b/scalding-date/src/main/scala/com/twitter/scalding/DateOps.scala index dabe312fa7..9ce0c87d97 100644 --- a/scalding-date/src/main/scala/com/twitter/scalding/DateOps.scala +++ b/scalding-date/src/main/scala/com/twitter/scalding/DateOps.scala @@ -15,8 +15,6 @@ limitations under the License. */ package com.twitter.scalding -import java.util.Calendar -import java.util.Date import java.util.TimeZone import java.text.SimpleDateFormat diff --git a/scalding-date/src/test/scala/com/twitter/scalding/DateProperties.scala b/scalding-date/src/test/scala/com/twitter/scalding/DateProperties.scala index 3d3340efd6..3380d6f42d 100644 --- a/scalding-date/src/test/scala/com/twitter/scalding/DateProperties.scala +++ b/scalding-date/src/test/scala/com/twitter/scalding/DateProperties.scala @@ -21,7 +21,6 @@ import org.scalacheck.Prop.forAll import org.scalacheck.Gen.choose import org.scalacheck.Prop._ -import scala.util.control.Exception.allCatch import AbsoluteDuration.fromMillisecs object DateProperties extends Properties("Date Properties") { diff --git a/scalding-date/src/test/scala/com/twitter/scalding/GlobifierOps.scala b/scalding-date/src/test/scala/com/twitter/scalding/GlobifierOps.scala index 4cd4291c0a..55fca6a397 100644 --- a/scalding-date/src/test/scala/com/twitter/scalding/GlobifierOps.scala +++ b/scalding-date/src/test/scala/com/twitter/scalding/GlobifierOps.scala @@ -15,7 +15,6 @@ limitations under the License. */ package com.twitter.scalding -import com.twitter.scalding._ import java.util.TimeZone import scala.util.{ Try, Success, Failure } diff --git a/scalding-date/src/test/scala/com/twitter/scalding/GlobifierProperties.scala b/scalding-date/src/test/scala/com/twitter/scalding/GlobifierProperties.scala index 4724f29823..e79f6dc3dc 100644 --- a/scalding-date/src/test/scala/com/twitter/scalding/GlobifierProperties.scala +++ b/scalding-date/src/test/scala/com/twitter/scalding/GlobifierProperties.scala @@ -21,8 +21,6 @@ import org.scalacheck.Prop.forAll import org.scalacheck.Gen.choose import org.scalacheck.Prop._ -import scala.util.control.Exception.allCatch -import AbsoluteDuration.fromMillisecs import java.util.TimeZone object GlobifierProperties extends Properties("Globifier Properties") { diff --git a/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/HadoopPlatformJobTest.scala b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/HadoopPlatformJobTest.scala index c5e3cbb687..c0d527eb85 100644 --- a/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/HadoopPlatformJobTest.scala +++ b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/HadoopPlatformJobTest.scala @@ -22,8 +22,6 @@ import java.io.{ BufferedWriter, File, FileWriter } import org.apache.hadoop.mapred.JobConf -import scala.collection.mutable.Buffer - import org.slf4j.LoggerFactory /** diff --git a/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/LocalCluster.scala b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/LocalCluster.scala index 92ce1c6fa1..5cd14dcc76 100644 --- a/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/LocalCluster.scala +++ b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/LocalCluster.scala @@ -17,24 +17,12 @@ package com.twitter.scalding.platform import com.twitter.scalding._ -import java.io.{ - BufferedInputStream, - BufferedReader, - BufferedWriter, - DataInputStream, - DataOutputStream, - File, - FileInputStream, - FileOutputStream, - FileReader, - FileWriter, - RandomAccessFile -} +import java.io.{ File, RandomAccessFile } import java.nio.channels.FileLock import org.apache.hadoop.conf.Configuration import org.apache.hadoop.filecache.DistributedCache -import org.apache.hadoop.fs.{ FileSystem, FileUtil, Path } +import org.apache.hadoop.fs.{ FileUtil, Path } import org.apache.hadoop.hdfs.MiniDFSCluster import org.apache.hadoop.mapred.{ JobConf, MiniMRCluster } import org.slf4j.LoggerFactory diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala index 4975d6d2e0..5bba7fc79b 100644 --- a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala +++ b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala @@ -1,8 +1,7 @@ package com.twitter.scalding.reducer_estimation import com.twitter.scalding._ -import com.twitter.scalding.platform.{ HadoopPlatformJobTest, HadoopPlatformTest, LocalCluster } -import org.apache.hadoop.mapred.JobConf +import com.twitter.scalding.platform.{ HadoopPlatformJobTest, HadoopPlatformTest } import org.scalatest.{ Matchers, WordSpec } import scala.collection.JavaConverters._ diff --git a/scalding-json/src/main/scala/com/twitter/scalding/TypedJson.scala b/scalding-json/src/main/scala/com/twitter/scalding/TypedJson.scala index 9685059276..1dde3d1648 100644 --- a/scalding-json/src/main/scala/com/twitter/scalding/TypedJson.scala +++ b/scalding-json/src/main/scala/com/twitter/scalding/TypedJson.scala @@ -1,13 +1,10 @@ package com.twitter.scalding -import com.twitter.bijection._ import com.twitter.bijection.{ Injection, AbstractInjection } import com.twitter.bijection.Inversion._ -import com.twitter.scalding._ import com.twitter.elephantbird.cascading2.scheme.LzoTextLine import org.json4s._ -import org.json4s.native.JsonMethods._ import org.json4s.native.Serialization._ import org.json4s.{ NoTypeHints, native } diff --git a/scalding-repl/src/main/scala-2.10/com/twitter/scalding/ILoopCompat.scala b/scalding-repl/src/main/scala-2.10/com/twitter/scalding/ILoopCompat.scala index dbf9df69c2..0069d91f11 100644 --- a/scalding-repl/src/main/scala-2.10/com/twitter/scalding/ILoopCompat.scala +++ b/scalding-repl/src/main/scala-2.10/com/twitter/scalding/ILoopCompat.scala @@ -1,5 +1,5 @@ package com.twitter.scalding -import scala.tools.nsc.interpreter.{ ILoop, IMain } +import scala.tools.nsc.interpreter.ILoop trait ILoopCompat extends ILoop diff --git a/scalding-repl/src/main/scala/com/twitter/scalding/ScaldingShell.scala b/scalding-repl/src/main/scala/com/twitter/scalding/ScaldingShell.scala index c1febab692..e7f0dc03ca 100644 --- a/scalding-repl/src/main/scala/com/twitter/scalding/ScaldingShell.scala +++ b/scalding-repl/src/main/scala/com/twitter/scalding/ScaldingShell.scala @@ -23,7 +23,7 @@ import java.util.jar.JarOutputStream import org.apache.hadoop.util.GenericOptionsParser import org.apache.hadoop.conf.Configuration -import scala.tools.nsc.{ Settings, GenericRunnerCommand, MainGenericRunner } +import scala.tools.nsc.{ GenericRunnerCommand, MainGenericRunner } import scala.tools.nsc.interpreter.ILoop import scala.tools.nsc.io.VirtualDirectory diff --git a/scalding-repl/src/main/scala/com/twitter/scalding/ShellPipe.scala b/scalding-repl/src/main/scala/com/twitter/scalding/ShellPipe.scala index 7c3b62319d..f223625119 100644 --- a/scalding-repl/src/main/scala/com/twitter/scalding/ShellPipe.scala +++ b/scalding-repl/src/main/scala/com/twitter/scalding/ShellPipe.scala @@ -15,8 +15,6 @@ package com.twitter.scalding -import com.twitter.scalding.typed._ - /** * Enrichment on TypedPipes allowing them to be run locally, independent of the overall flow. * @param pipe to wrap From ffb044d968fc1d6913eb1652c52f08c2d9935a85 Mon Sep 17 00:00:00 2001 From: Tom Dyas Date: Wed, 1 Apr 2015 00:38:38 -0400 Subject: [PATCH 072/177] prefer JavaConverters over dangerous JavaConversions JavaConversions performs conversion between Scala and Java collections without any obvious indication in the source code. This is highly dangerous. Use JavaConverters instead which requires `asJava` and `asScala` method calls. --- .../com/twitter/scalding/FieldConversions.scala | 14 ++++++-------- .../src/main/scala/com/twitter/scalding/Job.scala | 2 -- .../scala/com/twitter/scalding/MemoryTap.scala | 7 ++++--- .../main/scala/com/twitter/scalding/Stats.scala | 6 +++--- 4 files changed, 13 insertions(+), 16 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/FieldConversions.scala b/scalding-core/src/main/scala/com/twitter/scalding/FieldConversions.scala index 343ac0c9c2..60885c1311 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/FieldConversions.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/FieldConversions.scala @@ -16,15 +16,13 @@ limitations under the License. package com.twitter.scalding import cascading.tuple.Fields - -import scala.collection.JavaConversions._ - import cascading.pipe.Pipe -import scala.annotation.tailrec -import java.util.Comparator - import com.esotericsoftware.kryo.DefaultSerializer +import java.util.Comparator +import scala.annotation.tailrec +import scala.collection.JavaConverters._ + trait LowPriorityFieldConversions { protected def anyToFieldArg(f: Any): Comparable[_] = f match { @@ -67,7 +65,7 @@ trait FieldConversions extends LowPriorityFieldConversions { // Cascading Fields are either java.lang.String or java.lang.Integer, both are comparable. def asList(f: Fields): List[Comparable[_]] = { - f.iterator.toList.asInstanceOf[List[Comparable[_]]] + f.iterator.asScala.toList.asInstanceOf[List[Comparable[_]]] } // Cascading Fields are either java.lang.String or java.lang.Integer, both are comparable. def asSet(f: Fields): Set[Comparable[_]] = asList(f).toSet @@ -75,7 +73,7 @@ trait FieldConversions extends LowPriorityFieldConversions { // TODO get the comparator also def getField(f: Fields, idx: Int): Fields = { new Fields(f.get(idx)) } - def hasInts(f: Fields): Boolean = f.iterator.exists { _.isInstanceOf[java.lang.Integer] } + def hasInts(f: Fields): Boolean = f.iterator.asScala.exists { _.isInstanceOf[java.lang.Integer] } /** * Rather than give the full power of cascading's selectors, we have diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Job.scala b/scalding-core/src/main/scala/com/twitter/scalding/Job.scala index 37ac298b52..f83ad7ab7a 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Job.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Job.scala @@ -24,8 +24,6 @@ import cascading.stats.CascadingStats import org.apache.hadoop.io.serializer.{ Serialization => HSerialization } -// For java -> scala implicits on collections -import scala.collection.JavaConversions._ import scala.concurrent.{ Future, Promise } import scala.util.Try diff --git a/scalding-core/src/main/scala/com/twitter/scalding/MemoryTap.scala b/scalding-core/src/main/scala/com/twitter/scalding/MemoryTap.scala index 8889b56dcc..896c63496a 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/MemoryTap.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/MemoryTap.scala @@ -18,10 +18,11 @@ package com.twitter.scalding import cascading.tap.Tap import java.util.Properties import cascading.tuple._ -import scala.collection.JavaConversions._ import cascading.scheme.Scheme import cascading.flow.FlowProcess -import collection.mutable.Buffer + +import scala.collection.mutable.Buffer +import scala.collection.JavaConverters._ class MemoryTap[In, Out](val scheme: Scheme[Properties, In, Out, _, _], val tupleBuffer: Buffer[Tuple]) extends Tap[Properties, In, Out](scheme) { @@ -44,7 +45,7 @@ class MemoryTap[In, Out](val scheme: Scheme[Properties, In, Out, _, _], val tupl override lazy val getIdentifier: String = scala.math.random.toString override def openForRead(flowProcess: FlowProcess[Properties], input: In) = { - new TupleEntryChainIterator(scheme.getSourceFields, tupleBuffer.toIterator) + new TupleEntryChainIterator(scheme.getSourceFields, tupleBuffer.toIterator.asJava) } override def openForWrite(flowProcess: FlowProcess[Properties], output: Out): TupleEntryCollector = { diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Stats.scala b/scalding-core/src/main/scala/com/twitter/scalding/Stats.scala index ce55eb5ff1..9029110bc9 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Stats.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Stats.scala @@ -4,7 +4,6 @@ import cascading.flow.{ FlowDef, FlowProcess } import cascading.stats.CascadingStats import java.util.concurrent.ConcurrentHashMap import org.slf4j.{ Logger, LoggerFactory } -import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ import scala.collection.mutable import scala.ref.WeakReference @@ -108,8 +107,9 @@ object UniqueID { object RuntimeStats extends java.io.Serializable { @transient private lazy val logger: Logger = LoggerFactory.getLogger(this.getClass) - private val flowMappingStore: mutable.Map[String, WeakReference[FlowProcess[_]]] = - new ConcurrentHashMap[String, WeakReference[FlowProcess[_]]] + private val flowMappingStore: mutable.Map[String, WeakReference[FlowProcess[_]]] = { + (new ConcurrentHashMap[String, WeakReference[FlowProcess[_]]]).asScala + } def getFlowProcessForUniqueId(uniqueId: UniqueID): FlowProcess[_] = { (for { From 99dc54b8900c2301f8102165a0fd266fa419f4d2 Mon Sep 17 00:00:00 2001 From: Tom Dyas Date: Wed, 1 Apr 2015 15:19:37 -0400 Subject: [PATCH 073/177] remove more unused imports --- .../scala/com/twitter/scalding/bdd/BddDsl.scala | 2 -- .../scala/com/twitter/scalding/bdd/TBddDsl.scala | 2 -- .../bdd/TypedPipeOperationsConversions.scala | 2 +- .../examples/WeightedPageRankFromMatrix.scala | 2 -- .../serialization/CascadingBinaryComparator.scala | 3 ++- .../scalding/serialization/Externalizer.scala | 2 ++ .../scalding/serialization/KryoHadoop.scala | 14 -------------- .../scalding/serialization/KryoSerializers.scala | 11 ----------- .../scalding/serialization/MurmurHashUtils.scala | 8 -------- .../scalding/serialization/Serialization.scala | 2 +- .../scalding/serialization/Serialization2.scala | 2 +- .../serialization/StringOrderedSerialization.scala | 2 +- .../serialization/WrappedSerialization.scala | 2 +- 13 files changed, 9 insertions(+), 45 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/bdd/BddDsl.scala b/scalding-core/src/main/scala/com/twitter/scalding/bdd/BddDsl.scala index 1ca5fe3fdd..65592dac5e 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/bdd/BddDsl.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/bdd/BddDsl.scala @@ -3,9 +3,7 @@ package com.twitter.scalding.bdd import com.twitter.scalding._ import scala.collection.mutable.Buffer import cascading.tuple.Fields -import scala.Predef._ import com.twitter.scalding.Tsv -import org.slf4j.LoggerFactory trait BddDsl extends FieldConversions with PipeOperationsConversions { def Given(source: TestSource): TestCaseGiven1 = new TestCaseGiven1(source) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/bdd/TBddDsl.scala b/scalding-core/src/main/scala/com/twitter/scalding/bdd/TBddDsl.scala index 3594ac2198..ce7440f1cb 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/bdd/TBddDsl.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/bdd/TBddDsl.scala @@ -3,8 +3,6 @@ package com.twitter.scalding.bdd import cascading.flow.FlowDef import com.twitter.scalding._ import scala.collection.mutable.Buffer -import cascading.tuple.Fields -import scala.Predef._ import TDsl._ trait TBddDsl extends FieldConversions with TypedPipeOperationsConversions { diff --git a/scalding-core/src/main/scala/com/twitter/scalding/bdd/TypedPipeOperationsConversions.scala b/scalding-core/src/main/scala/com/twitter/scalding/bdd/TypedPipeOperationsConversions.scala index d62c49537d..967f80486d 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/bdd/TypedPipeOperationsConversions.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/bdd/TypedPipeOperationsConversions.scala @@ -1,7 +1,7 @@ package com.twitter.scalding.bdd import com.twitter.scalding.TypedPipe -import com.twitter.scalding.{ Dsl, RichPipe } +import com.twitter.scalding.Dsl trait TypedPipeOperationsConversions { import Dsl._ diff --git a/scalding-core/src/main/scala/com/twitter/scalding/examples/WeightedPageRankFromMatrix.scala b/scalding-core/src/main/scala/com/twitter/scalding/examples/WeightedPageRankFromMatrix.scala index 852cffc3b2..32e9dee952 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/examples/WeightedPageRankFromMatrix.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/examples/WeightedPageRankFromMatrix.scala @@ -1,7 +1,5 @@ package com.twitter.scalding.examples -import scala.collection._ - import com.twitter.scalding._ import com.twitter.scalding.mathematics.{ Matrix, ColVector } import com.twitter.scalding.mathematics.Matrix._ diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/CascadingBinaryComparator.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/CascadingBinaryComparator.scala index 20135a729d..abed86b59b 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/CascadingBinaryComparator.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/CascadingBinaryComparator.scala @@ -19,9 +19,10 @@ package com.twitter.scalding.serialization import java.io.InputStream import java.util.Comparator import cascading.flow.FlowDef -import cascading.tuple.{ Fields, Hasher => CHasher, StreamComparator } +import cascading.tuple.{ Hasher => CHasher, StreamComparator } import scala.util.{ Failure, Success, Try } + /** * This is the type that should be fed to cascading to enable binary comparators */ diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Externalizer.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Externalizer.scala index 0946454c12..dbe5ca4826 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Externalizer.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Externalizer.scala @@ -14,12 +14,14 @@ See the License for the specific language governing permissions and limitations under the License. */ package com.twitter.scalding.serialization + import com.twitter.chill.{ Externalizer => ChillExtern } import com.esotericsoftware.kryo.DefaultSerializer import com.esotericsoftware.kryo.serializers.JavaSerializer import com.twitter.chill.config.ScalaAnyRefMapConfig + /** * We need to control the Kryo created */ diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala index ac0ee43722..0a4420262a 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala @@ -15,23 +15,9 @@ limitations under the License. */ package com.twitter.scalding.serialization -import java.io.InputStream -import java.io.OutputStream -import java.io.Serializable -import java.nio.ByteBuffer - import com.esotericsoftware.kryo.Kryo -import com.esotericsoftware.kryo.{ Serializer => KSerializer } -import com.esotericsoftware.kryo.io.{ Input, Output } import com.esotericsoftware.kryo.serializers.FieldSerializer -import cascading.tuple.hadoop.TupleSerialization -import cascading.tuple.hadoop.io.BufferedInputStream - -import scala.annotation.tailrec -import scala.collection.immutable.ListMap -import scala.collection.immutable.HashMap - import com.twitter.scalding.DateRange import com.twitter.scalding.RichDate import com.twitter.scalding.Args diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoSerializers.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoSerializers.scala index 4e20dc59f6..96999f198c 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoSerializers.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoSerializers.scala @@ -15,21 +15,10 @@ limitations under the License. */ package com.twitter.scalding.serialization -import java.io.InputStream -import java.io.OutputStream -import java.io.Serializable -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.io.{ Input, Output } -import scala.annotation.tailrec -import scala.collection.immutable.ListMap -import scala.collection.mutable.{ Map => MMap } - import com.twitter.scalding._ /** diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/MurmurHashUtils.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/MurmurHashUtils.scala index 0ae55c1a43..27d4c54ed3 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/MurmurHashUtils.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/MurmurHashUtils.scala @@ -15,14 +15,6 @@ limitations under the License. */ package com.twitter.scalding.serialization -import java.io.InputStream -import scala.reflect.ClassTag -import scala.collection.generic.CanBuildFrom -import java.lang.Integer.{ rotateLeft => rotl } - -// Be careful using this, the product/array or similar will attempt to call system hash codes. -import scala.util.hashing.MurmurHash3 - // Taking a few functions from: // https://guava-libraries.googlecode.com/git/guava/src/com/google/common/hash/Murmur3_32HashFunction.java object MurmurHashUtils { diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization.scala index 4de221d9f5..b038253432 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization.scala @@ -18,7 +18,7 @@ package com.twitter.scalding.serialization import java.io.{ ByteArrayInputStream, ByteArrayOutputStream, InputStream, OutputStream, Serializable } -import scala.util.{ Failure, Success, Try } +import scala.util.{ Success, Try } import scala.util.hashing.Hashing /** diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization2.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization2.scala index 3acd4635bc..e364476854 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization2.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization2.scala @@ -16,7 +16,7 @@ limitations under the License. package com.twitter.scalding.serialization -import java.io.{ ByteArrayInputStream, ByteArrayOutputStream, InputStream, OutputStream, Serializable } +import java.io.{ InputStream, OutputStream } import scala.util.{ Failure, Success, Try } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/StringOrderedSerialization.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/StringOrderedSerialization.scala index c8c527d551..cbbb55f710 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/StringOrderedSerialization.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/StringOrderedSerialization.scala @@ -17,7 +17,7 @@ limitations under the License. package com.twitter.scalding.serialization import java.io.{ InputStream, OutputStream } -import scala.util.{ Failure, Success, Try } +import scala.util.{ Failure, Success } import scala.util.control.NonFatal import JavaStreamEnrichments._ diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/WrappedSerialization.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/WrappedSerialization.scala index 566d701031..961a09f909 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/WrappedSerialization.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/WrappedSerialization.scala @@ -18,7 +18,7 @@ package com.twitter.scalding.serialization import org.apache.hadoop.io.serializer.{ Serialization => HSerialization, Deserializer, Serializer } import org.apache.hadoop.conf.{ Configurable, Configuration } -import java.io.{ DataInputStream, DataOutputStream, InputStream, OutputStream } +import java.io.{ InputStream, OutputStream } import com.twitter.bijection.{ Injection, JavaSerializationInjection, Base64String } import scala.collection.JavaConverters._ From 7aec7452902551e6e9aefabd69b142f790c968dd Mon Sep 17 00:00:00 2001 From: Tom Dyas Date: Wed, 1 Apr 2015 15:25:06 -0400 Subject: [PATCH 074/177] remove unused imports from maple java code Includes removal of a reference to an internal Sun API. --- maple/src/main/java/com/twitter/maple/hbase/HBaseScheme.java | 1 - maple/src/main/java/com/twitter/maple/hbase/HBaseTap.java | 3 --- 2 files changed, 4 deletions(-) diff --git a/maple/src/main/java/com/twitter/maple/hbase/HBaseScheme.java b/maple/src/main/java/com/twitter/maple/hbase/HBaseScheme.java index 6ee34d5404..0f830ede86 100644 --- a/maple/src/main/java/com/twitter/maple/hbase/HBaseScheme.java +++ b/maple/src/main/java/com/twitter/maple/hbase/HBaseScheme.java @@ -31,7 +31,6 @@ import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.OutputCollector; import org.apache.hadoop.mapred.RecordReader; -import org.mortbay.log.Log; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/maple/src/main/java/com/twitter/maple/hbase/HBaseTap.java b/maple/src/main/java/com/twitter/maple/hbase/HBaseTap.java index f65b3db0b3..37ebfb0a8e 100644 --- a/maple/src/main/java/com/twitter/maple/hbase/HBaseTap.java +++ b/maple/src/main/java/com/twitter/maple/hbase/HBaseTap.java @@ -17,7 +17,6 @@ import cascading.flow.FlowProcess; import cascading.tap.SinkMode; import cascading.tap.Tap; -import cascading.tap.hadoop.io.HadoopTupleEntrySchemeCollector; import cascading.tap.hadoop.io.HadoopTupleEntrySchemeIterator; import cascading.tuple.TupleEntryCollector; import cascading.tuple.TupleEntryIterator; @@ -33,10 +32,8 @@ import org.apache.hadoop.mapred.RecordReader; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import sun.reflect.generics.reflectiveObjects.NotImplementedException; import java.io.IOException; -import java.util.Map.Entry; import java.util.UUID; /** From 6561a06549e78cf493ba326a2270e4000ef0a21f Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Wed, 1 Apr 2015 10:51:44 -1000 Subject: [PATCH 075/177] Optimize string and (hopefully) number comparisons a bit --- .../serialization/PositionInputStream.scala | 11 +++-- .../StringOrderedSerialization.scala | 44 +++++++------------ .../providers/PrimitiveOrderedBuf.scala | 3 +- 3 files changed, 26 insertions(+), 32 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/PositionInputStream.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/PositionInputStream.scala index 434adfc610..0700528842 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/PositionInputStream.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/PositionInputStream.scala @@ -42,20 +42,22 @@ class PositionInputStream(val wraps: InputStream) extends InputStream { override val markSupported: Boolean = wraps.markSupported override def read: Int = { + // returns -1 on eof or 0 to 255 store 1 byte. val result = wraps.read - // returns -1 on eof, otherwise non-negative number if (result >= 0) pos += 1 result } override def read(bytes: Array[Byte]): Int = { val count = wraps.read(bytes) - if (count > 0) pos += count + // Make this branch true as much as possible to improve branch prediction + if (count >= 0) pos += count count } override def read(bytes: Array[Byte], off: Int, len: Int): Int = { val count = wraps.read(bytes, off, len) - if (count > 0) pos += count + // Make this branch true as much as possible to improve branch prediction + if (count >= 0) pos += count count } @@ -70,7 +72,8 @@ class PositionInputStream(val wraps: InputStream) extends InputStream { override def skip(n: Long): Long = { if (n < 0) illegal("Must seek fowards") val count = skip(n) - if (count > 0) pos += count + // Make this branch true as much as possible to improve branch prediction + if (count >= 0) pos += count count } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/StringOrderedSerialization.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/StringOrderedSerialization.scala index c8c527d551..ea0ffd8aad 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/StringOrderedSerialization.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/StringOrderedSerialization.scala @@ -24,23 +24,12 @@ import JavaStreamEnrichments._ object StringOrderedSerialization { final def binaryIntCompare(leftSize: Int, seekingLeft: InputStream, rightSize: Int, seekingRight: InputStream): Int = { - val toCheck = math.min(leftSize, rightSize) - // we can check longs at a time this way: - val longs = toCheck / 8 - - @annotation.tailrec - def compareLong(count: Int): Int = - if (count == 0) 0 - else { - val cmp = UnsignedComparisons.unsignedLongCompare(seekingLeft.readLong, seekingRight.readLong) - if (cmp == 0) compareLong(count - 1) - else cmp - } - /* * This algorithm only works if count in {0, 1, 2, 3}. Since we only * call it that way below it is safe. */ + + @inline def compareBytes(count: Int): Int = if ((count & 2) == 2) { // there are 2 or 3 bytes to read @@ -60,22 +49,23 @@ object StringOrderedSerialization { /** * Now we start by comparing blocks of longs, then maybe 1 int, then 0 - 3 bytes */ - val lc = compareLong(longs) - if (lc != 0) lc + val toCheck = math.min(leftSize, rightSize) + val ints = toCheck / 4 + var counter = ints + var ic = 0 + while ((counter > 0) && (ic == 0)) { + // Unsigned compare of ints is cheaper than longs, because we can do it + // by upcasting to Long + ic = UnsignedComparisons.unsignedIntCompare(seekingLeft.readInt, seekingRight.readInt) + counter = counter - 1 + } + if (ic != 0) ic else { - val remaining = (toCheck - 8 * longs) - val read1Int = (remaining >= 4) - - val ic = if (read1Int) UnsignedComparisons.unsignedIntCompare(seekingLeft.readInt, seekingRight.readInt) else 0 - if (ic != 0) ic + val bc = compareBytes(toCheck - 4 * ints) + if (bc != 0) bc else { - val bytes = remaining - (if (read1Int) 4 else 0) - val bc = compareBytes(bytes) - if (bc != 0) bc - else { - // the size is the fallback when the prefixes match: - Integer.compare(leftSize, rightSize) - } + // the size is the fallback when the prefixes match: + Integer.compare(leftSize, rightSize) } } } diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/PrimitiveOrderedBuf.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/PrimitiveOrderedBuf.scala index d224a6202d..e036592551 100644 --- a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/PrimitiveOrderedBuf.scala +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/PrimitiveOrderedBuf.scala @@ -76,7 +76,8 @@ object PrimitiveOrderedBuf { val bbPutter = newTermName("write" + shortName) def genBinaryCompare(inputStreamA: TermName, inputStreamB: TermName): Tree = - if (Set("Float", "Double", "Character").contains(javaTypeStr)) { + //if (Set("Float", "Double", "Character").contains(javaTypeStr)) { + if (true) { // These cannot be compared using byte-wise approach q"""_root_.java.lang.$javaType.compare($inputStreamA.$bbGetter, $inputStreamB.$bbGetter)""" } else { From 7e99b01aefb1821bb913c6b187da2efbe41eea5f Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Wed, 1 Apr 2015 13:42:46 -1000 Subject: [PATCH 076/177] Cleanup and bugfix --- .../serialization/PositionInputStream.scala | 2 +- .../providers/PrimitiveOrderedBuf.scala | 30 +------------------ 2 files changed, 2 insertions(+), 30 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/PositionInputStream.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/PositionInputStream.scala index 0700528842..3407f61307 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/PositionInputStream.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/PositionInputStream.scala @@ -71,7 +71,7 @@ class PositionInputStream(val wraps: InputStream) extends InputStream { override def skip(n: Long): Long = { if (n < 0) illegal("Must seek fowards") - val count = skip(n) + val count = wraps.skip(n) // Make this branch true as much as possible to improve branch prediction if (count >= 0) pos += count count diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/PrimitiveOrderedBuf.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/PrimitiveOrderedBuf.scala index e036592551..8be80e8ecc 100644 --- a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/PrimitiveOrderedBuf.scala +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/PrimitiveOrderedBuf.scala @@ -76,35 +76,7 @@ object PrimitiveOrderedBuf { val bbPutter = newTermName("write" + shortName) def genBinaryCompare(inputStreamA: TermName, inputStreamB: TermName): Tree = - //if (Set("Float", "Double", "Character").contains(javaTypeStr)) { - if (true) { - // These cannot be compared using byte-wise approach - q"""_root_.java.lang.$javaType.compare($inputStreamA.$bbGetter, $inputStreamB.$bbGetter)""" - } else { - // Big endian numbers can be compared byte by byte - (0 until lenInBytes).map { i => - if (i == 0) { - //signed for the first byte - q"""_root_.java.lang.Byte.compare($inputStreamA.readByte, $inputStreamB.readByte)""" - } else { - q"""_root_.java.lang.Integer.compare($inputStreamA.readUnsignedByte, $inputStreamB.readUnsignedByte)""" - } - } - .toList - .reverse // go through last to first - .foldLeft(None: Option[Tree]) { - case (Some(rest), next) => - val nextV = newTermName("next") - Some( - q"""val $nextV = $next - if ($nextV != 0) $nextV - else { - $rest - } - """) - case (None, next) => Some(q"""$next""") - }.get // there must be at least one item because no primitive is zero bytes - } + q"""_root_.java.lang.$javaType.compare($inputStreamA.$bbGetter, $inputStreamB.$bbGetter)""" def accessor(e: c.TermName): c.Tree = { val primitiveAccessor = newTermName(shortName.toLowerCase + "Value") From 2056b281964a112cc66b6420a54dd1e748103481 Mon Sep 17 00:00:00 2001 From: Gera Shegalov Date: Wed, 1 Apr 2015 18:34:29 -0700 Subject: [PATCH 077/177] Swap hash and filename to accommodate for filename-extension-sensitive code --- .../scalding/filecache/DistributedCacheFile.scala | 2 +- .../filecache/DistributedCacheFileSpec.scala | 13 +++++++------ 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/filecache/DistributedCacheFile.scala b/scalding-core/src/main/scala/com/twitter/scalding/filecache/DistributedCacheFile.scala index 33dbc1845a..0dbe743d28 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/filecache/DistributedCacheFile.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/filecache/DistributedCacheFile.scala @@ -75,7 +75,7 @@ object DistributedCacheFile { val hexsum = URIHasher(uri) val fileName = new File(uri.toString).getName - Seq(fileName, hexsum).mkString("-") + Seq(hexsum, fileName).mkString("-") } def symlinkedUriFor(sourceUri: URI): URI = diff --git a/scalding-core/src/test/scala/com/twitter/scalding/filecache/DistributedCacheFileSpec.scala b/scalding-core/src/test/scala/com/twitter/scalding/filecache/DistributedCacheFileSpec.scala index 70aec69590..4bca622e3e 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/filecache/DistributedCacheFileSpec.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/filecache/DistributedCacheFileSpec.scala @@ -22,11 +22,11 @@ import java.net.URI import org.apache.hadoop.conf.Configuration import org.scalatest.{ Matchers, WordSpec } import scala.collection.mutable -/* -TODO: fix? is it worth having the dep on mockito just for this? + +// TODO: fix? is it worth having the dep on mockito just for this? class DistributedCacheFileSpec extends WordSpec with Matchers { case class UnknownMode(buffers: Map[Source, mutable.Buffer[Tuple]]) extends TestMode with CascadingLocal - + /* val conf = smartMock[Configuration] lazy val hdfsMode = { @@ -44,11 +44,11 @@ class DistributedCacheFileSpec extends WordSpec with Matchers { lazy val testMode = smartMock[Test] lazy val localMode = smartMock[Local] - +*/ val uriString = "hdfs://foo.example:1234/path/to/the/stuff/thefilename.blah" val uri = new URI(uriString) val hashHex = URIHasher(uri) - val hashedFilename = "thefilename.blah-" + hashHex + val hashedFilename = hashHex + "-thefilename.blah" "DistributedCacheFile" should { "symlinkNameFor must return a hashed name" in { @@ -56,6 +56,7 @@ class DistributedCacheFileSpec extends WordSpec with Matchers { } } + /* "UncachedFile.add" should { val dcf = new UncachedFile(Right(uri)) @@ -81,5 +82,5 @@ class DistributedCacheFileSpec extends WordSpec with Matchers { an[RuntimeException] should be thrownBy (dcf.add()(mode)) } } + */ } -*/ From b3f7f275ca9d8cae4bb9e9d73105de38c747f0ae Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Wed, 1 Apr 2015 15:39:41 -1000 Subject: [PATCH 078/177] Improve Execution semantics --- .../com/twitter/scalding/Execution.scala | 161 ++++++++---------- 1 file changed, 68 insertions(+), 93 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala b/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala index 5715e03a0c..8078120b4d 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala @@ -44,7 +44,7 @@ import cascading.flow.{ FlowDef, Flow } * zip to flatMap if you want to run two Executions in parallel. */ sealed trait Execution[+T] extends java.io.Serializable { - import Execution.{ emptyCache, EvalCache, FactoryExecution, FlatMapped, MapCounters, Mapped, OnComplete, RecoverWith, Zipped } + import Execution.{ EvalCache, FactoryExecution, FlatMapped, MapCounters, Mapped, OnComplete, RecoverWith, Zipped } /** * Scala uses the filter method in for syntax for pattern matches that can fail. @@ -128,7 +128,7 @@ sealed trait Execution[+T] extends java.io.Serializable { * Seriously: pro-style is for this to be called only once in a program. */ def run(conf: Config, mode: Mode)(implicit cec: ConcurrentExecutionContext): Future[T] = - runStats(conf, mode, emptyCache)(cec)._2.map(_._1) + runStats(conf, mode, new EvalCache)(cec).map(_._1) /** * This is the internal method that must be implemented @@ -139,7 +139,7 @@ sealed trait Execution[+T] extends java.io.Serializable { */ protected def runStats(conf: Config, mode: Mode, - cache: EvalCache)(implicit cec: ConcurrentExecutionContext): (EvalCache, Future[(T, ExecutionCounters, EvalCache)]) + cache: EvalCache)(implicit cec: ConcurrentExecutionContext): Future[(T, ExecutionCounters)] /** * This is convenience for when we don't care about the result. @@ -196,89 +196,63 @@ object Execution { override def join[T, U](t: Execution[T], u: Execution[U]): Execution[(T, U)] = t.zip(u) } - trait EvalCache { self => - /** - * For a given execution, return the EvalCache before the future is executed, - * and a Future of the result, counters, and cache after - * This takes care of merging the input cache with cache in the future - * result, so you don't need to worry about that (but it wouldn't be an - * error to add something to the cache twice clearly). - */ - def getOrElseInsert[T](ex: Execution[T], - res: => (EvalCache, Future[(T, ExecutionCounters, EvalCache)]))(implicit ec: ConcurrentExecutionContext): (EvalCache, Future[(T, ExecutionCounters, EvalCache)]) - - def ++(that: EvalCache): EvalCache = new EvalCache { - def getOrElseInsert[T](ex: Execution[T], - res: => (EvalCache, Future[(T, ExecutionCounters, EvalCache)]))(implicit ec: ConcurrentExecutionContext) = - that.getOrElseInsert(ex, self.getOrElseInsert(ex, res)) - } - } /** - * This is an implementation that remembers history forever. - * Since Hadoop jobs are generally long running and not infinite loops, - * this is generally safe. If someone wants to make an infinite loop or giant loop, - * this may OOM. The solution might be use an immutable LRU cache. - */ - private case class MapEvalCache(cache: Map[Execution[_], Future[(_, ExecutionCounters, EvalCache)]]) extends EvalCache { - def getOrElseInsert[T](ex: Execution[T], res: => (EvalCache, Future[(T, ExecutionCounters, EvalCache)]))(implicit ec: ConcurrentExecutionContext) = cache.get(ex) match { - case None => - val (next, fut) = res - // Make sure ex is added to the cache: - val resCache = next ++ MapEvalCache(cache + (ex -> fut)) - /* - * Note in this branch, the future returned includes a - * next and the ex -> fut mapping - */ - (resCache, fut.map { case (t, ec, fcache) => (t, ec, resCache ++ fcache) }) + * This is a mutable state that is kept internal to an execution + * as it is evaluating. + */ + private[scalding] class EvalCache { + private[this] val lock = new AnyRef + private[this] val cache = + scala.collection.mutable.Map[Execution[Any], Future[(Any, ExecutionCounters)]]() - case Some(fut) => - /* - * The future recorded here may not itself it it's inner cache - * (nothing else is ensuring it). So we make sure the same way we do above + def getOrElseInsert[T](ex: Execution[T], + res: => Future[(T, ExecutionCounters)])(implicit ec: ConcurrentExecutionContext): Future[(T, ExecutionCounters)] = lock.synchronized { + cache.get(ex) match { + case Some(fut) => fut.asInstanceOf[Future[(T, ExecutionCounters)]] + case None => + /* + * The evaluation of res could wind up looking into the map for this same + * Execution. So, we first put in a Promise, and then we complete with res */ - val typedFut = fut.asInstanceOf[Future[(T, ExecutionCounters, EvalCache)]] - (this, typedFut.map { case (t, ec, fcache) => (t, ec, this ++ fcache) }) - } - override def ++(that: EvalCache): EvalCache = that match { - case MapEvalCache(thatCache) => MapEvalCache(cache ++ thatCache) - case _ => super.++(that) + val promise = Promise[(T, ExecutionCounters)]() + cache += (ex -> promise.future) + val fut = res + promise.completeWith(fut) + fut + } } } - private def emptyCache: EvalCache = MapEvalCache(Map.empty) private case class FutureConst[T](get: ConcurrentExecutionContext => Future[T]) extends Execution[T] { def runStats(conf: Config, mode: Mode, cache: EvalCache)(implicit cec: ConcurrentExecutionContext) = - cache.getOrElseInsert(this, { - val fft: Future[Future[T]] = toFuture(Try(get(cec))) - (cache, for { - futt <- fft + cache.getOrElseInsert(this, + for { + futt <- toFuture(Try(get(cec))) t <- futt - } yield (t, ExecutionCounters.empty, cache)) - }) + } yield (t, ExecutionCounters.empty) + ) // Note that unit is not optimized away, since Futures are often used with side-effects, so, // we ensure that get is always called in contrast to Mapped, which assumes that fn is pure. } private case class FlatMapped[S, T](prev: Execution[S], fn: S => Execution[T]) extends Execution[T] { def runStats(conf: Config, mode: Mode, cache: EvalCache)(implicit cec: ConcurrentExecutionContext) = - cache.getOrElseInsert(this, { - val (cache1, fut) = prev.runStats(conf, mode, cache) - val finalFut = for { - (s, st1, cache1a) <- fut + cache.getOrElseInsert(this, + for { + (s, st1) <- prev.runStats(conf, mode, cache) next = fn(s) - (_, fut2) = next.runStats(conf, mode, cache1a) - (t, st2, cache2a) <- fut2 - } yield (t, Monoid.plus(st1, st2), cache2a) - (cache1, finalFut) - }) + fut2 = next.runStats(conf, mode, cache) + (t, st2) <- fut2 + } yield (t, Monoid.plus(st1, st2)) + ) } private case class Mapped[S, T](prev: Execution[S], fn: S => T) extends Execution[T] { def runStats(conf: Config, mode: Mode, cache: EvalCache)(implicit cec: ConcurrentExecutionContext) = - cache.getOrElseInsert(this, { - val (cache1, fut) = prev.runStats(conf, mode, cache) - (cache1, fut.map { case (s, stats, c) => (fn(s), stats, c) }) - }) + cache.getOrElseInsert(this, + prev.runStats(conf, mode, cache) + .map { case (s, stats) => (fn(s), stats) } + ) // Don't bother applying the function if we are mapped override def unit = prev.unit @@ -286,38 +260,32 @@ object Execution { private case class MapCounters[T, U](prev: Execution[T], fn: ((T, ExecutionCounters)) => (U, ExecutionCounters)) extends Execution[U] { def runStats(conf: Config, mode: Mode, cache: EvalCache)(implicit cec: ConcurrentExecutionContext) = - cache.getOrElseInsert(this, { - val (cache1, fut) = prev.runStats(conf, mode, cache) - (cache1, fut.map { - case (t, counters, c) => - val (u, counters2) = fn((t, counters)) - (u, counters2, c) - }) - }) + cache.getOrElseInsert(this, + prev.runStats(conf, mode, cache).map(fn) + ) } private case class OnComplete[T](prev: Execution[T], fn: Try[T] => Unit) extends Execution[T] { def runStats(conf: Config, mode: Mode, cache: EvalCache)(implicit cec: ConcurrentExecutionContext) = cache.getOrElseInsert(this, { val res = prev.runStats(conf, mode, cache) - res._2.map(_._1).onComplete(fn) + res.map(_._1).onComplete(fn) res }) } private case class RecoverWith[T](prev: Execution[T], fn: PartialFunction[Throwable, Execution[T]]) extends Execution[T] { def runStats(conf: Config, mode: Mode, cache: EvalCache)(implicit cec: ConcurrentExecutionContext) = - cache.getOrElseInsert(this, { - val (cache1, fut) = prev.runStats(conf, mode, cache) - // Note, if the future fails, we restart from the input cache - (cache1, fut.recoverWith(fn.andThen(_.runStats(conf, mode, cache)._2))) - }) + cache.getOrElseInsert(this, + prev.runStats(conf, mode, cache) + .recoverWith(fn.andThen(_.runStats(conf, mode, cache))) + ) } private case class Zipped[S, T](one: Execution[S], two: Execution[T]) extends Execution[(S, T)] { def runStats(conf: Config, mode: Mode, cache: EvalCache)(implicit cec: ConcurrentExecutionContext) = cache.getOrElseInsert(this, { - val (cache1, f1) = one.runStats(conf, mode, cache) - val (cache2, f2) = two.runStats(conf, mode, cache1) - (cache2, f1.zip(f2) - .map { case ((s, ss, c1a), (t, st, c2a)) => ((s, t), Monoid.plus(ss, st), c1a ++ c2a) }) + val f1 = one.runStats(conf, mode, cache) + val f2 = two.runStats(conf, mode, cache) + f1.zip(f2) + .map { case ((s, ss), (t, st)) => ((s, t), Monoid.plus(ss, st)) } }) // Make sure we remove any mapping functions on both sides @@ -336,13 +304,13 @@ object Execution { private case class FlowDefExecution[T](result: (Config, Mode) => (FlowDef, (JobStats => Future[T]))) extends Execution[T] { def runStats(conf: Config, mode: Mode, cache: EvalCache)(implicit cec: ConcurrentExecutionContext) = cache.getOrElseInsert(this, - (cache, for { + for { (flowDef, fn) <- toFuture(Try(result(conf, mode))) _ = FlowStateMap.validateSources(flowDef, mode) jobStats <- ExecutionContext.newContext(conf)(flowDef, mode).run _ = FlowStateMap.clear(flowDef) t <- fn(jobStats) - } yield (t, ExecutionCounters.fromJobStats(jobStats), cache))) + } yield (t, ExecutionCounters.fromJobStats(jobStats))) /* * Cascading can run parallel Executions in the same flow if they are both FlowDefExecutions @@ -368,7 +336,11 @@ object Execution { } private case class FactoryExecution[T](result: (Config, Mode) => Execution[T]) extends Execution[T] { def runStats(conf: Config, mode: Mode, cache: EvalCache)(implicit cec: ConcurrentExecutionContext) = - cache.getOrElseInsert(this, unwrap(conf, mode, this).runStats(conf, mode, cache)) + /* + * We don't use unwrap here to make sure each step in a nested FactoryExecution + * is added to the cache + */ + cache.getOrElseInsert(this, result(conf, mode).runStats(conf, mode, cache)) @annotation.tailrec private def unwrap[U](conf: Config, mode: Mode, that: Execution[U]): Execution[U] = @@ -377,19 +349,21 @@ object Execution { case nonFactory => nonFactory } /* - * Cascading can run parallel Executions in the same flow if they are both FlowDefExecutions + * The goal here is try to call zip on FlowDefExecution instances, and let + * scalding compose the operations into a single flow, so we zip on the unwrapped + * operations */ override def zip[U](that: Execution[U]): Execution[(T, U)] = that match { - case FactoryExecution(result2) => + case thatFE@FactoryExecution(result2) => FactoryExecution({ (conf, m) => - val exec1 = unwrap(conf, m, result(conf, m)) - val exec2 = unwrap(conf, m, result2(conf, m)) + val exec1 = unwrap(conf, m, this) + val exec2 = unwrap(conf, m, thatFE) exec1.zip(exec2) }) case _ => FactoryExecution({ (conf, m) => - val exec1 = unwrap(conf, m, result(conf, m)) + val exec1 = unwrap(conf, m, this) exec1.zip(that) }) } @@ -412,7 +386,8 @@ object Execution { * Note this is a lazy parameter that is evaluated every * time run is called. */ - def from[T](t: => T): Execution[T] = fromFuture { _ => toFuture(Try(t)) } + def from[T](t: => T): Execution[T] = fromTry(Try(t)) + def fromTry[T](t: Try[T]): Execution[T] = fromFuture { _ => toFuture(t) } /** * The call to fn will happen when the run method on the result is called. From 978b03be0571a2e674a215d0b39abb41b70dea17 Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Thu, 2 Apr 2015 12:19:07 -1000 Subject: [PATCH 079/177] More simplifications/fixes --- .../com/twitter/scalding/Execution.scala | 272 ++++++++++-------- .../twitter/scalding/typed/TypedPipe.scala | 48 ++-- 2 files changed, 177 insertions(+), 143 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala b/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala index 03f71e06c7..e09cf8241e 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala @@ -19,9 +19,10 @@ import com.twitter.algebird.monad.Reader import com.twitter.algebird.{ Monoid, Monad } import com.twitter.scalding.cascading_interop.FlowListenerPromise import com.twitter.scalding.Dsl.flowDefToRichFlowDef - -import scala.concurrent.{ Await, Future, ExecutionContext => ConcurrentExecutionContext } +import java.util.concurrent.{ ConcurrentHashMap, LinkedBlockingQueue } +import scala.concurrent.{ Await, Future, ExecutionContext => ConcurrentExecutionContext, Promise } import scala.util.{ Failure, Success, Try } +import scala.util.control.NonFatal import cascading.flow.{ FlowDef, Flow } /** @@ -44,7 +45,7 @@ import cascading.flow.{ FlowDef, Flow } * zip to flatMap if you want to run two Executions in parallel. */ sealed trait Execution[+T] extends java.io.Serializable { - import Execution.{ EvalCache, FactoryExecution, FlatMapped, MapCounters, Mapped, OnComplete, RecoverWith, Zipped } + import Execution.{ EvalCache, FlatMapped, MapCounters, Mapped, OnComplete, RecoverWith, Zipped } /** * Scala uses the filter method in for syntax for pattern matches that can fail. @@ -94,11 +95,14 @@ sealed trait Execution[+T] extends java.io.Serializable { /** * This function is called when the current run is completed. This is - * a only a side effect (see unit return). - * Note, this is the only way to force a side effect. Map and FlatMap - * are not safe for side effects. ALSO You must run the result. If + * only a side effect (see unit return). + * + * ALSO You must .run the result. If * you throw away the result of this call, your fn will never be - * called. + * called. When you run the result, the Future you get will not + * be complete unless fn has completed running. If fn throws, it + * will be handled be the scala.concurrent.ExecutionContext.reportFailure + * NOT by returning a Failure in the Future. */ def onComplete(fn: Try[T] => Unit): Execution[T] = OnComplete(this, fn) @@ -127,8 +131,15 @@ sealed trait Execution[+T] extends java.io.Serializable { * * Seriously: pro-style is for this to be called only once in a program. */ - def run(conf: Config, mode: Mode)(implicit cec: ConcurrentExecutionContext): Future[T] = - runStats(conf, mode, new EvalCache)(cec).map(_._1) + final def run(conf: Config, mode: Mode)(implicit cec: ConcurrentExecutionContext): Future[T] = { + val ec = new EvalCache + val result = runStats(conf, mode, ec)(cec).map(_._1) + // When the final future in complete we stop the submit thread + result.onComplete { _ => ec.finished() } + // wait till the end to start the thread in case the above throws + ec.start() + result + } /** * This is the internal method that must be implemented @@ -144,10 +155,6 @@ sealed trait Execution[+T] extends java.io.Serializable { /** * This is convenience for when we don't care about the result. * like .map(_ => ()) - * Note: When called after a map, the map never happens. Use onComplete - * to attach side effects. - * - * .map(fn).unit == .unit */ def unit: Execution[Unit] = map(_ => ()) @@ -157,10 +164,9 @@ sealed trait Execution[+T] extends java.io.Serializable { * composition. Every time someone calls this, be very suspect. It is * always code smell. Very seldom should you need to wait on a future. */ - def waitFor(conf: Config, mode: Mode): Try[T] = { + def waitFor(conf: Config, mode: Mode): Try[T] = Try(Await.result(run(conf, mode)(ConcurrentExecutionContext.global), scala.concurrent.duration.Duration.Inf)) - } /** * This is here to silence warnings in for comprehensions, but is @@ -173,11 +179,8 @@ sealed trait Execution[+T] extends java.io.Serializable { * run this and that in parallel, without any dependency. This will * be done in a single cascading flow if possible. */ - def zip[U](that: Execution[U]): Execution[(T, U)] = that match { - // push zips as low as possible - case fact @ FactoryExecution(_) => fact.zip(this).map(_.swap) - case _ => Zipped(this, that) - } + def zip[U](that: Execution[U]): Execution[(T, U)] = + Zipped(this, that) } /** @@ -201,24 +204,78 @@ object Execution { * as it is evaluating. */ private[scalding] class EvalCache { - private[this] val lock = new AnyRef private[this] val cache = - scala.collection.mutable.Map[Execution[Any], Future[(Any, ExecutionCounters)]]() + new ConcurrentHashMap[Execution[Any], Future[(Any, ExecutionCounters)]]() + + /** + * We send messages from other threads into the submit thread here + */ + sealed trait FlowDefAction + case class RunFlowDef(conf: Config, + mode: Mode, + fd: FlowDef, + result: Promise[JobStats]) extends FlowDefAction + case object Stop extends FlowDefAction + private val messageQueue = new LinkedBlockingQueue[FlowDefAction]() + /** + * Hadoop and/or cascading has some issues, it seems, with starting jobs + * from multiple threads. This thread does all the Flow starting. + */ + private val thread = new Thread(new Runnable { + def run() { + @annotation.tailrec + def go(): Unit = messageQueue.take match { + case Stop => + () + case RunFlowDef(conf, mode, fd, promise) => + promise.completeWith( + try { + ExecutionContext.newContext(conf)(fd, mode).run + } catch { + // Try our best to complete the future + case e: Throwable => Future.failed(e) + }) + // Loop + go() + } + + // Now we actually run the recursive loop + go() + } + }) + + def runFlowDef(conf: Config, mode: Mode, fd: FlowDef): Future[JobStats] = { + try { + val promise = Promise[JobStats]() + messageQueue.put(RunFlowDef(conf, mode, fd, promise)) + promise.future + } + catch { + case NonFatal(e) => + Future.failed(e) + } + } + + def start(): Unit = thread.start() + /* + * This is called after we are done submitting all jobs + */ + def finished(): Unit = messageQueue.put(Stop) def getOrElseInsert[T](ex: Execution[T], - res: => Future[(T, ExecutionCounters)])(implicit ec: ConcurrentExecutionContext): Future[(T, ExecutionCounters)] = lock.synchronized { - cache.get(ex) match { - case Some(fut) => fut.asInstanceOf[Future[(T, ExecutionCounters)]] - case None => - /* - * The evaluation of res could wind up looking into the map for this same - * Execution. So, we first put in a Promise, and then we complete with res - */ - val promise = Promise[(T, ExecutionCounters)]() - cache += (ex -> promise.future) - val fut = res - promise.completeWith(fut) + res: => Future[(T, ExecutionCounters)])(implicit ec: ConcurrentExecutionContext): Future[(T, ExecutionCounters)] = { + /* + * Since we don't want to evaluate res twice, we make a promise + * which we will use if it has not already been evaluated + */ + val promise = Promise[(T, ExecutionCounters)]() + val fut = promise.future + cache.putIfAbsent(ex, fut) match { + case null => + // note res is by-name, so we just evaluate it now: + promise.completeWith(res) fut + case exists => exists.asInstanceOf[Future[(T, ExecutionCounters)]] } } } @@ -229,8 +286,7 @@ object Execution { for { futt <- toFuture(Try(get(cec))) t <- futt - } yield (t, ExecutionCounters.empty) - ) + } yield (t, ExecutionCounters.empty)) // Note that unit is not optimized away, since Futures are often used with side-effects, so, // we ensure that get is always called in contrast to Mapped, which assumes that fn is pure. @@ -243,41 +299,48 @@ object Execution { next = fn(s) fut2 = next.runStats(conf, mode, cache) (t, st2) <- fut2 - } yield (t, Monoid.plus(st1, st2)) - ) + } yield (t, Monoid.plus(st1, st2))) } private case class Mapped[S, T](prev: Execution[S], fn: S => T) extends Execution[T] { def runStats(conf: Config, mode: Mode, cache: EvalCache)(implicit cec: ConcurrentExecutionContext) = cache.getOrElseInsert(this, prev.runStats(conf, mode, cache) - .map { case (s, stats) => (fn(s), stats) } - ) - - // Don't bother applying the function if we are mapped - override def unit = prev.unit + .map { case (s, stats) => (fn(s), stats) }) } private case class MapCounters[T, U](prev: Execution[T], fn: ((T, ExecutionCounters)) => (U, ExecutionCounters)) extends Execution[U] { def runStats(conf: Config, mode: Mode, cache: EvalCache)(implicit cec: ConcurrentExecutionContext) = cache.getOrElseInsert(this, - prev.runStats(conf, mode, cache).map(fn) - ) + prev.runStats(conf, mode, cache).map(fn)) } + private case class OnComplete[T](prev: Execution[T], fn: Try[T] => Unit) extends Execution[T] { def runStats(conf: Config, mode: Mode, cache: EvalCache)(implicit cec: ConcurrentExecutionContext) = cache.getOrElseInsert(this, { val res = prev.runStats(conf, mode, cache) - res.map(_._1).onComplete(fn) - res + /** + * The result we give is only completed AFTER fn is run + * so callers can wait on the result of this OnComplete + */ + val finished = Promise[(T, ExecutionCounters)]() + res.onComplete { tryT => + try { + fn(tryT.map(_._1)) + } + finally { + // Do our best to signal when we are done + finished.complete(tryT) + } + } + finished.future }) } private case class RecoverWith[T](prev: Execution[T], fn: PartialFunction[Throwable, Execution[T]]) extends Execution[T] { def runStats(conf: Config, mode: Mode, cache: EvalCache)(implicit cec: ConcurrentExecutionContext) = cache.getOrElseInsert(this, prev.runStats(conf, mode, cache) - .recoverWith(fn.andThen(_.runStats(conf, mode, cache))) - ) + .recoverWith(fn.andThen(_.runStats(conf, mode, cache)))) } private case class Zipped[S, T](one: Execution[S], two: Execution[T]) extends Execution[(S, T)] { def runStats(conf: Config, mode: Mode, cache: EvalCache)(implicit cec: ConcurrentExecutionContext) = @@ -287,9 +350,6 @@ object Execution { f1.zip(f2) .map { case ((s, ss), (t, st)) => ((s, t), Monoid.plus(ss, st)) } }) - - // Make sure we remove any mapping functions on both sides - override def unit = one.unit.zip(two.unit).map(_ => ()) } private case class UniqueIdExecution[T](fn: UniqueID => Execution[T]) extends Execution[T] { def runStats(conf: Config, mode: Mode, cache: EvalCache)(implicit cec: ConcurrentExecutionContext) = @@ -299,74 +359,52 @@ object Execution { }) } /* - * This is the main class the represents a flow without any combinators + * This allows you to run any cascading flowDef as an Execution. */ - private case class FlowDefExecution[T](result: (Config, Mode) => (FlowDef, (JobStats => Future[T]))) extends Execution[T] { + private case class FlowDefExecution(result: (Config, Mode) => FlowDef) extends Execution[Unit] { def runStats(conf: Config, mode: Mode, cache: EvalCache)(implicit cec: ConcurrentExecutionContext) = cache.getOrElseInsert(this, for { - (flowDef, fn) <- toFuture(Try(result(conf, mode))) + flowDef <- toFuture(Try(result(conf, mode))) _ = FlowStateMap.validateSources(flowDef, mode) - jobStats <- ExecutionContext.newContext(conf)(flowDef, mode).run + jobStats <- cache.runFlowDef(conf, mode, flowDef) _ = FlowStateMap.clear(flowDef) - t <- fn(jobStats) - } yield (t, ExecutionCounters.fromJobStats(jobStats))) + } yield ((), ExecutionCounters.fromJobStats(jobStats))) + } - /* - * Cascading can run parallel Executions in the same flow if they are both FlowDefExecutions - */ - override def zip[U](that: Execution[U]): Execution[(T, U)] = - that match { - /* - * This merging parallelism only works if the names of the - * sources are distinct. Scalding allocates uuids to each - * pipe that starts a head, so a collision should be HIGHLY - * unlikely. - */ - case FlowDefExecution(result2) => - FlowDefExecution({ (conf, m) => - val (fd1, fn1) = result(conf, m) - val (fd2, fn2) = result2(conf, m) - val merged = fd1.copy - merged.mergeFrom(fd2) - (merged, { (js: JobStats) => fn1(js).zip(fn2(js)) }) - }) - case _ => super.zip(that) - } + /* + * This is here so we can call without knowing the type T + * but with proof that pipe matches sink + */ + private case class ToWrite[T](pipe: TypedPipe[T], sink: TypedSink[T]) { + def write(flowDef: FlowDef, mode: Mode): Unit = { + // This has the side effect of mutating flowDef + pipe.write(sink)(flowDef, mode) + () + } } - private case class FactoryExecution[T](result: (Config, Mode) => Execution[T]) extends Execution[T] { + /** + * This is the fundamental execution that actually happens in TypedPipes, all the rest + * are based on on this one. By keeping the Pipe and the Sink, can inspect the Execution + * DAG and optimize it later (a goal, but not done yet). + */ + private case class WriteExecution(head: ToWrite[_], tail: List[ToWrite[_]]) extends Execution[Unit] { def runStats(conf: Config, mode: Mode, cache: EvalCache)(implicit cec: ConcurrentExecutionContext) = - /* - * We don't use unwrap here to make sure each step in a nested FactoryExecution - * is added to the cache - */ - cache.getOrElseInsert(this, result(conf, mode).runStats(conf, mode, cache)) + cache.getOrElseInsert(this, + for { + flowDef <- toFuture(Try { val fd = new FlowDef; (head :: tail).foreach(_.write(fd, mode)); fd }) + _ = FlowStateMap.validateSources(flowDef, mode) + jobStats <- cache.runFlowDef(conf, mode, flowDef) + _ = FlowStateMap.clear(flowDef) + } yield ((), ExecutionCounters.fromJobStats(jobStats))) + } - @annotation.tailrec - private def unwrap[U](conf: Config, mode: Mode, that: Execution[U]): Execution[U] = - that match { - case FactoryExecution(fn) => unwrap(conf, mode, fn(conf, mode)) - case nonFactory => nonFactory - } - /* - * The goal here is try to call zip on FlowDefExecution instances, and let - * scalding compose the operations into a single flow, so we zip on the unwrapped - * operations - */ - override def zip[U](that: Execution[U]): Execution[(T, U)] = - that match { - case thatFE@FactoryExecution(result2) => - FactoryExecution({ (conf, m) => - val exec1 = unwrap(conf, m, this) - val exec2 = unwrap(conf, m, thatFE) - exec1.zip(exec2) - }) - case _ => - FactoryExecution({ (conf, m) => - val exec1 = unwrap(conf, m, this) - exec1.zip(that) - }) - } + /** + * This is called Reader, because it just returns its input to run as the output + */ + private case object ReaderExecution extends Execution[(Config, Mode)] { + def runStats(conf: Config, mode: Mode, cache: EvalCache)(implicit cec: ConcurrentExecutionContext) = + Future.successful(((conf, mode), ExecutionCounters.empty)) } private def toFuture[R](t: Try[R]): Future[R] = @@ -402,17 +440,21 @@ object Execution { val unit: Execution[Unit] = from(()) private[scalding] def factory[T](fn: (Config, Mode) => Execution[T]): Execution[T] = - FactoryExecution(fn) + ReaderExecution.flatMap { case (c, m) => fn(c, m) } /** * This converts a function into an Execution monad. The flowDef returned - * is never mutated. The returned callback funcion is called after the flow - * is run and succeeds. + * is never mutated. */ - def fromFn[T]( - fn: (Config, Mode) => ((FlowDef, JobStats => Future[T]))): Execution[T] = + def fromFn(fn: (Config, Mode) => FlowDef): Execution[Unit] = FlowDefExecution(fn) + /** + * Creates an Execution to do a write + */ + private[scalding] def write[T](pipe: TypedPipe[T], sink: TypedSink[T]): Execution[Unit] = + WriteExecution(ToWrite(pipe, sink), Nil) + /** * Use this to read the configuration, which may contain Args or options * which describe input on which to run diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipe.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipe.scala index 0b2f918dc0..79c2afc0b6 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipe.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipe.scala @@ -457,29 +457,26 @@ trait TypedPipe[+T] extends Serializable { * This writes the current TypedPipe into a temporary file * and then opens it after complete so that you can continue from that point */ - def forceToDiskExecution: Execution[TypedPipe[T]] = Execution.fromFn { (conf, mode) => - val flowDef = new FlowDef - mode match { - case _: CascadingLocal => // Local or Test mode - val dest = new MemorySink[T] - write(dest)(flowDef, mode) - - // We can't read until the job finishes - (flowDef, { (js: JobStats) => Future.successful(TypedPipe.from(dest.readResults)) }) - case _: HadoopMode => - // come up with unique temporary filename, use the config here - // TODO: refactor into TemporarySequenceFile class - val tmpDir = conf.get("hadoop.tmp.dir") - .orElse(conf.get("cascading.tmp.dir")) - .getOrElse("/tmp") - - val tmpSeq = tmpDir + "/scalding/snapshot-" + java.util.UUID.randomUUID + ".seq" - val dest = source.TypedSequenceFile[T](tmpSeq) - write(dest)(flowDef, mode) - - (flowDef, { (js: JobStats) => Future.successful(TypedPipe.from(dest)) }) + def forceToDiskExecution: Execution[TypedPipe[T]] = Execution + .getConfigMode + .flatMap { + case (conf, mode) => + mode match { + case _: CascadingLocal => // Local or Test mode + val dest = new MemorySink[T] + writeExecution(dest).map { _ => TypedPipe.from(dest.readResults) } + case _: HadoopMode => + // come up with unique temporary filename, use the config here + // TODO: refactor into TemporarySequenceFile class + val tmpDir = conf.get("hadoop.tmp.dir") + .orElse(conf.get("cascading.tmp.dir")) + .getOrElse("/tmp") + + val tmpSeq = tmpDir + "/scalding/snapshot-" + java.util.UUID.randomUUID + ".seq" + val dest = source.TypedSequenceFile[T](tmpSeq) + writeThrough(dest) + } } - } /** * This gives an Execution that when run evaluates the TypedPipe, @@ -524,12 +521,7 @@ trait TypedPipe[+T] extends Serializable { * into an Execution that is run for anything to happen here. */ def writeExecution(dest: TypedSink[T]): Execution[Unit] = - Execution.fromFn { (conf: Config, m: Mode) => - val fd = new FlowDef - write(dest)(fd, m) - - (fd, { (js: JobStats) => Future.successful(()) }) - } + Execution.write(this, dest) /** * If you want to write to a specific location, and then read from From a1b59b5a04452dd01dea64f4456a64e4f2f24fa7 Mon Sep 17 00:00:00 2001 From: Gera Shegalov Date: Thu, 2 Apr 2015 23:27:15 -0700 Subject: [PATCH 080/177] Provide useHdfsLocalMode for an easy swith to mapreduce local mode --- .../com/twitter/scalding/ReplImplicits.scala | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/scalding-repl/src/main/scala/com/twitter/scalding/ReplImplicits.scala b/scalding-repl/src/main/scala/com/twitter/scalding/ReplImplicits.scala index db851fe4df..32e4f0ad3d 100644 --- a/scalding-repl/src/main/scala/com/twitter/scalding/ReplImplicits.scala +++ b/scalding-repl/src/main/scala/com/twitter/scalding/ReplImplicits.scala @@ -26,6 +26,9 @@ import scala.concurrent.{ Future, ExecutionContext => ConcurrentExecutionContext * Most of these conversions come from the [[com.twitter.scalding.Job]] class. */ object ReplImplicits extends FieldConversions { + val mr1Key = "mapred.job.tracker" + val mr2Key = "mapreduce.framework.name" + val mrLocal = "local" /** Implicit flowDef for this Scalding shell session. */ var flowDef: FlowDef = getEmptyFlowDef @@ -42,13 +45,25 @@ object ReplImplicits extends FieldConversions { def useStrictLocalMode() { mode = Local(true) } /** Switch to Hdfs mode */ - def useHdfsMode() { + private def useHdfsMode_() { storedHdfsMode match { case Some(hdfsMode) => mode = hdfsMode case None => println("To use HDFS/Hadoop mode, you must *start* the repl in hadoop mode to get the hadoop configuration from the hadoop command.") } } + def useHdfsMode() { + useHdfsMode_() + customConfig -= mr1Key + customConfig -= mr2Key + } + + def useHdfsLocalMode() { + useHdfsMode_() + customConfig += mr1Key -> mrLocal + customConfig += mr2Key -> mrLocal + } + /** * Configuration to use for REPL executions. * From 0718f11db07fafb8e557b79917123dcc9917b0b8 Mon Sep 17 00:00:00 2001 From: Tom Dyas Date: Sat, 28 Mar 2015 18:32:37 -0400 Subject: [PATCH 081/177] Note the active FlowProcess for Joiners Certain Scalding features (e.g., counters) require a FlowProcess. If a method is invoked (indirectly) by a Joiner, however, a FlowProcess currently will not be available. Modify all calls into Cascading that deal with Joiners to wrap them in a proxy which records the active FlowProcess. --- .../com/twitter/scalding/CoGroupBuilder.scala | 2 +- .../com/twitter/scalding/JoinAlgorithms.scala | 40 +++++++- .../twitter/scalding/typed/CoGrouped.scala | 8 +- .../twitter/scalding/typed/HashJoinable.scala | 2 +- .../twitter/scalding/WrappedJoinerTest.scala | 71 ++++++++++++++ .../scalding/JoinerFlowProcessTest.scala | 93 +++++++++++++++++++ 6 files changed, 207 insertions(+), 9 deletions(-) create mode 100644 scalding-core/src/test/scala/com/twitter/scalding/WrappedJoinerTest.scala create mode 100644 scalding-hadoop-test/src/test/scala/com/twitter/scalding/JoinerFlowProcessTest.scala diff --git a/scalding-core/src/main/scala/com/twitter/scalding/CoGroupBuilder.scala b/scalding-core/src/main/scala/com/twitter/scalding/CoGroupBuilder.scala index 0610596d0d..aa8ee5d786 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/CoGroupBuilder.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/CoGroupBuilder.scala @@ -44,7 +44,7 @@ class CoGroupBuilder(groupFields: Fields, joinMode: JoinMode) extends GroupBuild 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) + val cg: Pipe = new CoGroup(pipes, fields, null, WrappedJoiner(mixedJoiner)) overrideReducers(cg) evs.foldRight(cg)((op: Pipe => Every, p) => op(p)) } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/JoinAlgorithms.scala b/scalding-core/src/main/scala/com/twitter/scalding/JoinAlgorithms.scala index c43e135d63..f234a93d23 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/JoinAlgorithms.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/JoinAlgorithms.scala @@ -19,6 +19,7 @@ import cascading.pipe._ import cascading.pipe.joiner._ import cascading.tuple._ +import java.util.{ Iterator => JIterator } import java.util.Random // this one is serializable, scala.util.Random is not import scala.collection.JavaConverters._ @@ -112,7 +113,7 @@ trait JoinAlgorithms { /** * Flip between LeftJoin to RightJoin */ - private def flipJoiner(j: Joiner) = { + private def flipJoiner(j: Joiner): Joiner = { j match { case outer: OuterJoin => outer case inner: InnerJoin => inner @@ -216,17 +217,17 @@ trait JoinAlgorithms { def joinWithTiny(fs: (Fields, Fields), that: Pipe) = { val intersection = asSet(fs._1).intersect(asSet(fs._2)) if (intersection.isEmpty) { - new HashJoin(assignName(pipe), fs._1, assignName(that), fs._2, new InnerJoin) + new HashJoin(assignName(pipe), fs._1, assignName(that), fs._2, WrappedJoiner(new InnerJoin)) } else { val (renamedThat, newJoinFields, temp) = renameCollidingFields(that, fs._2, intersection) - (new HashJoin(assignName(pipe), fs._1, assignName(renamedThat), newJoinFields, new InnerJoin)) + (new HashJoin(assignName(pipe), fs._1, assignName(renamedThat), newJoinFields, WrappedJoiner(new InnerJoin))) .discard(temp) } } def leftJoinWithTiny(fs: (Fields, Fields), that: Pipe) = { //Rename these pipes to avoid cascading name conflicts - new HashJoin(assignName(pipe), fs._1, assignName(that), fs._2, new LeftJoin) + new HashJoin(assignName(pipe), fs._1, assignName(that), fs._2, WrappedJoiner(new LeftJoin)) } /** @@ -466,3 +467,34 @@ trait JoinAlgorithms { } class InvalidJoinModeException(args: String) extends Exception(args) + +/** + * Wraps a Joiner instance so that the active FlowProcess may be noted. This allows features of Scalding that need + * access to a FlowProcess (e.g., counters) to function properly inside a Joiner. + */ +private[scalding] class WrappedJoiner(val joiner: Joiner) extends Joiner { + override def getIterator(joinerClosure: JoinerClosure): JIterator[Tuple] = { + RuntimeStats.addFlowProcess(joinerClosure.getFlowProcess) + joiner.getIterator(joinerClosure) + } + + override def numJoins(): Int = joiner.numJoins() + + override def hashCode(): Int = joiner.hashCode() + + override def toString: String = joiner.toString + + override def equals(other: Any): Boolean = joiner.equals(other) +} + +private[scalding] object WrappedJoiner { + /** + * Wrap the given Joiner in a WrappedJoiner instance if it is not already wrapped. + */ + def apply(joiner: Joiner): WrappedJoiner = { + joiner match { + case wrapped: WrappedJoiner => wrapped + case _ => new WrappedJoiner(joiner) + } + } +} diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/CoGrouped.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/CoGrouped.scala index 33a5066fe0..9625ce180b 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/CoGrouped.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/CoGrouped.scala @@ -219,7 +219,7 @@ trait CoGrouped[K, +R] extends KeyedListLike[K, R, CoGrouped] with CoGroupable[K ordKeyField, NUM_OF_SELF_JOINS, outFields(firstCount), - new DistinctCoGroupJoiner(firstCount, Grouped.keyGetter(ord), joinFunction)) + WrappedJoiner(new DistinctCoGroupJoiner(firstCount, Grouped.keyGetter(ord), joinFunction))) } else if (firstCount == 1) { def keyId(idx: Int): String = "key%d".format(idx) @@ -265,9 +265,11 @@ trait CoGrouped[K, +R] extends KeyedListLike[K, R, CoGrouped] with CoGroupable[K val distinctSize = dsize def distinctIndexOf(orig: Int) = mapping(orig) } - } else new DistinctCoGroupJoiner(isize, Grouped.keyGetter(ord), joinFunction) + } else { + new DistinctCoGroupJoiner(isize, Grouped.keyGetter(ord), joinFunction) + } - new CoGroup(pipes, groupFields, outFields(dsize), cjoiner) + new CoGroup(pipes, groupFields, outFields(dsize), WrappedJoiner(cjoiner)) } else { /** * This is non-trivial to encode in the type system, so we throw this exception diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/HashJoinable.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/HashJoinable.scala index 903cf5ea54..37764cb3ba 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/HashJoinable.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/HashJoinable.scala @@ -52,7 +52,7 @@ trait HashJoinable[K, +V] extends CoGroupable[K, V] with KeyedPipe[K] { Field.singleOrdered("key")(keyOrdering), mapped.toPipe(('key1, 'value1))(fd, mode, tup2Setter), Field.singleOrdered("key1")(keyOrdering), - new HashJoiner(joinFunction, joiner)) + WrappedJoiner(new HashJoiner(joinFunction, joiner))) //Construct the new TypedPipe TypedPipe.from[(K, R)](newPipe.project('key, 'value), ('key, 'value))(fd, mode, tuple2Converter) diff --git a/scalding-core/src/test/scala/com/twitter/scalding/WrappedJoinerTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/WrappedJoinerTest.scala new file mode 100644 index 0000000000..b507ca6460 --- /dev/null +++ b/scalding-core/src/test/scala/com/twitter/scalding/WrappedJoinerTest.scala @@ -0,0 +1,71 @@ +package com.twitter.scalding + +import cascading.flow.FlowException +import cascading.pipe.CoGroup +import cascading.pipe.joiner.{ JoinerClosure, InnerJoin } +import cascading.tuple.Tuple +import org.scalatest.{ Matchers, WordSpec } + +import java.util.{ Iterator => JIterator } + +class CheckFlowProcessJoiner(uniqueID: UniqueID) extends InnerJoin { + override def getIterator(joinerClosure: JoinerClosure): JIterator[Tuple] = { + val flowProcess = RuntimeStats.getFlowProcessForUniqueId(uniqueID) + if (flowProcess == null) { + throw new NullPointerException("No active FlowProcess was available.") + } + + super.getIterator(joinerClosure) + } +} + +class TestWrappedJoinerJob(args: Args) extends Job(args) { + val uniqueID = UniqueID.getIDFor(flowDef) + + val inA = Tsv("inputA", ('a, 'b)) + val inB = Tsv("inputB", ('x, 'y)) + + val joiner = { + val checkJoiner = new CheckFlowProcessJoiner(uniqueID) + if (args.boolean("wrapJoiner")) WrappedJoiner(checkJoiner) else checkJoiner + } + + val p1 = new CoGroup(inA, 'a, inB, 'x, joiner) + + // The .forceToDisk is necessary to have the test work properly. + p1.forceToDisk.write(Tsv("output")) +} + +class WrappedJoinerTest extends WordSpec with Matchers { + "Methods called from a Joiner" should { + "have access to a FlowProcess when WrappedJoiner is used" in { + JobTest(new TestWrappedJoinerJob(_)) + .arg("wrapJoiner", "true") + .source(Tsv("inputA"), Seq(("1", "alpha"), ("2", "beta"))) + .source(Tsv("inputB"), Seq(("1", "first"), ("2", "second"))) + .sink[(Int, String)](Tsv("output")) { outBuf => + // The job will fail with an exception if the FlowProcess is unavailable. + } + .runHadoop + .finish + } + + "have no access to a FlowProcess when WrappedJoiner is not used" in { + try { + JobTest(new TestWrappedJoinerJob(_)) + .source(Tsv("inputA"), Seq(("1", "alpha"), ("2", "beta"))) + .source(Tsv("inputB"), Seq(("1", "first"), ("2", "second"))) + .sink[(Int, String)](Tsv("output")) { outBuf => + // The job will fail with an exception if the FlowProcess is unavailable. + } + .runHadoop + .finish + + fail("The test Job without WrappedJoiner should fail.") + } catch { + case ex: FlowException => + ex.getCause.getMessage should include ("the FlowProcess for unique id") + } + } + } +} diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/JoinerFlowProcessTest.scala b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/JoinerFlowProcessTest.scala new file mode 100644 index 0000000000..cb2ea779e9 --- /dev/null +++ b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/JoinerFlowProcessTest.scala @@ -0,0 +1,93 @@ +package com.twitter.scalding + +import cascading.pipe.joiner.{ JoinerClosure, InnerJoin } +import cascading.tuple.Tuple +import com.twitter.scalding.platform.{ HadoopSharedPlatformTest, HadoopPlatformJobTest, HadoopPlatformTest } +import org.scalatest.{ Matchers, WordSpec } + +import java.util.{ Iterator => JIterator } + +import org.slf4j.{ LoggerFactory, Logger } + +class CheckFlowProcessJoiner(uniqueID: UniqueID) extends InnerJoin { + override def getIterator(joinerClosure: JoinerClosure): JIterator[Tuple] = { + println("CheckFlowProcessJoiner.getItertor") + + val flowProcess = RuntimeStats.getFlowProcessForUniqueId(uniqueID) + if (flowProcess == null) { + throw new NullPointerException("No active FlowProcess was available.") + } + + super.getIterator(joinerClosure) + } +} + +class CheckForFlowProcessInFieldsJob(args: Args) extends Job(args) { + val uniqueID = UniqueID.getIDFor(flowDef) + val stat = Stat("joins") + + val inA = Tsv("inputA", ('a, 'b)) + val inB = Tsv("inputB", ('x, 'y)) + + val p = inA.joinWithSmaller('a -> 'x, inB).map(('b, 'y) -> 'z) { args: (String, String) => + stat.inc + + val flowProcess = RuntimeStats.getFlowProcessForUniqueId(uniqueID) + if (flowProcess == null) { + throw new NullPointerException("No active FlowProcess was available.") + } + + s"${args._1},${args._2}" + } + + p.write(Tsv("output", ('b, 'y))) +} + +class CheckForFlowProcessInTypedJob(args: Args) extends Job(args) { + val uniqueID = UniqueID.getIDFor(flowDef) + val stat = Stat("joins") + + val inA = TypedPipe.from(TypedTsv[(String, String)]("inputA")) + val inB = TypedPipe.from(TypedTsv[(String, String)]("inputB")) + + inA.group.join(inB.group).forceToReducers.mapGroup((key, valuesIter) => { + stat.inc + + val flowProcess = RuntimeStats.getFlowProcessForUniqueId(uniqueID) + if (flowProcess == null) { + throw new NullPointerException("No active FlowProcess was available.") + } + + valuesIter.map({ case (a, b) => s"$a:$b" }) + }).toTypedPipe.write(TypedTsv[(String, String)]("output")) +} + +class JoinerFlowProcessTest extends WordSpec with Matchers with HadoopSharedPlatformTest { + "Methods called from a Joiner" should { + "have access to a FlowProcess from a join in the Fields-based API" in { + HadoopPlatformJobTest(new CheckForFlowProcessInFieldsJob(_), cluster) + .source(TypedTsv[(String, String)]("inputA"), Seq(("1", "alpha"), ("2", "beta"))) + .source(TypedTsv[(String, String)]("inputB"), Seq(("1", "first"), ("2", "second"))) + .sink(TypedTsv[(String, String)]("output")) { _ => + // The job will fail with an exception if the FlowProcess is unavailable. + } + .inspectCompletedFlow({ flow => + flow.getFlowStats.getCounterValue(Stats.ScaldingGroup, "joins") shouldBe 2 + }) + .run + } + + "have access to a FlowProcess from a join in the Typed API" in { + HadoopPlatformJobTest(new CheckForFlowProcessInTypedJob(_), cluster) + .source(TypedTsv[(String, String)]("inputA"), Seq(("1", "alpha"), ("2", "beta"))) + .source(TypedTsv[(String, String)]("inputB"), Seq(("1", "first"), ("2", "second"))) + .sink[(String, String)](TypedTsv[(String, String)]("output")) { _ => + // The job will fail with an exception if the FlowProcess is unavailable. + } + .inspectCompletedFlow({ flow => + flow.getFlowStats.getCounterValue(Stats.ScaldingGroup, "joins") shouldBe 2 + }) + .run + } + } +} From d04448e384a2f65950757df233a9946205e52cba Mon Sep 17 00:00:00 2001 From: Gera Shegalov Date: Fri, 3 Apr 2015 15:28:08 -0700 Subject: [PATCH 082/177] Changed scope for useHdfsLocalMode to private --- .../main/scala/com/twitter/scalding/ReplImplicits.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/scalding-repl/src/main/scala/com/twitter/scalding/ReplImplicits.scala b/scalding-repl/src/main/scala/com/twitter/scalding/ReplImplicits.scala index 32e4f0ad3d..6bdb89f9d5 100644 --- a/scalding-repl/src/main/scala/com/twitter/scalding/ReplImplicits.scala +++ b/scalding-repl/src/main/scala/com/twitter/scalding/ReplImplicits.scala @@ -26,9 +26,11 @@ import scala.concurrent.{ Future, ExecutionContext => ConcurrentExecutionContext * Most of these conversions come from the [[com.twitter.scalding.Job]] class. */ object ReplImplicits extends FieldConversions { - val mr1Key = "mapred.job.tracker" - val mr2Key = "mapreduce.framework.name" - val mrLocal = "local" + + /** required for switching to hdfs local mode */ + private val mr1Key = "mapred.job.tracker" + private val mr2Key = "mapreduce.framework.name" + private val mrLocal = "local" /** Implicit flowDef for this Scalding shell session. */ var flowDef: FlowDef = getEmptyFlowDef From 82a044932417587e452ff001e2bb7c4d0317eed0 Mon Sep 17 00:00:00 2001 From: Mansur Ashraf Date: Mon, 6 Apr 2015 11:41:22 -0700 Subject: [PATCH 083/177] upgrade scalacheck and scalatest --- project/Build.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/project/Build.scala b/project/Build.scala index d3e4a80a88..ffa4197dd9 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -35,8 +35,8 @@ object ScaldingBuild extends Build { val json4SVersion = "3.2.11" val parquetVersion = "1.6.0rc4" val protobufVersion = "2.4.1" - val scalaCheckVersion = "1.12.1" - val scalaTestVersion = "2.2.2" + val scalaCheckVersion = "1.12.2" + val scalaTestVersion = "2.2.4" val scalameterVersion = "0.6" val scroogeVersion = "3.17.0" val slf4jVersion = "1.6.6" From 4a4844c7b221f0c93883ab230484709b2d577e5a Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Tue, 7 Apr 2015 09:41:32 -1000 Subject: [PATCH 084/177] Fix comment --- .../scalding/serialization/StringOrderedSerialization.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/StringOrderedSerialization.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/StringOrderedSerialization.scala index ea0ffd8aad..7ad1791ee3 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/StringOrderedSerialization.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/StringOrderedSerialization.scala @@ -47,7 +47,7 @@ object StringOrderedSerialization { } /** - * Now we start by comparing blocks of longs, then maybe 1 int, then 0 - 3 bytes + * Now we start by comparing blocks of ints, then 0 - 3 bytes */ val toCheck = math.min(leftSize, rightSize) val ints = toCheck / 4 From 7fe7fe4eea8559cc60edcf7fbea8a0b37c958813 Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Fri, 10 Apr 2015 16:24:38 -0700 Subject: [PATCH 085/177] Add StringUtility and test --- .../com/twitter/scalding/StringUtility.scala | 39 ++++++++++++ .../twitter/scalding/StringUtilityTest.scala | 62 +++++++++++++++++++ 2 files changed, 101 insertions(+) create mode 100644 scalding-core/src/main/scala/com/twitter/scalding/StringUtility.scala create mode 100644 scalding-core/src/test/scala/com/twitter/scalding/StringUtilityTest.scala diff --git a/scalding-core/src/main/scala/com/twitter/scalding/StringUtility.scala b/scalding-core/src/main/scala/com/twitter/scalding/StringUtility.scala new file mode 100644 index 0000000000..65d404d801 --- /dev/null +++ b/scalding-core/src/main/scala/com/twitter/scalding/StringUtility.scala @@ -0,0 +1,39 @@ +package com.twitter.scalding + +object StringUtility { + // Scala's split function uses regex which is too slow thus we create this faster function + private def isTrailingWith(text: String, separator: String) = { + val result = text.foldRight((List[Boolean](), true)) { + case (c, (tillNow, isTrailingBefore)) => + val trailing = isTrailingBefore && (c.toString == separator) + (tillNow ++ List(trailing), trailing) + } + result._1.reverse + } + + private def fastSplitHelper(text: String, key: String, isTrailingAt: List[Boolean]): Seq[String] = { + if (text.isEmpty) { + Seq() + } else { + val firstIndex = text.indexOf(key) + if (firstIndex == -1) { + Seq(text) + } else { + // the text till the separator should be kept in any case + val currentChunk = text.substring(0, firstIndex) + // if it's trailing from this index, then ignore the rest of the string + if (isTrailingAt(firstIndex)) { + Seq(currentChunk) + } else { + // Otherwise, we want to see the rest of the string + val nextText = text.substring(firstIndex + 1) + Seq(currentChunk) ++ fastSplitHelper(nextText, key, isTrailingAt.slice(firstIndex + 1, text.length)) + } + } + } + + } + def fastSplit(text: String, key: String): Seq[String] = { + fastSplitHelper(text, key, isTrailingWith(text, key)) + } +} \ No newline at end of file diff --git a/scalding-core/src/test/scala/com/twitter/scalding/StringUtilityTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/StringUtilityTest.scala new file mode 100644 index 0000000000..ee7c440ca7 --- /dev/null +++ b/scalding-core/src/test/scala/com/twitter/scalding/StringUtilityTest.scala @@ -0,0 +1,62 @@ +package com.twitter.scalding + +import org.scalatest.{ Matchers, WordSpec } + +class StringUtilityTest extends WordSpec with Matchers { + "fastSplitTest" should { + "be able to split white space" in { + val text1 = "this is good time" + val res1 = StringUtility.fastSplit(text1, " ") // split single white space + res1 should be { + Seq("this", "is", "good", "time") + } + } + } + "be able to split other separators" in { + val text2 = "a:b:c:d:" + val res2 = StringUtility.fastSplit(text2, ":") + res2 should be { + Seq("a", "b", "c", "d") + } + } + "be able to split only one separators" in { + val text2 = "a@" + val res2 = StringUtility.fastSplit(text2, "@") + res2 should be { + Seq("a") + } + } + "be able to split when separator doesn't show up" in { + val text2 = "a" + val res2 = StringUtility.fastSplit(text2, "@") + res2 should be { + Seq("a") + } + } + "be able to generate the same result with Java's split for random strings and separators" in { + def randomString(length: Int) = { + val possibleChars = "abcdefgABCDEFG #@:!$%^&*()+-_" + val nPossibleChar = possibleChars.length + val r = new scala.util.Random + val sb = new StringBuilder + for (i <- 1 to length) { + sb.append(possibleChars(r.nextInt(nPossibleChar))) + } + sb.toString + } + // randomly test to make sure the fastSplit function works the same (with high probability) with java's impl + // for loop is to test for different separator it works exactly the same with Java split + for (i <- 1 to 100) { + val randomStrings: List[String] = (1 to 1000).map { + x => + randomString(20) + }.toList + val randomSeparatorIndex = scala.util.Random.nextInt(5) + val separator = "#@/: "(randomSeparatorIndex).toString + val splittedByRegex = randomStrings.map { s => s.split(separator).toList } + val splittedByFastSpliter = randomStrings.map { s => StringUtility.fastSplit(s, separator).toList } + splittedByRegex should be(splittedByFastSpliter) + } + } + +} From e2c0c16e7027a61ee99f1f06fa7f3596d5165824 Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Fri, 10 Apr 2015 16:28:21 -0700 Subject: [PATCH 086/177] Change all the split() in scalding --- .../src/main/scala/com/twitter/scalding/Config.scala | 4 ++-- .../src/main/scala/com/twitter/scalding/RichFlowDef.scala | 2 +- scalding-core/src/main/scala/com/twitter/scalding/Stats.scala | 2 +- .../com/twitter/scalding/reducer_estimation/Common.scala | 4 ++-- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala index 1a286d5085..feb1122871 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala @@ -179,7 +179,7 @@ trait Config { def addUniqueId(u: UniqueID): Config = update(UniqueID.UNIQUE_JOB_ID) { case None => (Some(u.get), ()) - case Some(str) => (Some((str.split(",").toSet + u.get).mkString(",")), ()) + case Some(str) => (Some((StringUtility.fastSplit(str, ",").toSet + u.get).mkString(",")), ()) }._2 /** @@ -191,7 +191,7 @@ trait Config { val uid = UniqueID.getRandom (Some(uid.get), uid) case s @ Some(str) => - (s, UniqueID(str.split(",").head)) + (s, UniqueID(StringUtility.fastSplit(str, ",").head)) } /* diff --git a/scalding-core/src/main/scala/com/twitter/scalding/RichFlowDef.scala b/scalding-core/src/main/scala/com/twitter/scalding/RichFlowDef.scala index 8f0b6edfc3..49a3404470 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/RichFlowDef.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/RichFlowDef.scala @@ -43,7 +43,7 @@ class RichFlowDef(val fd: FlowDef) { */ private[scalding] def mergeMiscFrom(o: FlowDef): Unit = { // See the cascading code that this string is a "," separated set. - o.getTags.split(",").foreach(fd.addTag) + StringUtility.fastSplit(o.getTags, ",").foreach(fd.addTag) mergeLeft(fd.getTraps, o.getTraps) mergeLeft(fd.getCheckpoints, o.getCheckpoints) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Stats.scala b/scalding-core/src/main/scala/com/twitter/scalding/Stats.scala index ce55eb5ff1..8dcafa80ac 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Stats.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Stats.scala @@ -125,7 +125,7 @@ object RuntimeStats extends java.io.Serializable { def addFlowProcess(fp: FlowProcess[_]) { val uniqueJobIdObj = fp.getProperty(UniqueID.UNIQUE_JOB_ID) if (uniqueJobIdObj != null) { - uniqueJobIdObj.asInstanceOf[String].split(",").foreach { uniqueId => + StringUtility.fastSplit(uniqueJobIdObj.asInstanceOf[String], ",").foreach { uniqueId => logger.debug("Adding flow process id: " + uniqueId) flowMappingStore.put(uniqueId, new WeakReference(fp)) } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala index 5657224737..60fc54d1c1 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala @@ -2,7 +2,7 @@ package com.twitter.scalding.reducer_estimation import cascading.flow.{ FlowStep, Flow, FlowStepStrategy } import com.twitter.algebird.Monoid -import com.twitter.scalding.Config +import com.twitter.scalding.{ StringUtility, Config } import org.apache.hadoop.mapred.JobConf import java.util.{ List => JList } @@ -89,7 +89,7 @@ object ReducerEstimatorStepStrategy extends FlowStepStrategy[JobConf] { val clsLoader = Thread.currentThread.getContextClassLoader - val estimators = clsNames.split(",") + val estimators = StringUtility.fastSplit(clsNames, ",") .map(clsLoader.loadClass(_).newInstance.asInstanceOf[ReducerEstimator]) val combinedEstimator = Monoid.sum(estimators) From 2e56b06169720f07e3926c3a8b152d65aa189f93 Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Sat, 11 Apr 2015 17:33:29 -0700 Subject: [PATCH 087/177] Make the splitter faster --- .../com/twitter/scalding/StringUtility.scala | 44 +++++++------------ 1 file changed, 15 insertions(+), 29 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/StringUtility.scala b/scalding-core/src/main/scala/com/twitter/scalding/StringUtility.scala index 65d404d801..11a4ece041 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/StringUtility.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/StringUtility.scala @@ -1,39 +1,25 @@ package com.twitter.scalding -object StringUtility { - // Scala's split function uses regex which is too slow thus we create this faster function - private def isTrailingWith(text: String, separator: String) = { - val result = text.foldRight((List[Boolean](), true)) { - case (c, (tillNow, isTrailingBefore)) => - val trailing = isTrailingBefore && (c.toString == separator) - (tillNow ++ List(trailing), trailing) - } - result._1.reverse - } +import java.util - private def fastSplitHelper(text: String, key: String, isTrailingAt: List[Boolean]): Seq[String] = { - if (text.isEmpty) { - Seq() - } else { - val firstIndex = text.indexOf(key) - if (firstIndex == -1) { - Seq(text) +import scala.collection.mutable.ArrayBuffer + +object StringUtility { + private def fastSplitHelper(text: String, key: String, from: Int, textLength: Int): List[String] = { + val firstIndex = text.indexOf(key, from) + if (firstIndex == -1) { + if (from < textLength) { + List(text.substring(from)) } else { - // the text till the separator should be kept in any case - val currentChunk = text.substring(0, firstIndex) - // if it's trailing from this index, then ignore the rest of the string - if (isTrailingAt(firstIndex)) { - Seq(currentChunk) - } else { - // Otherwise, we want to see the rest of the string - val nextText = text.substring(firstIndex + 1) - Seq(currentChunk) ++ fastSplitHelper(nextText, key, isTrailingAt.slice(firstIndex + 1, text.length)) - } + List() } + } else { + // the text till the separator should be kept in any case + text.substring(from, firstIndex) :: fastSplitHelper(text, key, firstIndex + 1, textLength) } - } + def fastSplit(text: String, key: String): Seq[String] = { - fastSplitHelper(text, key, isTrailingWith(text, key)) + fastSplitHelper(text, key, 0, text.length) } } \ No newline at end of file From 9f7a41e312c36548497b28cbe30f6bfd709a16ec Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Sat, 11 Apr 2015 18:13:57 -0700 Subject: [PATCH 088/177] Add timing to test --- .../twitter/scalding/StringUtilityTest.scala | 57 +++++++++++++++---- 1 file changed, 45 insertions(+), 12 deletions(-) diff --git a/scalding-core/src/test/scala/com/twitter/scalding/StringUtilityTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/StringUtilityTest.scala index ee7c440ca7..49acd4feed 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/StringUtilityTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/StringUtilityTest.scala @@ -33,9 +33,18 @@ class StringUtilityTest extends WordSpec with Matchers { Seq("a") } } - "be able to generate the same result with Java's split for random strings and separators" in { + "be able to be faster than java's split function" in { + // helper function to time + def time[R](block: => R): Double = { + val t0 = System.nanoTime() + val result = block // call-by-name + val t1 = System.nanoTime() + val timeDiff = (t1 - t0) + timeDiff + } + def randomString(length: Int) = { - val possibleChars = "abcdefgABCDEFG #@:!$%^&*()+-_" + val possibleChars = "abcdefg|" val nPossibleChar = possibleChars.length val r = new scala.util.Random val sb = new StringBuilder @@ -44,19 +53,43 @@ class StringUtilityTest extends WordSpec with Matchers { } sb.toString } - // randomly test to make sure the fastSplit function works the same (with high probability) with java's impl - // for loop is to test for different separator it works exactly the same with Java split + + // randomly test + // for loop is to run the functions multiple times + var javaRunTimeList = List[Double]() + var fastSplitRunTimeList = List[Double]() for (i <- 1 to 100) { - val randomStrings: List[String] = (1 to 1000).map { + val randomStrings: List[String] = (1 to 100000).map { x => - randomString(20) + randomString(50) }.toList - val randomSeparatorIndex = scala.util.Random.nextInt(5) - val separator = "#@/: "(randomSeparatorIndex).toString - val splittedByRegex = randomStrings.map { s => s.split(separator).toList } - val splittedByFastSpliter = randomStrings.map { s => StringUtility.fastSplit(s, separator).toList } - splittedByRegex should be(splittedByFastSpliter) + val randomSeparatorIndex = scala.util.Random.nextInt(1) + val separator = "|"(randomSeparatorIndex).toString + + val fastSplitRunTime = time { + val splittedByFastSpliter = randomStrings.map { s => StringUtility.fastSplit(s, separator).toList } + } + fastSplitRunTimeList = fastSplitRunTime :: fastSplitRunTimeList + + val javaRunTime = time { + val splittedByRegex = randomStrings.map { s => s.split(separator).toList } + } + + javaRunTimeList = javaRunTime :: javaRunTimeList + } - } + def meanAndStd(list: List[Double]): (Double, Double, Double, Double) = { + val s = list.sum + val mean = s / list.size + val std = math.sqrt(list.map{ x => x * x }.sum / list.size - mean * mean) + val sorted = list.sorted + val median = sorted(list.length / 2) + (mean, std, median, s) + } + + // assert that total time for fastSplit is really faster here? + println("mean, std, median, and total time for running java's split" + meanAndStd(javaRunTimeList)) + println("mean, std, median, and total time for running java's split" + meanAndStd(fastSplitRunTimeList)) + } } From 231e16a91d27cabb337b76fafbd0af3cc64adedc Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Sat, 11 Apr 2015 18:14:10 -0700 Subject: [PATCH 089/177] Cache in stats --- .../src/main/scala/com/twitter/scalding/Stats.scala | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Stats.scala b/scalding-core/src/main/scala/com/twitter/scalding/Stats.scala index 8dcafa80ac..1d538d17f4 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Stats.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Stats.scala @@ -125,9 +125,15 @@ object RuntimeStats extends java.io.Serializable { def addFlowProcess(fp: FlowProcess[_]) { val uniqueJobIdObj = fp.getProperty(UniqueID.UNIQUE_JOB_ID) if (uniqueJobIdObj != null) { - StringUtility.fastSplit(uniqueJobIdObj.asInstanceOf[String], ",").foreach { uniqueId => + // for speed concern, use a while loop instead of foreach here + val splitted = StringUtility.fastSplit(uniqueJobIdObj.asInstanceOf[String], ",") + val size = splitted.length + var i = 0 + while (i < size) { + val uniqueId = splitted(i) logger.debug("Adding flow process id: " + uniqueId) - flowMappingStore.put(uniqueId, new WeakReference(fp)) + flowMappingStore.getOrElse(uniqueId, new WeakReference(fp)) + i += 1 } } } From aa6486856c7900824d766395940b7d12d586b5ed Mon Sep 17 00:00:00 2001 From: Jian TANG Date: Sat, 14 Feb 2015 09:48:29 +0100 Subject: [PATCH 090/177] Typed Parquet Tuple * Used as sink to write tuple with primitive fields * Create Parquet schema generation macro --- project/Build.scala | 10 +- .../parquet/tuple/TypedParquetTuple.scala | 62 +++++++++ .../parquet/tuple/macros/MacroImplicits.scala | 10 ++ .../parquet/tuple/macros/Macros.scala | 9 ++ .../macros/impl/SchemaProviderImpl.scala | 84 +++++++++++ .../parquet/tuple/TypedParquetTupleTest.scala | 130 ++++++++++++++++++ .../parquet/tuple/macros/MacroUnitTests.scala | 86 ++++++++++++ 7 files changed, 387 insertions(+), 4 deletions(-) create mode 100644 scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/TypedParquetTuple.scala create mode 100644 scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/MacroImplicits.scala create mode 100644 scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/Macros.scala create mode 100644 scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/SchemaProviderImpl.scala create mode 100644 scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala create mode 100644 scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/macros/MacroUnitTests.scala diff --git a/project/Build.scala b/project/Build.scala index ffa4197dd9..643c56c065 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -303,7 +303,7 @@ object ScaldingBuild extends Build { ).dependsOn(scaldingCore) lazy val scaldingParquet = module("parquet").settings( - libraryDependencies ++= Seq( + libraryDependencies <++= (scalaVersion) { scalaVersion => Seq( // see https://issues.apache.org/jira/browse/PARQUET-143 for exclusions "com.twitter" % "parquet-cascading" % parquetVersion exclude("com.twitter", "parquet-pig") @@ -311,9 +311,11 @@ object ScaldingBuild extends Build { exclude("com.twitter.elephantbird", "elephant-bird-core"), "org.apache.thrift" % "libthrift" % "0.7.0", "org.slf4j" % "slf4j-api" % slf4jVersion, - "org.apache.hadoop" % "hadoop-core" % hadoopVersion % "provided" - ) - ).dependsOn(scaldingCore) + "org.apache.hadoop" % "hadoop-core" % hadoopVersion % "provided", + "org.scala-lang" % "scala-reflect" % scalaVersion, + "com.twitter" %% "bijection-macros" % bijectionVersion + ) ++ (if(isScala210x(scalaVersion)) Seq("org.scalamacros" %% "quasiquotes" % "2.0.1") else Seq()) + }, addCompilerPlugin("org.scalamacros" % "paradise" % "2.0.1" cross CrossVersion.full)).dependsOn(scaldingCore) def scaldingParquetScroogeDeps(version: String) = { if (isScala210x(version)) diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/TypedParquetTuple.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/TypedParquetTuple.scala new file mode 100644 index 0000000000..89e554b5bf --- /dev/null +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/TypedParquetTuple.scala @@ -0,0 +1,62 @@ +package com.twitter.scalding.parquet.tuple + +import _root_.parquet.cascading.ParquetTupleScheme +import _root_.parquet.schema.MessageType +import cascading.scheme.Scheme +import cascading.tuple.Fields +import com.twitter.scalding._ +import com.twitter.scalding.parquet.HasFilterPredicate + +/** + * Typed parquet tuple source/sink, if used as sink, user should provide parquet schema definition. + * @author Jian Tang + */ +object TypedParquetTuple { + + def apply[T: Manifest : TupleConverter : TupleSetter](paths: Seq[String], sinkFields: Fields, + messageType: MessageType) = + new TypedFixedPathParquetTuple[T](paths, Fields.ALL, sinkFields, Some(messageType.toString)) + + def apply[T: Manifest : TupleConverter : TupleSetter](paths: Seq[String], sinkFields: Fields, messageType: String) = + new TypedFixedPathParquetTuple[T](paths, Fields.ALL, sinkFields, Some(messageType)) + + def apply[T: Manifest : TupleConverter : TupleSetter](paths: Seq[String], sourceFields: Fields) = + new TypedFixedPathParquetTuple[T](paths, sourceFields) +} + +/** + * Typed Parquet tuple source/sink. + */ +trait TypedParquetTuple[T] extends FileSource with Mappable[T] with TypedSink[T] with HasFilterPredicate { + + implicit val tupleConverter: TupleConverter[T] + + implicit val tupleSetter: TupleSetter[T] + + override def converter[U >: T] = TupleConverter.asSuperConverter[T, U](tupleConverter) + + override def setter[U <: T] = TupleSetter.asSubSetter[T, U](tupleSetter) + + /** + * Parquet schema definition for mapping with type T. + */ + def parquetSchema: Option[String] + + override def hdfsScheme = { + val scheme = parquetSchema match { + case Some(messageType) => new ParquetTupleScheme(sourceFields, sinkFields, messageType) + case _ => + withFilter match { + case Some(filterPredicate) => new ParquetTupleScheme(filterPredicate, sourceFields) + case _ => new ParquetTupleScheme(sourceFields) + } + } + HadoopSchemeInstance(scheme.asInstanceOf[Scheme[_, _, _, _, _]]) + } +} + +class TypedFixedPathParquetTuple[T](paths: Seq[String], + override val sourceFields: Fields = Fields.ALL, + override val sinkFields: Fields = Fields.UNKNOWN, + override val parquetSchema: Option[String] = None)(implicit override val tupleConverter: TupleConverter[T], + override val tupleSetter: TupleSetter[T]) extends FixedPathSource(paths: _*) with TypedParquetTuple[T] diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/MacroImplicits.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/MacroImplicits.scala new file mode 100644 index 0000000000..9e27a2f7d0 --- /dev/null +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/MacroImplicits.scala @@ -0,0 +1,10 @@ +package com.twitter.scalding.parquet.tuple.macros + +import scala.language.experimental.macros + +import com.twitter.scalding.parquet.tuple.macros.impl.SchemaProviderImpl +import _root_.parquet.schema.MessageType + +object MacroImplicits { + implicit def materializeCaseClassTypeDescriptor[T]: MessageType = macro SchemaProviderImpl.toParquetSchemaImp[T] +} diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/Macros.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/Macros.scala new file mode 100644 index 0000000000..a8511326ea --- /dev/null +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/Macros.scala @@ -0,0 +1,9 @@ +package com.twitter.scalding.parquet.tuple.macros + +import scala.language.experimental.macros + +import com.twitter.scalding.parquet.tuple.macros.impl.SchemaProviderImpl + +object Macros { + def caseClassParquetSchema[T]: _root_.parquet.schema.MessageType = macro SchemaProviderImpl.toParquetSchemaImp[T] +} diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/SchemaProviderImpl.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/SchemaProviderImpl.scala new file mode 100644 index 0000000000..dad4d7b29c --- /dev/null +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/SchemaProviderImpl.scala @@ -0,0 +1,84 @@ +package com.twitter.scalding.parquet.tuple.macros.impl + +import scala.language.experimental.macros +import com.twitter.bijection.macros.{ IsCaseClass, MacroGenerated } +import com.twitter.bijection.macros.impl.IsCaseClassImpl + +import scala.reflect.macros.Context + +object SchemaProviderImpl { + + def toParquetSchemaImp[T](c: Context)(implicit T: c.WeakTypeTag[T]): c.Expr[_root_.parquet.schema.MessageType] = { + + import c.universe._ + + if (!IsCaseClassImpl.isCaseClassType(c)(T.tpe)) + c.abort(c.enclosingPosition, s"""We cannot enforce ${T.tpe} is a case class, either it is not a case class or this macro call is possibly enclosed in a class. + This will mean the macro is operating on a non-resolved type.""") + + case class Extractor(tpe: Type, toTree: Tree) + case class Builder(toTree: Tree = q"") + + implicit val builderLiftable = new Liftable[Builder] { + def apply(b: Builder): Tree = b.toTree + } + + implicit val extractorLiftable = new Liftable[Extractor] { + def apply(b: Extractor): Tree = b.toTree + } + + lazy val REPETITION_REQUIRED = q"_root_.parquet.schema.Type.Repetition.REQUIRED" + lazy val REPETITION_OPTIONAL = q"_root_.parquet.schema.Type.Repetition.OPTIONAL" + + def getRepetition(isOption: Boolean): Tree = { + if (isOption) REPETITION_OPTIONAL else REPETITION_REQUIRED + } + + def matchField(fieldType: Type, outerName: String, fieldName: String, isOption: Boolean): List[Tree] = { + val parquetFieldName = s"$outerName$fieldName" + fieldType match { + case tpe if tpe =:= typeOf[String] => + List(q"""new _root_.parquet.schema.PrimitiveType(${getRepetition(isOption)}, _root_.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY, $parquetFieldName)""") + case tpe if tpe =:= typeOf[Boolean] => + List(q"""new _root_.parquet.schema.PrimitiveType(${getRepetition(isOption)}, _root_.parquet.schema.PrimitiveType.PrimitiveTypeName.BOOLEAN, $parquetFieldName)""") + case tpe if tpe =:= typeOf[Short] || tpe =:= typeOf[Int] => + List(q"""new _root_.parquet.schema.PrimitiveType(${getRepetition(isOption)}, _root_.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32, $parquetFieldName)""") + case tpe if tpe =:= typeOf[Long] => + List(q"""new _root_.parquet.schema.PrimitiveType(${getRepetition(isOption)}, _root_.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64, $parquetFieldName)""") + case tpe if tpe =:= typeOf[Float] => + List(q"""new _root_.parquet.schema.PrimitiveType(${getRepetition(isOption)}, _root_.parquet.schema.PrimitiveType.PrimitiveTypeName.FLOAT, $parquetFieldName)""") + case tpe if tpe =:= typeOf[Double] => + List(q"""new _root_.parquet.schema.PrimitiveType(${getRepetition(isOption)}, _root_.parquet.schema.PrimitiveType.PrimitiveTypeName.DOUBLE, $parquetFieldName)""") + case tpe if tpe.erasure =:= typeOf[Option[Any]] && isOption => + c.abort(c.enclosingPosition, s"Nested options do not make sense being mapped onto a tuple fields in cascading.") + case tpe if tpe.erasure =:= typeOf[Option[Any]] => + val innerType = tpe.asInstanceOf[TypeRefApi].args.head + matchField(innerType, outerName, fieldName, true) + case tpe if IsCaseClassImpl.isCaseClassType(c)(tpe) => expandMethod(tpe, s"$parquetFieldName.", isOption = false) + case _ => c.abort(c.enclosingPosition, s"Case class $T is not pure primitives or nested case classes") + } + } + + def expandMethod(outerTpe: Type, outerName: String, isOption: Boolean): List[Tree] = { + outerTpe + .declarations + .collect { case m: MethodSymbol if m.isCaseAccessor => m } + .flatMap { accessorMethod => + val fieldName = accessorMethod.name.toTermName.toString + val fieldType = accessorMethod.returnType + matchField(fieldType, outerName, fieldName, false) + }.toList + } + + def expandCaseClass(outerTpe: Type, outerName: String, isOption: Boolean): Tree = { + val expanded = expandMethod(outerTpe, outerName, isOption) + if (expanded.isEmpty) c.abort(c.enclosingPosition, s"Case class $outerTpe has no primitive types we were able to extract") + val messageTypeName = s"${outerTpe}".split("\\.").last + q"""new _root_.parquet.schema.MessageType($messageTypeName, + _root_.scala.Array.apply[parquet.schema.Type](..$expanded):_*) + """ + } + + c.Expr[parquet.schema.MessageType](expandCaseClass(T.tpe, "", isOption = false)) + } +} diff --git a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala new file mode 100644 index 0000000000..3b5a00a76c --- /dev/null +++ b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala @@ -0,0 +1,130 @@ +package com.twitter.scalding.parquet.tuple + +import java.io.File + +import cascading.tuple.{Fields, TupleEntry, Tuple} +import com.twitter.scalding.typed.TypedPipe +import com.twitter.scalding.{TupleSetter, TupleConverter, Job, Args, Mode, TypedCsv} +import scala.io.{Source => ScalaSource} + +import org.apache.hadoop +import org.scalatest.{Matchers, WordSpec} + +class TypedParquetTupleTest extends WordSpec with Matchers { + "TypedParquetTuple" should { + + "read and write correctly" in { + + val values = Seq(SampleClass("A", 1, 4.0D), SampleClass("B", 2, 3.0D), SampleClass("A", 2, 5.0D)) + + //write values to typed parquet sink + val tempParquet = createTempDir("parquet_tuple_test_parquet_") + val writeJobArgs = buildJobArgs(Array("--output", tempParquet, "--hdfs")) + val writeSuccess = new WriteToTypedParquetTupleJob(values, writeJobArgs).run + writeSuccess shouldEqual true + + //read from the written parquet tuple and write the values read into a csv file + val tempCsv = createTempDir("parquet_tuple_test_csv_") + val readJobArgs = buildJobArgs(Array("--input", tempParquet, "--output", tempCsv, "--hdfs")) + val readSuccess = new ReadFromTypedParquetTupleJob(readJobArgs).run + readSuccess shouldEqual true + + //check data correctness + val csv = ScalaSource.fromFile(new java.io.File(tempCsv, "part-00000")) + csv.getLines().toList shouldEqual Seq("A,1,4.0", "B,2,3.0", "A,2,5.0") + + //clean temporary files generated by the jobs + deletePath(tempCsv) + deletePath(tempParquet) + } + } + + def buildJobArgs(argArray: Array[String]): Args = { + val tool = new com.twitter.scalding.Tool + tool.setConf(new hadoop.mapred.JobConf) + val args = tool.parseModeArgs(argArray) + Mode.putMode(args._1, args._2) + } + + def createTempDir(prefix: String): String = { + java.nio.file.Files.createTempDirectory(prefix).toAbsolutePath.toString + } + + def deletePath(path: String) = { + val dir = new File(path) + for { + files <- Option(dir.listFiles) + file <- files + } file.delete() + dir.delete() + } +} + +case class SampleClass(stringValue: String, intValue: Int, doubleValue: Double) + +/** + * Test job write a sequence of sample class values into a typed parquet tuple. + * To test typed parquet tuple can be used as sink + */ +class WriteToTypedParquetTupleJob(values: Seq[SampleClass], args: Args) extends Job(args) { + + import SampleClassDescriptor._ + + val outputPath = args.required("output") + + val parquetTuple = TypedParquetTuple[SampleClass](Seq(outputPath), sampleClassFields, sampleClassParquetSchema) + TypedPipe.from(values).write(parquetTuple) +} + +/** + * Test job read from a typed parquet tuple and write the mapped value into a typed csv sink + * To test typed parquet tuple can bse used as source and read data correctly + */ +class ReadFromTypedParquetTupleJob(args: Args) extends Job(args) { + + import SampleClassDescriptor._ + + val inputPath = args.required("input") + val outputPath = args.required("output") + + val parquetTuple = TypedParquetTuple[SampleClass](Seq(inputPath), sampleClassFields) + + TypedPipe.from(parquetTuple).map { + case SampleClass(string, int, double) => (string, int, double) + }.write(TypedCsv[(String, Int, Double)](outputPath)) +} + +/** + * Helper class with tuple related setter and converter + + * parquet schema using parquet schema generation macro + */ +object SampleClassDescriptor { + + import com.twitter.scalding.parquet.tuple.macros.Macros._ + + implicit val valueTupleSetter = new TupleSetter[SampleClass] { + override def apply(value: SampleClass): Tuple = { + val tuple = new Tuple() + tuple.addString(value.stringValue) + tuple.addInteger(value.intValue) + tuple.addDouble(value.doubleValue) + tuple + } + + override def arity: Int = 3 + } + + implicit val valueTupleConverter = new TupleConverter[SampleClass] { + override def apply(te: TupleEntry): SampleClass = { + val stringValue = te.getString("stringValue") + val intValue = te.getInteger("intValue") + val doubleValue = te.getDouble("doubleValue") + SampleClass(stringValue, intValue, doubleValue) + } + + override def arity: Int = 3 + } + + val sampleClassFields: Fields = new Fields("stringValue", "intValue", "doubleValue") + val sampleClassParquetSchema = caseClassParquetSchema[SampleClass] +} diff --git a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/macros/MacroUnitTests.scala b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/macros/MacroUnitTests.scala new file mode 100644 index 0000000000..2e3f3650d5 --- /dev/null +++ b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/macros/MacroUnitTests.scala @@ -0,0 +1,86 @@ +package com.twitter.scalding.parquet.tuple.macros + +import org.scalatest.{ WordSpec, Matchers } +import parquet.schema.MessageTypeParser + +case class SampleClassA(x: Int, y: String) +case class SampleClassB(a1: SampleClassA, a2: SampleClassA, y: String) +case class SampleClassC(a: SampleClassA, b: SampleClassB, c: SampleClassA, d: SampleClassB, e: SampleClassB) +case class SampleClassD(a: String, b: Boolean, c: Option[Short], d: Int, e: Long, f: Float, g: Option[Double]) + +class MacroUnitTests extends WordSpec with Matchers { + + "MacroGenerated Parquet schema generator" should { + + "Generate parquet schema for SampleClassA" in { + val schema = Macros.caseClassParquetSchema[SampleClassA] + val expectedSchema = MessageTypeParser.parseMessageType(""" + |message SampleClassA { + | required int32 x; + | required binary y; + |} + """.stripMargin) + schema shouldEqual expectedSchema + } + + "Generate parquet schema for SampleClassB" in { + val schema = Macros.caseClassParquetSchema[SampleClassB] + + val expectedSchema = MessageTypeParser.parseMessageType(""" + |message SampleClassB { + | required int32 a1.x; + | required binary a1.y; + | required int32 a2.x; + | required binary a2.y; + | required binary y; + |} + """.stripMargin) + schema shouldEqual expectedSchema + } + + "Generate parquet schema for SampleClassC" in { + val schema = Macros.caseClassParquetSchema[SampleClassC] + + val expectedSchema = MessageTypeParser.parseMessageType(""" + |message SampleClassC { + | required int32 a.x; + | required binary a.y; + | required int32 b.a1.x; + | required binary b.a1.y; + | required int32 b.a2.x; + | required binary b.a2.y; + | required binary b.y; + | required int32 c.x; + | required binary c.y; + | required int32 d.a1.x; + | required binary d.a1.y; + | required int32 d.a2.x; + | required binary d.a2.y; + | required binary d.y; + | required int32 e.a1.x; + | required binary e.a1.y; + | required int32 e.a2.x; + | required binary e.a2.y; + | required binary e.y; + |} + """.stripMargin) + schema shouldEqual expectedSchema + } + + "Generate parquet schema for SampleClassD" in { + val schema = Macros.caseClassParquetSchema[SampleClassD] + val expectedSchema = MessageTypeParser.parseMessageType(""" + |message SampleClassD { + | required binary a; + | required boolean b; + | optional int32 c; + | required int32 d; + | required int64 e; + | required float f; + | optional double g; + |} + """.stripMargin) + schema shouldEqual expectedSchema + } + } +} From 97c19801f073b0562078747738116ef6dc16c23c Mon Sep 17 00:00:00 2001 From: beethere Date: Wed, 18 Feb 2015 01:46:21 +0100 Subject: [PATCH 091/177] Typed Parquet Tuple *Refacto unit test by using platform test *Refacto macro code (cherry picked from commit 40cd1eb772bbdeb38324848c5e84dde8931e8857) --- project/Build.scala | 3 +- .../parquet/tuple/TypedParquetTuple.scala | 8 +- .../parquet/tuple/macros/MacroImplicits.scala | 3 +- .../parquet/tuple/macros/Macros.scala | 4 +- .../macros/impl/SchemaProviderImpl.scala | 34 ++++---- .../parquet/tuple/TypedParquetTupleTest.scala | 86 +++++++------------ .../parquet/tuple/macros/MacroUnitTests.scala | 17 ++++ 7 files changed, 76 insertions(+), 79 deletions(-) diff --git a/project/Build.scala b/project/Build.scala index 643c56c065..991c764c8a 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -315,7 +315,8 @@ object ScaldingBuild extends Build { "org.scala-lang" % "scala-reflect" % scalaVersion, "com.twitter" %% "bijection-macros" % bijectionVersion ) ++ (if(isScala210x(scalaVersion)) Seq("org.scalamacros" %% "quasiquotes" % "2.0.1") else Seq()) - }, addCompilerPlugin("org.scalamacros" % "paradise" % "2.0.1" cross CrossVersion.full)).dependsOn(scaldingCore) + }, addCompilerPlugin("org.scalamacros" % "paradise" % "2.0.1" cross CrossVersion.full)) + .dependsOn(scaldingCore, scaldingHadoopTest) def scaldingParquetScroogeDeps(version: String) = { if (isScala210x(version)) diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/TypedParquetTuple.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/TypedParquetTuple.scala index 89e554b5bf..4b3f261740 100644 --- a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/TypedParquetTuple.scala +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/TypedParquetTuple.scala @@ -13,14 +13,10 @@ import com.twitter.scalding.parquet.HasFilterPredicate */ object TypedParquetTuple { - def apply[T: Manifest : TupleConverter : TupleSetter](paths: Seq[String], sinkFields: Fields, - messageType: MessageType) = + def apply[T: Manifest: TupleConverter: TupleSetter](paths: Seq[String])(implicit sinkFields: Fields, messageType: MessageType) = new TypedFixedPathParquetTuple[T](paths, Fields.ALL, sinkFields, Some(messageType.toString)) - def apply[T: Manifest : TupleConverter : TupleSetter](paths: Seq[String], sinkFields: Fields, messageType: String) = - new TypedFixedPathParquetTuple[T](paths, Fields.ALL, sinkFields, Some(messageType)) - - def apply[T: Manifest : TupleConverter : TupleSetter](paths: Seq[String], sourceFields: Fields) = + def apply[T: Manifest: TupleConverter: TupleSetter](paths: Seq[String], sourceFields: Fields) = new TypedFixedPathParquetTuple[T](paths, sourceFields) } diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/MacroImplicits.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/MacroImplicits.scala index 9e27a2f7d0..b288a01876 100644 --- a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/MacroImplicits.scala +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/MacroImplicits.scala @@ -6,5 +6,6 @@ import com.twitter.scalding.parquet.tuple.macros.impl.SchemaProviderImpl import _root_.parquet.schema.MessageType object MacroImplicits { - implicit def materializeCaseClassTypeDescriptor[T]: MessageType = macro SchemaProviderImpl.toParquetSchemaImp[T] + //implicits conversion from case class to parquet message type + implicit def materializeCaseClassParquetSchema[T]: MessageType = macro SchemaProviderImpl.toParquetSchemaImp[T] } diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/Macros.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/Macros.scala index a8511326ea..be48770475 100644 --- a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/Macros.scala +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/Macros.scala @@ -1,9 +1,11 @@ package com.twitter.scalding.parquet.tuple.macros +import parquet.schema.MessageType + import scala.language.experimental.macros import com.twitter.scalding.parquet.tuple.macros.impl.SchemaProviderImpl object Macros { - def caseClassParquetSchema[T]: _root_.parquet.schema.MessageType = macro SchemaProviderImpl.toParquetSchemaImp[T] + def caseClassParquetSchema[T]: MessageType = macro SchemaProviderImpl.toParquetSchemaImp[T] } diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/SchemaProviderImpl.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/SchemaProviderImpl.scala index dad4d7b29c..b6d9c754f8 100644 --- a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/SchemaProviderImpl.scala +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/SchemaProviderImpl.scala @@ -1,14 +1,14 @@ package com.twitter.scalding.parquet.tuple.macros.impl import scala.language.experimental.macros -import com.twitter.bijection.macros.{ IsCaseClass, MacroGenerated } import com.twitter.bijection.macros.impl.IsCaseClassImpl +import parquet.schema.MessageType import scala.reflect.macros.Context object SchemaProviderImpl { - def toParquetSchemaImp[T](c: Context)(implicit T: c.WeakTypeTag[T]): c.Expr[_root_.parquet.schema.MessageType] = { + def toParquetSchemaImp[T](c: Context)(implicit T: c.WeakTypeTag[T]): c.Expr[MessageType] = { import c.universe._ @@ -27,28 +27,29 @@ object SchemaProviderImpl { def apply(b: Extractor): Tree = b.toTree } - lazy val REPETITION_REQUIRED = q"_root_.parquet.schema.Type.Repetition.REQUIRED" - lazy val REPETITION_OPTIONAL = q"_root_.parquet.schema.Type.Repetition.OPTIONAL" + val REPETITION_REQUIRED = q"ParquetType.Repetition.REQUIRED" + val REPETITION_OPTIONAL = q"ParquetType.Repetition.OPTIONAL" - def getRepetition(isOption: Boolean): Tree = { - if (isOption) REPETITION_OPTIONAL else REPETITION_REQUIRED - } + def createPrimitiveTypeField(isOption: Boolean, primitiveType: Tree, fieldName: String): List[Tree] = + List(q"""new PrimitiveType(${getRepetition(isOption)}, $primitiveType, $fieldName)""") + + def getRepetition(isOption: Boolean): Tree = if (isOption) REPETITION_OPTIONAL else REPETITION_REQUIRED def matchField(fieldType: Type, outerName: String, fieldName: String, isOption: Boolean): List[Tree] = { val parquetFieldName = s"$outerName$fieldName" fieldType match { case tpe if tpe =:= typeOf[String] => - List(q"""new _root_.parquet.schema.PrimitiveType(${getRepetition(isOption)}, _root_.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY, $parquetFieldName)""") + createPrimitiveTypeField(isOption, q"PrimitiveType.PrimitiveTypeName.BINARY", parquetFieldName) case tpe if tpe =:= typeOf[Boolean] => - List(q"""new _root_.parquet.schema.PrimitiveType(${getRepetition(isOption)}, _root_.parquet.schema.PrimitiveType.PrimitiveTypeName.BOOLEAN, $parquetFieldName)""") + createPrimitiveTypeField(isOption, q"PrimitiveType.PrimitiveTypeName.BOOLEAN", parquetFieldName) case tpe if tpe =:= typeOf[Short] || tpe =:= typeOf[Int] => - List(q"""new _root_.parquet.schema.PrimitiveType(${getRepetition(isOption)}, _root_.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32, $parquetFieldName)""") + createPrimitiveTypeField(isOption, q"PrimitiveType.PrimitiveTypeName.INT32", parquetFieldName) case tpe if tpe =:= typeOf[Long] => - List(q"""new _root_.parquet.schema.PrimitiveType(${getRepetition(isOption)}, _root_.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64, $parquetFieldName)""") + createPrimitiveTypeField(isOption, q"PrimitiveType.PrimitiveTypeName.INT64", parquetFieldName) case tpe if tpe =:= typeOf[Float] => - List(q"""new _root_.parquet.schema.PrimitiveType(${getRepetition(isOption)}, _root_.parquet.schema.PrimitiveType.PrimitiveTypeName.FLOAT, $parquetFieldName)""") + createPrimitiveTypeField(isOption, q"PrimitiveType.PrimitiveTypeName.FLOAT", parquetFieldName) case tpe if tpe =:= typeOf[Double] => - List(q"""new _root_.parquet.schema.PrimitiveType(${getRepetition(isOption)}, _root_.parquet.schema.PrimitiveType.PrimitiveTypeName.DOUBLE, $parquetFieldName)""") + createPrimitiveTypeField(isOption, q"PrimitiveType.PrimitiveTypeName.DOUBLE", parquetFieldName) case tpe if tpe.erasure =:= typeOf[Option[Any]] && isOption => c.abort(c.enclosingPosition, s"Nested options do not make sense being mapped onto a tuple fields in cascading.") case tpe if tpe.erasure =:= typeOf[Option[Any]] => @@ -74,11 +75,10 @@ object SchemaProviderImpl { val expanded = expandMethod(outerTpe, outerName, isOption) if (expanded.isEmpty) c.abort(c.enclosingPosition, s"Case class $outerTpe has no primitive types we were able to extract") val messageTypeName = s"${outerTpe}".split("\\.").last - q"""new _root_.parquet.schema.MessageType($messageTypeName, - _root_.scala.Array.apply[parquet.schema.Type](..$expanded):_*) - """ + q"""import parquet.schema.{MessageType, PrimitiveType, Type => ParquetType} + new MessageType($messageTypeName, Array.apply[ParquetType](..$expanded):_*)""" } - c.Expr[parquet.schema.MessageType](expandCaseClass(T.tpe, "", isOption = false)) + c.Expr[MessageType](expandCaseClass(T.tpe, "", isOption = false)) } } diff --git a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala index 3b5a00a76c..57803540f5 100644 --- a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala +++ b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala @@ -2,54 +2,35 @@ package com.twitter.scalding.parquet.tuple import java.io.File -import cascading.tuple.{Fields, TupleEntry, Tuple} +import cascading.tuple.{ Fields, TupleEntry, Tuple } +import com.twitter.scalding.platform.{ HadoopPlatformJobTest, HadoopPlatformTest } import com.twitter.scalding.typed.TypedPipe -import com.twitter.scalding.{TupleSetter, TupleConverter, Job, Args, Mode, TypedCsv} -import scala.io.{Source => ScalaSource} +import com.twitter.scalding.{ TupleSetter, TupleConverter, Job, Args, TypedTsv } +import parquet.schema.MessageType -import org.apache.hadoop -import org.scalatest.{Matchers, WordSpec} +import org.scalatest.{ Matchers, WordSpec } -class TypedParquetTupleTest extends WordSpec with Matchers { +class TypedParquetTupleTest extends WordSpec with Matchers with HadoopPlatformTest { "TypedParquetTuple" should { "read and write correctly" in { - - val values = Seq(SampleClass("A", 1, 4.0D), SampleClass("B", 2, 3.0D), SampleClass("A", 2, 5.0D)) - - //write values to typed parquet sink - val tempParquet = createTempDir("parquet_tuple_test_parquet_") - val writeJobArgs = buildJobArgs(Array("--output", tempParquet, "--hdfs")) - val writeSuccess = new WriteToTypedParquetTupleJob(values, writeJobArgs).run - writeSuccess shouldEqual true - - //read from the written parquet tuple and write the values read into a csv file - val tempCsv = createTempDir("parquet_tuple_test_csv_") - val readJobArgs = buildJobArgs(Array("--input", tempParquet, "--output", tempCsv, "--hdfs")) - val readSuccess = new ReadFromTypedParquetTupleJob(readJobArgs).run - readSuccess shouldEqual true - - //check data correctness - val csv = ScalaSource.fromFile(new java.io.File(tempCsv, "part-00000")) - csv.getLines().toList shouldEqual Seq("A,1,4.0", "B,2,3.0", "A,2,5.0") - - //clean temporary files generated by the jobs - deletePath(tempCsv) - deletePath(tempParquet) + import TestValues._ + val tempParquet = java.nio.file.Files.createTempDirectory("parquet_tuple_test_parquet_").toAbsolutePath.toString + try { + HadoopPlatformJobTest(new WriteToTypedParquetTupleJob(_), cluster) + .arg("output", tempParquet) + .run + + HadoopPlatformJobTest(new ReadFromTypedParquetTupleJob(_), cluster) + .arg("input", tempParquet) + .sink[String]("output") { _.toSet shouldBe values.map(_.stringValue).toSet } + .run + } finally { + deletePath(tempParquet) + } } } - def buildJobArgs(argArray: Array[String]): Args = { - val tool = new com.twitter.scalding.Tool - tool.setConf(new hadoop.mapred.JobConf) - val args = tool.parseModeArgs(argArray) - Mode.putMode(args._1, args._2) - } - - def createTempDir(prefix: String): String = { - java.nio.file.Files.createTempDirectory(prefix).toAbsolutePath.toString - } - def deletePath(path: String) = { val dir = new File(path) for { @@ -60,19 +41,23 @@ class TypedParquetTupleTest extends WordSpec with Matchers { } } +object TestValues { + val values = Seq(SampleClass("A", 1, 4.0D), SampleClass("B", 2, 3.0D), SampleClass("A", 2, 5.0D)) +} + case class SampleClass(stringValue: String, intValue: Int, doubleValue: Double) /** * Test job write a sequence of sample class values into a typed parquet tuple. * To test typed parquet tuple can be used as sink */ -class WriteToTypedParquetTupleJob(values: Seq[SampleClass], args: Args) extends Job(args) { +class WriteToTypedParquetTupleJob(args: Args) extends Job(args) { import SampleClassDescriptor._ - + import TestValues._ val outputPath = args.required("output") - val parquetTuple = TypedParquetTuple[SampleClass](Seq(outputPath), sampleClassFields, sampleClassParquetSchema) + val parquetTuple = TypedParquetTuple[SampleClass](Seq(outputPath)) TypedPipe.from(values).write(parquetTuple) } @@ -85,13 +70,10 @@ class ReadFromTypedParquetTupleJob(args: Args) extends Job(args) { import SampleClassDescriptor._ val inputPath = args.required("input") - val outputPath = args.required("output") - val parquetTuple = TypedParquetTuple[SampleClass](Seq(inputPath), sampleClassFields) + val parquetTuple = TypedParquetTuple[SampleClass](Seq(inputPath)) - TypedPipe.from(parquetTuple).map { - case SampleClass(string, int, double) => (string, int, double) - }.write(TypedCsv[(String, Int, Double)](outputPath)) + TypedPipe.from(parquetTuple).map(_.stringValue).write(TypedTsv[String]("output")) } /** @@ -99,10 +81,8 @@ class ReadFromTypedParquetTupleJob(args: Args) extends Job(args) { * parquet schema using parquet schema generation macro */ object SampleClassDescriptor { - import com.twitter.scalding.parquet.tuple.macros.Macros._ - - implicit val valueTupleSetter = new TupleSetter[SampleClass] { + implicit val valueTupleSetter: TupleSetter[SampleClass] = new TupleSetter[SampleClass] { override def apply(value: SampleClass): Tuple = { val tuple = new Tuple() tuple.addString(value.stringValue) @@ -114,7 +94,7 @@ object SampleClassDescriptor { override def arity: Int = 3 } - implicit val valueTupleConverter = new TupleConverter[SampleClass] { + implicit val valueTupleConverter: TupleConverter[SampleClass] = new TupleConverter[SampleClass] { override def apply(te: TupleEntry): SampleClass = { val stringValue = te.getString("stringValue") val intValue = te.getInteger("intValue") @@ -125,6 +105,6 @@ object SampleClassDescriptor { override def arity: Int = 3 } - val sampleClassFields: Fields = new Fields("stringValue", "intValue", "doubleValue") - val sampleClassParquetSchema = caseClassParquetSchema[SampleClass] + implicit val sampleClassFields: Fields = new Fields("stringValue", "intValue", "doubleValue") + implicit val sampleClassParquetSchema: MessageType = caseClassParquetSchema[SampleClass] } diff --git a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/macros/MacroUnitTests.scala b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/macros/MacroUnitTests.scala index 2e3f3650d5..ed28ae96f2 100644 --- a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/macros/MacroUnitTests.scala +++ b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/macros/MacroUnitTests.scala @@ -7,6 +7,7 @@ case class SampleClassA(x: Int, y: String) case class SampleClassB(a1: SampleClassA, a2: SampleClassA, y: String) case class SampleClassC(a: SampleClassA, b: SampleClassB, c: SampleClassA, d: SampleClassB, e: SampleClassB) case class SampleClassD(a: String, b: Boolean, c: Option[Short], d: Int, e: Long, f: Float, g: Option[Double]) +case class SampleClassE(a: Int, b: Long, c: Short, d: Boolean, e: Float, f: Double, g: String) class MacroUnitTests extends WordSpec with Matchers { @@ -82,5 +83,21 @@ class MacroUnitTests extends WordSpec with Matchers { """.stripMargin) schema shouldEqual expectedSchema } + + "Generate parquet schema for SampleClassE" in { + val schema = Macros.caseClassParquetSchema[SampleClassE] + val expectedSchema = MessageTypeParser.parseMessageType(""" + |message SampleClassE { + | required int32 a; + | required int64 b; + | required int32 c; + | required boolean d; + | required float e; + | required double f; + | required binary g; + |} + """.stripMargin) + schema shouldEqual expectedSchema + } } } From 8dbef396c1fac76de88e804d8047a844311b567b Mon Sep 17 00:00:00 2001 From: beethere Date: Mon, 23 Feb 2015 23:34:01 +0100 Subject: [PATCH 092/177] Typed Parquet Tuple #1198 *Macro support nested group type *Add r/w support nested case classes *Tests + refacto --- .../scalding/parquet/tuple/TypedParquet.scala | 110 ++++++++++ .../parquet/tuple/TypedParquetTuple.scala | 58 ----- .../parquet/tuple/macros/MacroImplicits.scala | 11 - .../parquet/tuple/macros/Macros.scala | 58 ++++- .../macros/impl/FieldValuesProvider.scala | 76 +++++++ .../macros/impl/ParquetSchemaProvider.scala | 69 ++++++ .../impl/ParquetTupleConverterProvider.scala | 89 ++++++++ .../macros/impl/SchemaProviderImpl.scala | 84 ------- .../tuple/scheme/ParquetTupleConverter.scala | 89 ++++++++ .../scheme/TypedParquetTupleScheme.scala | 205 +++++++++++++++++ .../parquet/tuple/TypedParquetTupleTest.scala | 90 ++++---- .../parquet/tuple/macros/MacroUnitTests.scala | 206 +++++++++++++++--- 12 files changed, 912 insertions(+), 233 deletions(-) create mode 100644 scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/TypedParquet.scala delete mode 100644 scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/TypedParquetTuple.scala delete mode 100644 scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/MacroImplicits.scala create mode 100644 scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/FieldValuesProvider.scala create mode 100644 scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/ParquetSchemaProvider.scala create mode 100644 scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/ParquetTupleConverterProvider.scala delete mode 100644 scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/SchemaProviderImpl.scala create mode 100644 scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/ParquetTupleConverter.scala create mode 100644 scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/TypedParquetTupleScheme.scala diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/TypedParquet.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/TypedParquet.scala new file mode 100644 index 0000000000..b9baff93bf --- /dev/null +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/TypedParquet.scala @@ -0,0 +1,110 @@ +package com.twitter.scalding.parquet.tuple + +import _root_.parquet.filter2.predicate.FilterPredicate +import cascading.scheme.Scheme +import com.twitter.scalding._ +import com.twitter.scalding.parquet.HasFilterPredicate +import com.twitter.scalding.parquet.tuple.scheme.{ ParquetWriteSupport, ParquetReadSupport, TypedParquetTupleScheme } + +import scala.reflect.ClassTag + +/** + * Typed parquet tuple + * @author Jian Tang + */ +object TypedParquet { + /** + * Create readable typed parquet source. + * Here is an example: + * + * case class SampleClassB(string: String, int: Int, double: Option[Double], a: SampleClassA) + * + * class ReadSupport extends ParquetReadSupport[SampleClassB] { + * import com.twitter.scalding.parquet.tuple.macros.Macros._ + * override val tupleConverter: ParquetTupleConverter = caseClassParquetTupleConverter[SampleClassB] + * override val rootSchema: String = caseClassParquetSchema[SampleClassB] + * } + * + * val parquetTuple = TypedParquet[SampleClassB, ReadSupport](Seq(outputPath)) + * + * @param paths paths of parquet I/O + * @param t Read support type tag + * @tparam T Tuple type + * @tparam R Read support type + * @return a typed parquet source. + */ + def apply[T, R <: ParquetReadSupport[T]](paths: Seq[String])(implicit t: ClassTag[R]) = + new TypedFixedPathParquetTuple[T, R, ParquetWriteSupport[T]](paths, t.runtimeClass.asInstanceOf[Class[R]], null) + + /** + * Create readable typed parquet source with filter predicate. + */ + def apply[T, R <: ParquetReadSupport[T]](paths: Seq[String], fp: Option[FilterPredicate])(implicit t: ClassTag[R]) = + new TypedFixedPathParquetTuple[T, R, ParquetWriteSupport[T]](paths, t.runtimeClass.asInstanceOf[Class[R]], null) { + override def withFilter = fp + } + + /** + * Create typed parquet source supports both R/W. + * @param paths paths of parquet I/O + * @param r Read support type tag + * @param w Write support type tag + * @tparam T Tuple type + * @tparam R Read support type + * @return a typed parquet source. + */ + def apply[T, R <: ParquetReadSupport[T], W <: ParquetWriteSupport[T]](paths: Seq[String])(implicit r: ClassTag[R], + w: ClassTag[W]) = { + val readSupport = r.runtimeClass.asInstanceOf[Class[R]] + val writeSupport = w.runtimeClass.asInstanceOf[Class[W]] + new TypedFixedPathParquetTuple[T, R, W](paths, readSupport, writeSupport) + } + +} + +object TypedParquetSink { + /** + * Create typed parquet sink. + * Here is an example: + * + * case class SampleClassB(string: String, int: Int, double: Option[Double], a: SampleClassA) + * + * class WriteSupport extends ParquetWriteSupport[SampleClassB] { + * import com.twitter.scalding.parquet.tuple.macros.Macros._ + * override val fieldValues: (SampleClassB) => Map[Int, Any] = caseClassFieldValues[SampleClassB] + * override val rootSchema: String = caseClassParquetSchema[SampleClassB] + * } + * + * val sink = TypedParquetSink[SampleClassB, WriteSupport](Seq(outputPath)) + * + * @param paths paths of parquet I/O + * @param t Read support type tag + * @tparam T Tuple type + * @tparam W Write support type + * @return a typed parquet source. + */ + def apply[T, W <: ParquetWriteSupport[T]](paths: Seq[String])(implicit t: ClassTag[W]) = + new TypedFixedPathParquetTuple[T, ParquetReadSupport[T], W](paths, null, t.runtimeClass.asInstanceOf[Class[W]]) +} + +/** + * Typed Parquet tuple source/sink. + */ +trait TypedParquet[T, R <: ParquetReadSupport[T], W <: ParquetWriteSupport[T]] extends FileSource with Mappable[T] + with TypedSink[T] with HasFilterPredicate { + + val readSupport: Class[R] + val writeSupport: Class[W] + + override def converter[U >: T] = TupleConverter.asSuperConverter[T, U](TupleConverter.singleConverter[T]) + + override def setter[U <: T] = TupleSetter.asSubSetter[T, U](TupleSetter.singleSetter[T]) + + override def hdfsScheme = { + val scheme = new TypedParquetTupleScheme[T](readSupport, writeSupport, withFilter) + HadoopSchemeInstance(scheme.asInstanceOf[Scheme[_, _, _, _, _]]) + } +} + +class TypedFixedPathParquetTuple[T, R <: ParquetReadSupport[T], W <: ParquetWriteSupport[T]](val paths: Seq[String], + val readSupport: Class[R], val writeSupport: Class[W]) extends FixedPathSource(paths: _*) with TypedParquet[T, R, W] diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/TypedParquetTuple.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/TypedParquetTuple.scala deleted file mode 100644 index 4b3f261740..0000000000 --- a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/TypedParquetTuple.scala +++ /dev/null @@ -1,58 +0,0 @@ -package com.twitter.scalding.parquet.tuple - -import _root_.parquet.cascading.ParquetTupleScheme -import _root_.parquet.schema.MessageType -import cascading.scheme.Scheme -import cascading.tuple.Fields -import com.twitter.scalding._ -import com.twitter.scalding.parquet.HasFilterPredicate - -/** - * Typed parquet tuple source/sink, if used as sink, user should provide parquet schema definition. - * @author Jian Tang - */ -object TypedParquetTuple { - - def apply[T: Manifest: TupleConverter: TupleSetter](paths: Seq[String])(implicit sinkFields: Fields, messageType: MessageType) = - new TypedFixedPathParquetTuple[T](paths, Fields.ALL, sinkFields, Some(messageType.toString)) - - def apply[T: Manifest: TupleConverter: TupleSetter](paths: Seq[String], sourceFields: Fields) = - new TypedFixedPathParquetTuple[T](paths, sourceFields) -} - -/** - * Typed Parquet tuple source/sink. - */ -trait TypedParquetTuple[T] extends FileSource with Mappable[T] with TypedSink[T] with HasFilterPredicate { - - implicit val tupleConverter: TupleConverter[T] - - implicit val tupleSetter: TupleSetter[T] - - override def converter[U >: T] = TupleConverter.asSuperConverter[T, U](tupleConverter) - - override def setter[U <: T] = TupleSetter.asSubSetter[T, U](tupleSetter) - - /** - * Parquet schema definition for mapping with type T. - */ - def parquetSchema: Option[String] - - override def hdfsScheme = { - val scheme = parquetSchema match { - case Some(messageType) => new ParquetTupleScheme(sourceFields, sinkFields, messageType) - case _ => - withFilter match { - case Some(filterPredicate) => new ParquetTupleScheme(filterPredicate, sourceFields) - case _ => new ParquetTupleScheme(sourceFields) - } - } - HadoopSchemeInstance(scheme.asInstanceOf[Scheme[_, _, _, _, _]]) - } -} - -class TypedFixedPathParquetTuple[T](paths: Seq[String], - override val sourceFields: Fields = Fields.ALL, - override val sinkFields: Fields = Fields.UNKNOWN, - override val parquetSchema: Option[String] = None)(implicit override val tupleConverter: TupleConverter[T], - override val tupleSetter: TupleSetter[T]) extends FixedPathSource(paths: _*) with TypedParquetTuple[T] diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/MacroImplicits.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/MacroImplicits.scala deleted file mode 100644 index b288a01876..0000000000 --- a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/MacroImplicits.scala +++ /dev/null @@ -1,11 +0,0 @@ -package com.twitter.scalding.parquet.tuple.macros - -import scala.language.experimental.macros - -import com.twitter.scalding.parquet.tuple.macros.impl.SchemaProviderImpl -import _root_.parquet.schema.MessageType - -object MacroImplicits { - //implicits conversion from case class to parquet message type - implicit def materializeCaseClassParquetSchema[T]: MessageType = macro SchemaProviderImpl.toParquetSchemaImp[T] -} diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/Macros.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/Macros.scala index be48770475..1d75d78888 100644 --- a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/Macros.scala +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/Macros.scala @@ -1,11 +1,61 @@ package com.twitter.scalding.parquet.tuple.macros -import parquet.schema.MessageType +import com.twitter.scalding.parquet.tuple.macros.impl.{ ParquetTupleConverterProvider, ParquetSchemaProvider, FieldValuesProvider } +import com.twitter.scalding.parquet.tuple.scheme.ParquetTupleConverter import scala.language.experimental.macros -import com.twitter.scalding.parquet.tuple.macros.impl.SchemaProviderImpl - object Macros { - def caseClassParquetSchema[T]: MessageType = macro SchemaProviderImpl.toParquetSchemaImp[T] + /** + * Macro used to generate parquet schema for a given case class that contains only primitive fields. + * Option field and nested group is supported. For example if we have: + * + * case class SampleClassA(x: Int, y: String) + * case class SampleClassB(a: SampleClassA, y: String) + * + * The macro will generate a parquet message type like this: + * + * """ + * message SampleClassB { + * required group a { + * required int32 x; + * required binary y; + * } + * required binary y; + * } + * """ + * + * @tparam T Case class type that contains only primitive fields or nested case class. + * @return Generated case class parquet message type string + */ + def caseClassParquetSchema[T]: String = macro ParquetSchemaProvider.toParquetSchemaImpl[T] + + /** + * Macro used to generate function which permits to flat(at every level) a record to a index-value map. + * For example if we have: + * + * case class SampleClassA(short: Short, int: Int) + * case class SampleClassB(bool: Boolean, a: SampleClassA, long: Long, float: Float) + * + * val b = SampleClassB(true, SampleClassA(1, 4), 6L, 5F) + * + * After flatting using the generated function , we will get a map like this: + * + * Map(0 -> true, 1 -> 1, 2 -> 4, 3 -> 6L, 4 -> 5F) + * This macro can be used to define [[com.twitter.scalding.parquet.tuple.scheme.ParquetWriteSupport]]. + * See this class for more details. + * + * @tparam T Case class type that contains only primitive fields or nested case class. + * @return Case class record field values flat function + */ + def caseClassFieldValues[T]: T => Map[Int, Any] = macro FieldValuesProvider.toFieldValuesImpl[T] + + /** + * Macro used to generate parquet tuple converter for a given case class that contains only primitive fields. + * Option field and nested group is supported. + * + * @tparam T Case class type that contains only primitive fields or nested case class. + * @return Generated parquet converter + */ + def caseClassParquetTupleConverter[T]: ParquetTupleConverter = macro ParquetTupleConverterProvider.toParquetTupleConverterImpl[T] } diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/FieldValuesProvider.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/FieldValuesProvider.scala new file mode 100644 index 0000000000..501a444961 --- /dev/null +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/FieldValuesProvider.scala @@ -0,0 +1,76 @@ +package com.twitter.scalding.parquet.tuple.macros.impl + +import com.twitter.bijection.macros.impl.IsCaseClassImpl + +import scala.reflect.macros.Context + +object FieldValuesProvider { + + def toFieldValuesImpl[T](ctx: Context)(implicit T: ctx.WeakTypeTag[T]): ctx.Expr[T => Map[Int, Any]] = { + import ctx.universe._ + + if (!IsCaseClassImpl.isCaseClassType(ctx)(T.tpe)) + ctx.abort(ctx.enclosingPosition, + s"""We cannot enforce ${T.tpe} is a case class, + either it is not a case class or this macro call is possibly enclosed in a class. + This will mean the macro is operating on a non-resolved type.""") + + def matchField(idx: Int, fieldType: Type, pTree: Tree): (Int, Tree) = { + def appendFieldValue(idx: Int): (Int, Tree) = + (idx + 1, q"""if($pTree != null) fieldValueMap += $idx -> $pTree""") + + fieldType match { + case tpe if tpe =:= typeOf[String] || + tpe =:= typeOf[Boolean] || + tpe =:= typeOf[Short] || + tpe =:= typeOf[Int] || + tpe =:= typeOf[Long] || + tpe =:= typeOf[Float] || + tpe =:= typeOf[Double] => + appendFieldValue(idx) + + case tpe if tpe.erasure =:= typeOf[Option[Any]] => + val cacheName = newTermName(ctx.fresh(s"optionIndex")) + val innerType = tpe.asInstanceOf[TypeRefApi].args.head + val (newIdx, subTree) = matchField(idx, innerType, q"$cacheName") + (newIdx, q""" + if($pTree.isDefined) { + val $cacheName = $pTree.get + $subTree + } + """) + + case tpe if IsCaseClassImpl.isCaseClassType(ctx)(tpe) => expandMethod(idx, tpe, pTree) + case _ => ctx.abort(ctx.enclosingPosition, s"Case class $T is not pure primitives or nested case classes") + } + } + + def expandMethod(parentIdx: Int, outerTpe: Type, pTree: Tree): (Int, Tree) = { + outerTpe + .declarations + .collect { case m: MethodSymbol if m.isCaseAccessor => m } + .foldLeft((parentIdx, q"")) { + case ((idx, existingTree), accessorMethod) => + val (newIdx, subTree) = matchField(idx, accessorMethod.returnType, q"""$pTree.$accessorMethod""") + (newIdx, q""" + $existingTree + $subTree""") + } + } + + val (finalIdx, allFieldValues) = expandMethod(0, T.tpe, q"t") + + if (finalIdx == 0) + ctx.abort(ctx.enclosingPosition, "Didn't consume any elements in the tuple, possibly empty case class?") + + val fieldValues = q""" + val values: $T => _root_.scala.collection.immutable.Map[Int, Any] = t => { + var fieldValueMap = _root_.scala.collection.immutable.Map[Int, Any]() + $allFieldValues + fieldValueMap + } + values + """ + ctx.Expr[T => Map[Int, Any]](fieldValues) + } +} diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/ParquetSchemaProvider.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/ParquetSchemaProvider.scala new file mode 100644 index 0000000000..61878eb723 --- /dev/null +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/ParquetSchemaProvider.scala @@ -0,0 +1,69 @@ +package com.twitter.scalding.parquet.tuple.macros.impl + +import com.twitter.bijection.macros.impl.IsCaseClassImpl + +import scala.reflect.macros.Context + +object ParquetSchemaProvider { + def toParquetSchemaImpl[T](c: Context)(implicit T: c.WeakTypeTag[T]): c.Expr[String] = { + import c.universe._ + + if (!IsCaseClassImpl.isCaseClassType(c)(T.tpe)) + c.abort(c.enclosingPosition, s"""We cannot enforce ${T.tpe} is a case class, either it is not a case class or this macro call is possibly enclosed in a class. + This will mean the macro is operating on a non-resolved type.""") + + def matchField(fieldType: Type, fieldName: String, isOption: Boolean): List[Tree] = { + val REPETITION_REQUIRED = q"_root_.parquet.schema.Type.Repetition.REQUIRED" + val REPETITION_OPTIONAL = q"_root_.parquet.schema.Type.Repetition.OPTIONAL" + + def repetition: Tree = if (isOption) REPETITION_OPTIONAL else REPETITION_REQUIRED + + def createPrimitiveTypeField(primitiveType: Tree): List[Tree] = + List(q"""new _root_.parquet.schema.PrimitiveType($repetition, $primitiveType, $fieldName)""") + + fieldType match { + case tpe if tpe =:= typeOf[String] => + createPrimitiveTypeField(q"_root_.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY") + case tpe if tpe =:= typeOf[Boolean] => + createPrimitiveTypeField(q"_root_.parquet.schema.PrimitiveType.PrimitiveTypeName.BOOLEAN") + case tpe if tpe =:= typeOf[Short] || tpe =:= typeOf[Int] => + createPrimitiveTypeField(q"_root_.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32") + case tpe if tpe =:= typeOf[Long] => + createPrimitiveTypeField(q"_root_.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64") + case tpe if tpe =:= typeOf[Float] => + createPrimitiveTypeField(q"_root_.parquet.schema.PrimitiveType.PrimitiveTypeName.FLOAT") + case tpe if tpe =:= typeOf[Double] => + createPrimitiveTypeField(q"_root_.parquet.schema.PrimitiveType.PrimitiveTypeName.DOUBLE") + case tpe if tpe.erasure =:= typeOf[Option[Any]] => + val innerType = tpe.asInstanceOf[TypeRefApi].args.head + matchField(innerType, fieldName, isOption = true) + case tpe if IsCaseClassImpl.isCaseClassType(c)(tpe) => + List(q"""new _root_.parquet.schema.GroupType($repetition, $fieldName, + _root_.scala.Array.apply[_root_.parquet.schema.Type](..${expandMethod(tpe)}):_*)""") + case _ => c.abort(c.enclosingPosition, s"Case class $T is not pure primitives or nested case classes") + } + } + + def expandMethod(outerTpe: Type): List[Tree] = { + outerTpe + .declarations + .collect { case m: MethodSymbol if m.isCaseAccessor => m } + .flatMap { accessorMethod => + val fieldName = accessorMethod.name.toTermName.toString + val fieldType = accessorMethod.returnType + matchField(fieldType, fieldName, isOption = false) + }.toList + } + + val expanded = expandMethod(T.tpe) + + if (expanded.isEmpty) + c.abort(c.enclosingPosition, s"Case class $T.tpe has no primitive types we were able to extract") + + val messageTypeName = s"${T.tpe}".split("\\.").last + val schema = q"""new _root_.parquet.schema.MessageType($messageTypeName, + _root_.scala.Array.apply[_root_.parquet.schema.Type](..$expanded):_*).toString""" + + c.Expr[String](schema) + } +} diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/ParquetTupleConverterProvider.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/ParquetTupleConverterProvider.scala new file mode 100644 index 0000000000..90462036c1 --- /dev/null +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/ParquetTupleConverterProvider.scala @@ -0,0 +1,89 @@ +package com.twitter.scalding.parquet.tuple.macros.impl + +import com.twitter.bijection.macros.impl.IsCaseClassImpl +import com.twitter.scalding.parquet.tuple.scheme.ParquetTupleConverter + +import scala.reflect.macros.Context + +object ParquetTupleConverterProvider { + def toParquetTupleConverterImpl[T](ctx: Context)(implicit T: ctx.WeakTypeTag[T]): ctx.Expr[ParquetTupleConverter] = { + import ctx.universe._ + + if (!IsCaseClassImpl.isCaseClassType(ctx)(T.tpe)) + ctx.abort(ctx.enclosingPosition, + s"""We cannot enforce ${T.tpe} is a case class, + either it is not a case class or this macro call is possibly enclosed in a class. + This will mean the macro is operating on a non-resolved type.""") + + def buildGroupConverter(tpe: Type, parentTree: Tree, isOption: Boolean, idx: Int, converterBodyTree: Tree): Tree = { + q"""new _root_.com.twitter.scalding.parquet.tuple.scheme.ParquetTupleConverter($parentTree, $isOption, $idx){ + override def newConverter(i: Int): _root_.parquet.io.api.Converter = { + $converterBodyTree + throw new RuntimeException("invalid index: " + i) + } + + override def createValue(): Any = { + if(fieldValues.isEmpty) null + else classOf[$tpe].getConstructors()(0).newInstance(fieldValues.toSeq.map(_.asInstanceOf[AnyRef]): _*) + } + }""" + } + + def matchField(idx: Int, fieldType: Type, isOption: Boolean): List[Tree] = { + + def createConverter(converter: Tree): Tree = q"if($idx == i) return $converter" + + fieldType match { + case tpe if tpe =:= typeOf[String] => + List(createConverter(q"new _root_.com.twitter.scalding.parquet.tuple.scheme.StringConverter($idx, this, $isOption)")) + case tpe if tpe =:= typeOf[Boolean] => + List(createConverter(q"new _root_.com.twitter.scalding.parquet.tuple.scheme.BooleanConverter($idx, this, $isOption)")) + case tpe if tpe =:= typeOf[Short] => + List(createConverter(q"new _root_.com.twitter.scalding.parquet.tuple.scheme.ShortConverter($idx, this, $isOption)")) + case tpe if tpe =:= typeOf[Int] => + List(createConverter(q"new _root_.com.twitter.scalding.parquet.tuple.scheme.IntConverter($idx, this, $isOption)")) + case tpe if tpe =:= typeOf[Long] => + List(createConverter(q"new _root_.com.twitter.scalding.parquet.tuple.scheme.LongConverter($idx, this, $isOption)")) + case tpe if tpe =:= typeOf[Float] => + List(createConverter(q"new _root_.com.twitter.scalding.parquet.tuple.scheme.FloatConverter($idx, this, $isOption)")) + case tpe if tpe =:= typeOf[Double] => + List(createConverter(q"new _root_.com.twitter.scalding.parquet.tuple.scheme.DoubleConverter($idx, this, $isOption)")) + case tpe if tpe.erasure =:= typeOf[Option[Any]] => + val innerType = tpe.asInstanceOf[TypeRefApi].args.head + matchField(idx, innerType, isOption = true) + case tpe if IsCaseClassImpl.isCaseClassType(ctx)(tpe) => + val innerConverterTrees = buildConverterBody(tpe, expandMethod(tpe)) + List(createConverter(buildGroupConverter(tpe, q"Option(this)", isOption, idx, innerConverterTrees))) + case _ => ctx.abort(ctx.enclosingPosition, s"Case class $T is not pure primitives or nested case classes") + } + } + + def expandMethod(outerTpe: Type): List[Tree] = { + outerTpe + .declarations + .collect { case m: MethodSymbol if m.isCaseAccessor => m } + .zipWithIndex + .flatMap { + case (accessorMethod, idx) => + val fieldType = accessorMethod.returnType + matchField(idx, fieldType, isOption = false) + }.toList + } + + def buildConverterBody(tpe: Type, trees: List[Tree]): Tree = { + if (trees.isEmpty) + ctx.abort(ctx.enclosingPosition, s"Case class $tpe has no primitive types we were able to extract") + trees.foldLeft(q"") { + case (existingTree, t) => + q"""$existingTree + $t""" + } + } + + val groupConverter = buildGroupConverter(T.tpe, q"None", isOption = false, -1, buildConverterBody(T.tpe, expandMethod(T.tpe))) + + ctx.Expr[ParquetTupleConverter](q""" + $groupConverter + """) + } +} diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/SchemaProviderImpl.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/SchemaProviderImpl.scala deleted file mode 100644 index b6d9c754f8..0000000000 --- a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/SchemaProviderImpl.scala +++ /dev/null @@ -1,84 +0,0 @@ -package com.twitter.scalding.parquet.tuple.macros.impl - -import scala.language.experimental.macros -import com.twitter.bijection.macros.impl.IsCaseClassImpl -import parquet.schema.MessageType - -import scala.reflect.macros.Context - -object SchemaProviderImpl { - - def toParquetSchemaImp[T](c: Context)(implicit T: c.WeakTypeTag[T]): c.Expr[MessageType] = { - - import c.universe._ - - if (!IsCaseClassImpl.isCaseClassType(c)(T.tpe)) - c.abort(c.enclosingPosition, s"""We cannot enforce ${T.tpe} is a case class, either it is not a case class or this macro call is possibly enclosed in a class. - This will mean the macro is operating on a non-resolved type.""") - - case class Extractor(tpe: Type, toTree: Tree) - case class Builder(toTree: Tree = q"") - - implicit val builderLiftable = new Liftable[Builder] { - def apply(b: Builder): Tree = b.toTree - } - - implicit val extractorLiftable = new Liftable[Extractor] { - def apply(b: Extractor): Tree = b.toTree - } - - val REPETITION_REQUIRED = q"ParquetType.Repetition.REQUIRED" - val REPETITION_OPTIONAL = q"ParquetType.Repetition.OPTIONAL" - - def createPrimitiveTypeField(isOption: Boolean, primitiveType: Tree, fieldName: String): List[Tree] = - List(q"""new PrimitiveType(${getRepetition(isOption)}, $primitiveType, $fieldName)""") - - def getRepetition(isOption: Boolean): Tree = if (isOption) REPETITION_OPTIONAL else REPETITION_REQUIRED - - def matchField(fieldType: Type, outerName: String, fieldName: String, isOption: Boolean): List[Tree] = { - val parquetFieldName = s"$outerName$fieldName" - fieldType match { - case tpe if tpe =:= typeOf[String] => - createPrimitiveTypeField(isOption, q"PrimitiveType.PrimitiveTypeName.BINARY", parquetFieldName) - case tpe if tpe =:= typeOf[Boolean] => - createPrimitiveTypeField(isOption, q"PrimitiveType.PrimitiveTypeName.BOOLEAN", parquetFieldName) - case tpe if tpe =:= typeOf[Short] || tpe =:= typeOf[Int] => - createPrimitiveTypeField(isOption, q"PrimitiveType.PrimitiveTypeName.INT32", parquetFieldName) - case tpe if tpe =:= typeOf[Long] => - createPrimitiveTypeField(isOption, q"PrimitiveType.PrimitiveTypeName.INT64", parquetFieldName) - case tpe if tpe =:= typeOf[Float] => - createPrimitiveTypeField(isOption, q"PrimitiveType.PrimitiveTypeName.FLOAT", parquetFieldName) - case tpe if tpe =:= typeOf[Double] => - createPrimitiveTypeField(isOption, q"PrimitiveType.PrimitiveTypeName.DOUBLE", parquetFieldName) - case tpe if tpe.erasure =:= typeOf[Option[Any]] && isOption => - c.abort(c.enclosingPosition, s"Nested options do not make sense being mapped onto a tuple fields in cascading.") - case tpe if tpe.erasure =:= typeOf[Option[Any]] => - val innerType = tpe.asInstanceOf[TypeRefApi].args.head - matchField(innerType, outerName, fieldName, true) - case tpe if IsCaseClassImpl.isCaseClassType(c)(tpe) => expandMethod(tpe, s"$parquetFieldName.", isOption = false) - case _ => c.abort(c.enclosingPosition, s"Case class $T is not pure primitives or nested case classes") - } - } - - def expandMethod(outerTpe: Type, outerName: String, isOption: Boolean): List[Tree] = { - outerTpe - .declarations - .collect { case m: MethodSymbol if m.isCaseAccessor => m } - .flatMap { accessorMethod => - val fieldName = accessorMethod.name.toTermName.toString - val fieldType = accessorMethod.returnType - matchField(fieldType, outerName, fieldName, false) - }.toList - } - - def expandCaseClass(outerTpe: Type, outerName: String, isOption: Boolean): Tree = { - val expanded = expandMethod(outerTpe, outerName, isOption) - if (expanded.isEmpty) c.abort(c.enclosingPosition, s"Case class $outerTpe has no primitive types we were able to extract") - val messageTypeName = s"${outerTpe}".split("\\.").last - q"""import parquet.schema.{MessageType, PrimitiveType, Type => ParquetType} - new MessageType($messageTypeName, Array.apply[ParquetType](..$expanded):_*)""" - } - - c.Expr[MessageType](expandCaseClass(T.tpe, "", isOption = false)) - } -} diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/ParquetTupleConverter.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/ParquetTupleConverter.scala new file mode 100644 index 0000000000..602c83ff79 --- /dev/null +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/ParquetTupleConverter.scala @@ -0,0 +1,89 @@ +package com.twitter.scalding.parquet.tuple.scheme + +import parquet.io.api.{ Binary, Converter, GroupConverter, PrimitiveConverter } + +import scala.collection.mutable +import scala.util.Try + +/** + * Parquet tuple converter used to create user defined tuple value from parquet column values + * @param parent parent parquet tuple converter + * @param isOption is the field optional + * @param outerIndex field index in parent tuple schema + */ +abstract class ParquetTupleConverter(val parent: Option[ParquetTupleConverter] = None, val isOption: Boolean = false, + val outerIndex: Int = -1) extends GroupConverter { + var converters: Map[Int, Converter] = Map() + val fieldValues: mutable.ArrayBuffer[Any] = mutable.ArrayBuffer() + + var currentValue: Any = null + + def createValue(): Any + + def newConverter(i: Int): Converter + + override def getConverter(i: Int) = { + val converter = converters.get(i) + if (converter.isDefined) converter.get + else { + val c = newConverter(i) + converters += i -> c + c + } + } + + override def end(): Unit = { + currentValue = createValue() + fieldValues.remove(0, fieldValues.size) + parent.map(_.addFieldValue(outerIndex, currentValue, isOption)) + } + + override def start(): Unit = () + + def addFieldValue(index: Int, value: Any, isOpt: Boolean) = { + val currentSize = fieldValues.size + //insert none for these optional fields that has non value written for given row + (currentSize until index).map(fieldValues.insert(_, None)) + if (isOpt) fieldValues.insert(index, Option(value)) else fieldValues.insert(index, value) + } +} + +class PrimitiveTypeConverter(val index: Int, val parent: ParquetTupleConverter, val isOption: Boolean) + extends PrimitiveConverter { + def appendValue(value: Any) = parent.addFieldValue(index, value, isOption) +} + +class StringConverter(index: Int, parent: ParquetTupleConverter, isOption: Boolean = false) + extends PrimitiveTypeConverter(index, parent, isOption) { + override def addBinary(value: Binary): Unit = appendValue(value.toStringUsingUTF8) +} + +class DoubleConverter(index: Int, parent: ParquetTupleConverter, isOption: Boolean = false) + extends PrimitiveTypeConverter(index, parent, isOption) { + override def addDouble(value: Double): Unit = appendValue(value) +} + +class FloatConverter(index: Int, parent: ParquetTupleConverter, isOption: Boolean = false) + extends PrimitiveTypeConverter(index, parent, isOption) { + override def addFloat(value: Float): Unit = appendValue(value) +} + +class LongConverter(index: Int, parent: ParquetTupleConverter, isOption: Boolean = false) + extends PrimitiveTypeConverter(index, parent, isOption) { + override def addLong(value: Long) = appendValue(value) +} + +class IntConverter(index: Int, parent: ParquetTupleConverter, isOption: Boolean = false) + extends PrimitiveTypeConverter(index, parent, isOption) { + override def addInt(value: Int) = appendValue(value) +} + +class ShortConverter(index: Int, parent: ParquetTupleConverter, isOption: Boolean = false) + extends PrimitiveTypeConverter(index, parent, isOption) { + override def addInt(value: Int) = appendValue(Try(value.toShort).getOrElse(null)) +} + +class BooleanConverter(index: Int, parent: ParquetTupleConverter, isOption: Boolean = false) + extends PrimitiveTypeConverter(index, parent, isOption) { + override def addBoolean(value: Boolean) = appendValue(value) +} \ No newline at end of file diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/TypedParquetTupleScheme.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/TypedParquetTupleScheme.scala new file mode 100644 index 0000000000..60b5c665c2 --- /dev/null +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/TypedParquetTupleScheme.scala @@ -0,0 +1,205 @@ +package com.twitter.scalding.parquet.tuple.scheme + +import java.util.{ HashMap => JHashMap, List => JList, Map => JMap } + +import _root_.parquet.filter2.predicate.FilterPredicate +import _root_.parquet.hadoop.api.ReadSupport.ReadContext +import _root_.parquet.hadoop.api.WriteSupport.WriteContext +import _root_.parquet.hadoop.api.{ ReadSupport, WriteSupport } +import _root_.parquet.hadoop.mapred.{ Container, DeprecatedParquetInputFormat, DeprecatedParquetOutputFormat } +import _root_.parquet.io.api._ +import cascading.flow.FlowProcess +import cascading.scheme.{ Scheme, SinkCall, SourceCall } +import cascading.tap.Tap +import cascading.tuple.Tuple +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.mapred.{ JobConf, OutputCollector, RecordReader } +import parquet.hadoop.{ ParquetInputFormat, ParquetOutputFormat } +import parquet.schema._ + +/** + * Parquet tuple materializer permits to create user defined type record from parquet tuple values + * @param converter root converter + * @tparam T User defined value type + */ +class ParquetTupleMaterializer[T](val converter: ParquetTupleConverter) extends RecordMaterializer[T] { + override def getCurrentRecord: T = converter.currentValue.asInstanceOf[T] + + override def getRootConverter: GroupConverter = converter +} + +/** + * Parquet read support used by [[parquet.hadoop.ParquetInputFormat]] to read values from parquet input. + * User must define record schema and parquet tuple converter that permits to convert parquet tuple to user defined type + * For case class types, we provide a macro to generate the schema and the tuple converter so that user + * can define a ParquetReadSupport like this: + * + * case class SampleClass(bool: Boolean, long: Long, float: Float) + * + * class SampleClassReadSupport extends ParquetReadSupport[SampleClass] { + * import com.twitter.scalding.parquet.tuple.macros.Macros._ + * override val tupleConverter: ParquetTupleConverter = caseClassParquetTupleConverter[SampleClass] + * override val rootSchema: String = caseClassParquetSchema[SampleClass] + * } + * + * @tparam T user defined value type + */ +trait ParquetReadSupport[T] extends ReadSupport[T] { + val tupleConverter: ParquetTupleConverter + val rootSchema: String + + lazy val rootType: MessageType = MessageTypeParser.parseMessageType(rootSchema) + + override def init(configuration: Configuration, map: JMap[String, String], messageType: MessageType): ReadContext = + new ReadContext(rootType) + + override def prepareForRead(configuration: Configuration, map: JMap[String, String], messageType: MessageType, + readContext: ReadContext): RecordMaterializer[T] = + new ParquetTupleMaterializer(tupleConverter) +} + +/** + * Parquet write support used by [[parquet.hadoop.ParquetOutputFormat]] to write values to parquet output. + * User must provide record schema and a function which permits to flat(at every level) a record to a index-value map. + * For example if we get: + * + * case class SampleClassA(short: Short, int: Int) + * case class SampleClassB(bool: Boolean, a: SampleClassA, long: Long, float: Float) + * + * val b = SampleClassB(true, SampleClassA(1, 4), 6L, 5F) + * + * After flatting using the function , we should get a map like this: + * + * Map(0 -> true, 1 -> 1, 2 -> 4, 3 -> 6L, 4 -> 5F) + * + * For case class value types, we provide a macro to generate the field values function so that user + * can define a ParquetWriteSupport like this: + * + * class SampleClassWriteSupport extends TupleWriteSupport[SampleClassB] { + * import com.twitter.scalding.parquet.tuple.macros.Macros._ + * override val fieldValues: (SampleClassB) => Map[Int, Any] = caseClassFieldValues[SampleClassB] + * override val rootSchema: String = caseClassParquetSchema[SampleClassB] + * } + * + * @tparam T user defined value type + */ +trait ParquetWriteSupport[T] extends WriteSupport[T] { + + //function which permits to flat(at every level) a record to a index-value map. + val fieldValues: T => Map[Int, Any] + + var recordConsumer: RecordConsumer = null + + val rootSchema: String + + lazy val rootType: MessageType = MessageTypeParser.parseMessageType(rootSchema) + + override def init(configuration: Configuration): WriteContext = + new WriteSupport.WriteContext(rootType, new JHashMap[String, String]) + + override def prepareForWrite(rc: RecordConsumer): Unit = recordConsumer = rc + + override def write(record: T): Unit = { + val valuesMap = fieldValues(record) + recordConsumer.startMessage() + if (record != null) writeGroupType(0, valuesMap, rootType) + recordConsumer.endMessage() + } + + private def writeGroupType(outerFieldIdx: Int, valuesMap: Map[Int, Any], groupType: GroupType): Unit = { + val fields: JList[Type] = groupType.getFields + (0 until fields.size).map { i => + val field = fields.get(i) + val valueFieldIndex = outerFieldIdx + i + valuesMap.get(valueFieldIndex).map { v => + recordConsumer.startField(field.getName, i) + if (field.isPrimitive) + writePrimitiveType(v, field.asPrimitiveType()) + else { + recordConsumer.startGroup() + writeGroupType(valueFieldIndex, valuesMap, field.asGroupType()) + recordConsumer.endGroup() + } + recordConsumer.endField(field.getName, i) + } + } + } + + private def writePrimitiveType(value: Any, field: PrimitiveType) = { + field.getPrimitiveTypeName match { + case PrimitiveType.PrimitiveTypeName.BINARY => + recordConsumer.addBinary(Binary.fromString(value.asInstanceOf[String])) + case PrimitiveType.PrimitiveTypeName.BOOLEAN => + recordConsumer.addBoolean(value.asInstanceOf[Boolean]) + case PrimitiveType.PrimitiveTypeName.INT32 => + value match { + case i: Int => + recordConsumer.addInteger(i) + case s: Short => + recordConsumer.addInteger(s.toInt) + case _ => + throw new UnsupportedOperationException(field.getName + " write to int not supported") + } + case PrimitiveType.PrimitiveTypeName.INT64 => + recordConsumer.addLong(value.asInstanceOf[Long]) + case PrimitiveType.PrimitiveTypeName.DOUBLE => + recordConsumer.addDouble(value.asInstanceOf[Double]) + case PrimitiveType.PrimitiveTypeName.FLOAT => + recordConsumer.addFloat(value.asInstanceOf[Float]) + case _ => + throw new UnsupportedOperationException(field.getName + " type not implemented") + } + } +} + +/** + * Typed parquet tuple scheme. + * @param readSupport read support class + * @param writeSupport write support class + * @param fp filter predicate + * @tparam T tuple value type + */ +class TypedParquetTupleScheme[T](val readSupport: Class[_], val writeSupport: Class[_], + val fp: Option[FilterPredicate] = None) + extends Scheme[JobConf, RecordReader[AnyRef, Container[T]], OutputCollector[AnyRef, T], Array[AnyRef], Array[AnyRef]] { + + type Output = OutputCollector[AnyRef, T] + type Reader = RecordReader[AnyRef, Container[T]] + type TapType = Tap[JobConf, Reader, Output] + type SourceCallType = SourceCall[Array[AnyRef], Reader] + type SinkCallType = SinkCall[Array[AnyRef], Output] + + override def sourceConfInit(flowProcess: FlowProcess[JobConf], tap: TapType, jobConf: JobConf): Unit = { + fp.map(ParquetInputFormat.setFilterPredicate(jobConf, _)) + jobConf.setInputFormat(classOf[DeprecatedParquetInputFormat[T]]) + ParquetInputFormat.setReadSupportClass(jobConf, readSupport) + } + + override def source(flowProcess: FlowProcess[JobConf], sc: SourceCallType): Boolean = { + val value: Container[T] = sc.getInput.createValue() + + val hasNext = sc.getInput.next(null, value) + + if (!hasNext) false + else if (value == null) true + else { + val tuple = new Tuple(value.get.asInstanceOf[AnyRef]) + sc.getIncomingEntry.setTuple(tuple) + true + } + } + + override def sinkConfInit(flowProcess: FlowProcess[JobConf], tap: TapType, jobConf: JobConf): Unit = { + jobConf.setOutputFormat(classOf[DeprecatedParquetOutputFormat[T]]) + ParquetOutputFormat.setWriteSupportClass(jobConf, writeSupport) + } + + override def sink(flowProcess: FlowProcess[JobConf], sinkCall: SinkCallType): Unit = { + val tuple = sinkCall.getOutgoingEntry + require(tuple.size == 1, + "TypedParquetTupleScheme expects tuple with an arity of exactly 1, but found " + tuple.getFields) + val value = tuple.getObject(0).asInstanceOf[T] + val outputCollector = sinkCall.getOutput + outputCollector.collect(null, value) + } +} diff --git a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala index 57803540f5..24eee042fa 100644 --- a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala +++ b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala @@ -2,19 +2,21 @@ package com.twitter.scalding.parquet.tuple import java.io.File -import cascading.tuple.{ Fields, TupleEntry, Tuple } +import com.twitter.scalding.parquet.tuple.macros.Macros +import com.twitter.scalding.parquet.tuple.scheme.{ ParquetTupleConverter, ParquetReadSupport, ParquetWriteSupport } import com.twitter.scalding.platform.{ HadoopPlatformJobTest, HadoopPlatformTest } import com.twitter.scalding.typed.TypedPipe -import com.twitter.scalding.{ TupleSetter, TupleConverter, Job, Args, TypedTsv } -import parquet.schema.MessageType - +import com.twitter.scalding.{ Args, Job, TypedTsv } import org.scalatest.{ Matchers, WordSpec } +import parquet.filter2.predicate.FilterApi.binaryColumn +import parquet.filter2.predicate.{FilterApi, FilterPredicate} +import parquet.io.api.Binary class TypedParquetTupleTest extends WordSpec with Matchers with HadoopPlatformTest { "TypedParquetTuple" should { "read and write correctly" in { - import TestValues._ + import com.twitter.scalding.parquet.tuple.TestValues._ val tempParquet = java.nio.file.Files.createTempDirectory("parquet_tuple_test_parquet_").toAbsolutePath.toString try { HadoopPlatformJobTest(new WriteToTypedParquetTupleJob(_), cluster) @@ -23,8 +25,14 @@ class TypedParquetTupleTest extends WordSpec with Matchers with HadoopPlatformTe HadoopPlatformJobTest(new ReadFromTypedParquetTupleJob(_), cluster) .arg("input", tempParquet) - .sink[String]("output") { _.toSet shouldBe values.map(_.stringValue).toSet } + .sink[Float]("output") { _.toSet shouldBe values.map(_.a.float).toSet } .run + + HadoopPlatformJobTest(new ReadWithFilterPredicateJob(_), cluster) + .arg("input", tempParquet) + .sink[Int]("output") { _.toSet shouldBe values.filter(_.string == "B1").map(_.int).toSet } + .run + } finally { deletePath(tempParquet) } @@ -42,69 +50,65 @@ class TypedParquetTupleTest extends WordSpec with Matchers with HadoopPlatformTe } object TestValues { - val values = Seq(SampleClass("A", 1, 4.0D), SampleClass("B", 2, 3.0D), SampleClass("A", 2, 5.0D)) + val values = Seq(SampleClassB("B1", 1, Some(4.0D), SampleClassA(bool = true, 5, 1L, 1.2F)), + SampleClassB("B2", 3, Some(3.0D), SampleClassA(bool = false, 4, 2L, 2.3F)), + SampleClassB("B3", 9, None, SampleClassA(bool = true, 6, 3L, 3.4F)), + SampleClassB("B4", 8, Some(5.0D), SampleClassA(bool = false, 7, 4L, 4.5F))) +} + +case class SampleClassA(bool: Boolean, short: Short, long: Long, float: Float) +case class SampleClassB(string: String, int: Int, double: Option[Double], a: SampleClassA) + +object SampleClassB { + val schema: String = Macros.caseClassParquetSchema[SampleClassB] +} + +class ReadSupport extends ParquetReadSupport[SampleClassB] { + override val tupleConverter: ParquetTupleConverter = Macros.caseClassParquetTupleConverter[SampleClassB] + override val rootSchema: String = SampleClassB.schema } -case class SampleClass(stringValue: String, intValue: Int, doubleValue: Double) +class WriteSupport extends ParquetWriteSupport[SampleClassB] { + override val fieldValues: (SampleClassB) => Map[Int, Any] = Macros.caseClassFieldValues[SampleClassB] + override val rootSchema: String = SampleClassB.schema +} /** * Test job write a sequence of sample class values into a typed parquet tuple. * To test typed parquet tuple can be used as sink */ class WriteToTypedParquetTupleJob(args: Args) extends Job(args) { + import com.twitter.scalding.parquet.tuple.TestValues._ - import SampleClassDescriptor._ - import TestValues._ val outputPath = args.required("output") - val parquetTuple = TypedParquetTuple[SampleClass](Seq(outputPath)) - TypedPipe.from(values).write(parquetTuple) + val sink = TypedParquetSink[SampleClassB, WriteSupport](Seq(outputPath)) + TypedPipe.from(values).write(sink) } /** - * Test job read from a typed parquet tuple and write the mapped value into a typed csv sink + * Test job read from a typed parquet tuple and write the mapped value into a typed tsv sink * To test typed parquet tuple can bse used as source and read data correctly */ class ReadFromTypedParquetTupleJob(args: Args) extends Job(args) { - import SampleClassDescriptor._ - val inputPath = args.required("input") - val parquetTuple = TypedParquetTuple[SampleClass](Seq(inputPath)) + val input = TypedParquet[SampleClassB, ReadSupport](Seq(inputPath)) - TypedPipe.from(parquetTuple).map(_.stringValue).write(TypedTsv[String]("output")) + TypedPipe.from(input).map(_.a.float).write(TypedTsv[Float]("output")) } /** - * Helper class with tuple related setter and converter + - * parquet schema using parquet schema generation macro + * Test job read from a typed parquet source with filter predicate and write the mapped value into a typed tsv sink + * To test typed parquet tuple can bse used as source and apply filter predicate correctly */ -object SampleClassDescriptor { - import com.twitter.scalding.parquet.tuple.macros.Macros._ - implicit val valueTupleSetter: TupleSetter[SampleClass] = new TupleSetter[SampleClass] { - override def apply(value: SampleClass): Tuple = { - val tuple = new Tuple() - tuple.addString(value.stringValue) - tuple.addInteger(value.intValue) - tuple.addDouble(value.doubleValue) - tuple - } +class ReadWithFilterPredicateJob(args: Args) extends Job(args) { + val fp: FilterPredicate = FilterApi.eq(binaryColumn("string"), Binary.fromString("B1")) - override def arity: Int = 3 - } - - implicit val valueTupleConverter: TupleConverter[SampleClass] = new TupleConverter[SampleClass] { - override def apply(te: TupleEntry): SampleClass = { - val stringValue = te.getString("stringValue") - val intValue = te.getInteger("intValue") - val doubleValue = te.getDouble("doubleValue") - SampleClass(stringValue, intValue, doubleValue) - } + val inputPath = args.required("input") - override def arity: Int = 3 - } + val input = TypedParquet[SampleClassB, ReadSupport](Seq(inputPath), Some(fp)) - implicit val sampleClassFields: Fields = new Fields("stringValue", "intValue", "doubleValue") - implicit val sampleClassParquetSchema: MessageType = caseClassParquetSchema[SampleClass] + TypedPipe.from(input).map(_.int).write(TypedTsv[Int]("output")) } diff --git a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/macros/MacroUnitTests.scala b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/macros/MacroUnitTests.scala index ed28ae96f2..020b7c3a6d 100644 --- a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/macros/MacroUnitTests.scala +++ b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/macros/MacroUnitTests.scala @@ -1,20 +1,29 @@ package com.twitter.scalding.parquet.tuple.macros -import org.scalatest.{ WordSpec, Matchers } +import org.scalatest.{ Matchers, WordSpec } +import parquet.io.api.Binary import parquet.schema.MessageTypeParser case class SampleClassA(x: Int, y: String) -case class SampleClassB(a1: SampleClassA, a2: SampleClassA, y: String) -case class SampleClassC(a: SampleClassA, b: SampleClassB, c: SampleClassA, d: SampleClassB, e: SampleClassB) + +case class SampleClassB(a: SampleClassA, y: String) + +case class SampleClassC(a: SampleClassA, b: SampleClassB) + case class SampleClassD(a: String, b: Boolean, c: Option[Short], d: Int, e: Long, f: Float, g: Option[Double]) + case class SampleClassE(a: Int, b: Long, c: Short, d: Boolean, e: Float, f: Double, g: String) +case class SampleClassF(a: Option[SampleClassA]) + +case class SampleClassG(a: Int, b: Option[SampleClassF], c: Double) + class MacroUnitTests extends WordSpec with Matchers { - "MacroGenerated Parquet schema generator" should { + "Macro generated case class parquet schema generator" should { "Generate parquet schema for SampleClassA" in { - val schema = Macros.caseClassParquetSchema[SampleClassA] + val schema = MessageTypeParser.parseMessageType(Macros.caseClassParquetSchema[SampleClassA]) val expectedSchema = MessageTypeParser.parseMessageType(""" |message SampleClassA { | required int32 x; @@ -25,14 +34,14 @@ class MacroUnitTests extends WordSpec with Matchers { } "Generate parquet schema for SampleClassB" in { - val schema = Macros.caseClassParquetSchema[SampleClassB] + val schema = MessageTypeParser.parseMessageType(Macros.caseClassParquetSchema[SampleClassB]) val expectedSchema = MessageTypeParser.parseMessageType(""" |message SampleClassB { - | required int32 a1.x; - | required binary a1.y; - | required int32 a2.x; - | required binary a2.y; + | required group a { + | required int32 x; + | required binary y; + | } | required binary y; |} """.stripMargin) @@ -40,36 +49,28 @@ class MacroUnitTests extends WordSpec with Matchers { } "Generate parquet schema for SampleClassC" in { - val schema = Macros.caseClassParquetSchema[SampleClassC] + val schema = MessageTypeParser.parseMessageType(Macros.caseClassParquetSchema[SampleClassC]) val expectedSchema = MessageTypeParser.parseMessageType(""" |message SampleClassC { - | required int32 a.x; - | required binary a.y; - | required int32 b.a1.x; - | required binary b.a1.y; - | required int32 b.a2.x; - | required binary b.a2.y; - | required binary b.y; - | required int32 c.x; - | required binary c.y; - | required int32 d.a1.x; - | required binary d.a1.y; - | required int32 d.a2.x; - | required binary d.a2.y; - | required binary d.y; - | required int32 e.a1.x; - | required binary e.a1.y; - | required int32 e.a2.x; - | required binary e.a2.y; - | required binary e.y; + | required group a { + | required int32 x; + | required binary y; + | } + | required group b { + | required group a { + | required int32 x; + | required binary y; + | } + | required binary y; + | } |} """.stripMargin) schema shouldEqual expectedSchema } "Generate parquet schema for SampleClassD" in { - val schema = Macros.caseClassParquetSchema[SampleClassD] + val schema = MessageTypeParser.parseMessageType(Macros.caseClassParquetSchema[SampleClassD]) val expectedSchema = MessageTypeParser.parseMessageType(""" |message SampleClassD { | required binary a; @@ -85,7 +86,7 @@ class MacroUnitTests extends WordSpec with Matchers { } "Generate parquet schema for SampleClassE" in { - val schema = Macros.caseClassParquetSchema[SampleClassE] + val schema = MessageTypeParser.parseMessageType(Macros.caseClassParquetSchema[SampleClassE]) val expectedSchema = MessageTypeParser.parseMessageType(""" |message SampleClassE { | required int32 a; @@ -99,5 +100,144 @@ class MacroUnitTests extends WordSpec with Matchers { """.stripMargin) schema shouldEqual expectedSchema } + + } + + "Macro generated case class field values generator" should { + "Generate field values for SampleClassA" in { + val a = SampleClassA(1, "foo") + val values = Macros.caseClassFieldValues[SampleClassA] + values(a) shouldEqual Map(0 -> 1, 1 -> "foo") + } + + "Generate field values for SampleClassB with nested case class" in { + val a = SampleClassA(1, "foo") + val b = SampleClassB(a, "b") + val values = Macros.caseClassFieldValues[SampleClassB] + + values(b) shouldEqual Map(0 -> 1, 1 -> "foo", 2 -> "b") + } + + "Generate field values for SampleClassC with two nested case classes" in { + + val a = SampleClassA(1, "foo") + + val b = SampleClassB(a, "b") + + val c = SampleClassC(a, b) + + val values = Macros.caseClassFieldValues[SampleClassC] + + values(c) shouldEqual Map(0 -> 1, 1 -> "foo", 2 -> 1, 3 -> "foo", 4 -> "b") + } + + "Generate field values for SampleClassD with option values" in { + val d = SampleClassD("toto", b = true, Some(2), 1, 2L, 3F, Some(5D)) + val values = Macros.caseClassFieldValues[SampleClassD] + values(d) shouldEqual Map(0 -> "toto", 1 -> true, 2 -> 2, 3 -> 1, 4 -> 2L, 5 -> 3F, 6 -> 5D) + + val d2 = SampleClassD("toto", b = true, None, 1, 2L, 3F, None) + val values2 = Macros.caseClassFieldValues[SampleClassD] + values2(d2) shouldEqual Map(0 -> "toto", 1 -> true, 3 -> 1, 4 -> 2L, 5 -> 3F) + } + + "Generate field values for SampleClassF with optional nested case class " in { + val a = SampleClassA(1, "foo") + val f1 = SampleClassF(Some(a)) + val values1 = Macros.caseClassFieldValues[SampleClassF] + values1(f1) shouldEqual Map(0 -> 1, 1 -> "foo") + + val f2 = SampleClassF(None) + val values2 = Macros.caseClassFieldValues[SampleClassF] + values2(f2) shouldEqual Map.empty + } + + "Generate field values for SampleClassG with nested case class containing optional fields" in { + val a = SampleClassA(1, "foo") + val f1 = SampleClassF(Some(a)) + val g1 = SampleClassG(0, Some(f1), 1D) + val values1 = Macros.caseClassFieldValues[SampleClassG] + values1(g1) shouldEqual Map(0 -> 0, 1 -> 1, 2 -> "foo", 3 -> 1D) + + val f2 = SampleClassF(None) + val g2 = SampleClassG(1, Some(f2), 2D) + val values2 = Macros.caseClassFieldValues[SampleClassG] + values2(g2) shouldEqual Map(0 -> 1, 3 -> 2D) + + val g3 = SampleClassG(1, None, 3D) + val values3 = Macros.caseClassFieldValues[SampleClassG] + values3(g3) shouldEqual Map(0 -> 1, 3 -> 3D) + } + } + + "Macro generated case class converters generator" should { + + "Generate converters for all primitive types" in { + val converter = Macros.caseClassParquetTupleConverter[SampleClassE] + + val intConverter = converter.getConverter(0).asPrimitiveConverter() + intConverter.addInt(0) + + val longConverter = converter.getConverter(1).asPrimitiveConverter() + longConverter.addLong(1L) + + val shortConverter = converter.getConverter(2).asPrimitiveConverter() + shortConverter.addInt(2) + + val boolean = converter.getConverter(3).asPrimitiveConverter() + boolean.addBoolean(true) + + val float = converter.getConverter(4).asPrimitiveConverter() + float.addFloat(3F) + + val double = converter.getConverter(5).asPrimitiveConverter() + double.addDouble(4D) + + val string = converter.getConverter(6).asPrimitiveConverter() + string.addBinary(Binary.fromString("foo")) + + converter.createValue shouldEqual SampleClassE(0, 1L, 2, d = true, 3F, 4D, "foo") + } + + "Generate converters for case class with nested class" in { + val converter = Macros.caseClassParquetTupleConverter[SampleClassB] + + val a = converter.getConverter(0).asGroupConverter() + + a.start() + val aInt = a.getConverter(0).asPrimitiveConverter() + aInt.addInt(2) + val aString = a.getConverter(1).asPrimitiveConverter() + aString.addBinary(Binary.fromString("foo")) + a.end() + + val bString = converter.getConverter(1).asPrimitiveConverter() + bString.addBinary(Binary.fromString("toto")) + + converter.createValue() shouldEqual SampleClassB(SampleClassA(2, "foo"), "toto") + } + + "Generate converters for case class with optional nested class" in { + val converter = Macros.caseClassParquetTupleConverter[SampleClassG] + + val a = converter.getConverter(0).asPrimitiveConverter() + a.addInt(0) + + val b = converter.getConverter(1).asGroupConverter() + b.start() + val ba = b.getConverter(0).asGroupConverter() + ba.start() + val baInt = ba.getConverter(0).asPrimitiveConverter() + baInt.addInt(2) + val baString = ba.getConverter(1).asPrimitiveConverter() + baString.addBinary(Binary.fromString("foo")) + ba.end() + b.end() + + val c = converter.getConverter(2).asPrimitiveConverter() + c.addDouble(4D) + + converter.createValue() shouldEqual SampleClassG(0, Some(SampleClassF(Some(SampleClassA(2, "foo")))), 4D) + } } -} +} \ No newline at end of file From 904858a039e25ea1a0166440716543225f347838 Mon Sep 17 00:00:00 2001 From: beethere Date: Wed, 25 Feb 2015 01:16:22 +0100 Subject: [PATCH 093/177] Typed Parquet Tuple #1198 *Macro write support improvement --- .../parquet/tuple/macros/Macros.scala | 31 +-- .../macros/impl/FieldValuesProvider.scala | 76 ------- .../macros/impl/WriteSupportProvider.scala | 106 +++++++++ .../scheme/TypedParquetTupleScheme.scala | 77 +------ .../parquet/tuple/TypedParquetTupleTest.scala | 27 ++- .../parquet/tuple/macros/MacroUnitTests.scala | 207 +++++++++++------- 6 files changed, 277 insertions(+), 247 deletions(-) delete mode 100644 scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/FieldValuesProvider.scala create mode 100644 scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/WriteSupportProvider.scala diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/Macros.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/Macros.scala index 1d75d78888..f055afa23b 100644 --- a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/Macros.scala +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/Macros.scala @@ -1,7 +1,9 @@ package com.twitter.scalding.parquet.tuple.macros -import com.twitter.scalding.parquet.tuple.macros.impl.{ ParquetTupleConverterProvider, ParquetSchemaProvider, FieldValuesProvider } +import com.twitter.scalding.parquet.tuple.macros.impl.{ ParquetSchemaProvider, ParquetTupleConverterProvider, WriteSupportProvider } import com.twitter.scalding.parquet.tuple.scheme.ParquetTupleConverter +import parquet.io.api.RecordConsumer +import parquet.schema.MessageType import scala.language.experimental.macros @@ -30,26 +32,6 @@ object Macros { */ def caseClassParquetSchema[T]: String = macro ParquetSchemaProvider.toParquetSchemaImpl[T] - /** - * Macro used to generate function which permits to flat(at every level) a record to a index-value map. - * For example if we have: - * - * case class SampleClassA(short: Short, int: Int) - * case class SampleClassB(bool: Boolean, a: SampleClassA, long: Long, float: Float) - * - * val b = SampleClassB(true, SampleClassA(1, 4), 6L, 5F) - * - * After flatting using the generated function , we will get a map like this: - * - * Map(0 -> true, 1 -> 1, 2 -> 4, 3 -> 6L, 4 -> 5F) - * This macro can be used to define [[com.twitter.scalding.parquet.tuple.scheme.ParquetWriteSupport]]. - * See this class for more details. - * - * @tparam T Case class type that contains only primitive fields or nested case class. - * @return Case class record field values flat function - */ - def caseClassFieldValues[T]: T => Map[Int, Any] = macro FieldValuesProvider.toFieldValuesImpl[T] - /** * Macro used to generate parquet tuple converter for a given case class that contains only primitive fields. * Option field and nested group is supported. @@ -58,4 +40,11 @@ object Macros { * @return Generated parquet converter */ def caseClassParquetTupleConverter[T]: ParquetTupleConverter = macro ParquetTupleConverterProvider.toParquetTupleConverterImpl[T] + + /** + * Macro used to generate case class write support to parquet. + * @tparam T User defined case class tuple type. + * @return Generated case class tuple write support function. + */ + def caseClassWriteSupport[T]: (T, RecordConsumer, MessageType) => Unit = macro WriteSupportProvider.toWriteSupportImpl[T] } diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/FieldValuesProvider.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/FieldValuesProvider.scala deleted file mode 100644 index 501a444961..0000000000 --- a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/FieldValuesProvider.scala +++ /dev/null @@ -1,76 +0,0 @@ -package com.twitter.scalding.parquet.tuple.macros.impl - -import com.twitter.bijection.macros.impl.IsCaseClassImpl - -import scala.reflect.macros.Context - -object FieldValuesProvider { - - def toFieldValuesImpl[T](ctx: Context)(implicit T: ctx.WeakTypeTag[T]): ctx.Expr[T => Map[Int, Any]] = { - import ctx.universe._ - - if (!IsCaseClassImpl.isCaseClassType(ctx)(T.tpe)) - ctx.abort(ctx.enclosingPosition, - s"""We cannot enforce ${T.tpe} is a case class, - either it is not a case class or this macro call is possibly enclosed in a class. - This will mean the macro is operating on a non-resolved type.""") - - def matchField(idx: Int, fieldType: Type, pTree: Tree): (Int, Tree) = { - def appendFieldValue(idx: Int): (Int, Tree) = - (idx + 1, q"""if($pTree != null) fieldValueMap += $idx -> $pTree""") - - fieldType match { - case tpe if tpe =:= typeOf[String] || - tpe =:= typeOf[Boolean] || - tpe =:= typeOf[Short] || - tpe =:= typeOf[Int] || - tpe =:= typeOf[Long] || - tpe =:= typeOf[Float] || - tpe =:= typeOf[Double] => - appendFieldValue(idx) - - case tpe if tpe.erasure =:= typeOf[Option[Any]] => - val cacheName = newTermName(ctx.fresh(s"optionIndex")) - val innerType = tpe.asInstanceOf[TypeRefApi].args.head - val (newIdx, subTree) = matchField(idx, innerType, q"$cacheName") - (newIdx, q""" - if($pTree.isDefined) { - val $cacheName = $pTree.get - $subTree - } - """) - - case tpe if IsCaseClassImpl.isCaseClassType(ctx)(tpe) => expandMethod(idx, tpe, pTree) - case _ => ctx.abort(ctx.enclosingPosition, s"Case class $T is not pure primitives or nested case classes") - } - } - - def expandMethod(parentIdx: Int, outerTpe: Type, pTree: Tree): (Int, Tree) = { - outerTpe - .declarations - .collect { case m: MethodSymbol if m.isCaseAccessor => m } - .foldLeft((parentIdx, q"")) { - case ((idx, existingTree), accessorMethod) => - val (newIdx, subTree) = matchField(idx, accessorMethod.returnType, q"""$pTree.$accessorMethod""") - (newIdx, q""" - $existingTree - $subTree""") - } - } - - val (finalIdx, allFieldValues) = expandMethod(0, T.tpe, q"t") - - if (finalIdx == 0) - ctx.abort(ctx.enclosingPosition, "Didn't consume any elements in the tuple, possibly empty case class?") - - val fieldValues = q""" - val values: $T => _root_.scala.collection.immutable.Map[Int, Any] = t => { - var fieldValueMap = _root_.scala.collection.immutable.Map[Int, Any]() - $allFieldValues - fieldValueMap - } - values - """ - ctx.Expr[T => Map[Int, Any]](fieldValues) - } -} diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/WriteSupportProvider.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/WriteSupportProvider.scala new file mode 100644 index 0000000000..b4a74ab435 --- /dev/null +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/WriteSupportProvider.scala @@ -0,0 +1,106 @@ +package com.twitter.scalding.parquet.tuple.macros.impl + +import com.twitter.bijection.macros.impl.IsCaseClassImpl +import parquet.io.api.RecordConsumer +import parquet.schema.MessageType + +import scala.reflect.macros.Context + +object WriteSupportProvider { + + def toWriteSupportImpl[T](ctx: Context)(implicit T: ctx.WeakTypeTag[T]): ctx.Expr[(T, RecordConsumer, MessageType) => Unit] = { + import ctx.universe._ + + if (!IsCaseClassImpl.isCaseClassType(ctx)(T.tpe)) + ctx.abort(ctx.enclosingPosition, + s"""We cannot enforce ${T.tpe} is a case class, + either it is not a case class or this macro call is possibly enclosed in a class. + This will mean the macro is operating on a non-resolved type.""") + + def matchField(idx: Int, fieldType: Type, fValue: Tree, groupName: TermName): (Int, Tree) = { + def writePrimitiveField(wTree: Tree) = + (idx + 1, q"""rc.startField($groupName.getFieldName($idx), $idx) + $wTree + rc.endField($groupName.getFieldName($idx), $idx)""") + + def writeGroupField(subTree: Tree) = + q"""rc.startGroup() + rc.startField($groupName.getFieldName($idx), $idx) + $subTree + rc.endField($groupName.getFieldName($idx), $idx) + rc.endGroup() + """ + fieldType match { + case tpe if tpe =:= typeOf[String] => + writePrimitiveField(q"rc.addBinary(Binary.fromString($fValue))") + case tpe if tpe =:= typeOf[Boolean] => + writePrimitiveField(q"rc.addBoolean($fValue)") + case tpe if tpe =:= typeOf[Short] => + writePrimitiveField(q"rc.addInteger($fValue.toInt)") + case tpe if tpe =:= typeOf[Int] => + writePrimitiveField(q"rc.addInteger($fValue)") + case tpe if tpe =:= typeOf[Long] => + writePrimitiveField(q"rc.addLong($fValue)") + case tpe if tpe =:= typeOf[Float] => + writePrimitiveField(q"rc.addFloat($fValue)") + case tpe if tpe =:= typeOf[Double] => + writePrimitiveField(q"rc.addDouble($fValue)") + case tpe if tpe.erasure =:= typeOf[Option[Any]] => + val cacheName = newTermName(ctx.fresh(s"optionIndex")) + val innerType = tpe.asInstanceOf[TypeRefApi].args.head + val (_, subTree) = matchField(idx, innerType, q"$cacheName", groupName) + (idx + 1, q""" + if($fValue.isDefined) { + val $cacheName = $fValue.get + $subTree + } + """) + + case tpe if IsCaseClassImpl.isCaseClassType(ctx)(tpe) => + val newGroupName = createGroupName() + val (_, subTree) = expandMethod(tpe, fValue, newGroupName) + (idx + 1, + q""" + val $newGroupName = $groupName.getType($idx).asGroupType() + ${writeGroupField(subTree)}""") + + case _ => ctx.abort(ctx.enclosingPosition, s"Case class $T is not pure primitives or nested case classes") + } + } + + def expandMethod(outerTpe: Type, pValueTree: Tree, groupName: TermName): (Int, Tree) = { + outerTpe + .declarations + .collect { case m: MethodSymbol if m.isCaseAccessor => m } + .foldLeft((0, q"")) { + case ((idx, existingTree), getter) => + val (newIdx, subTree) = matchField(idx, getter.returnType, q"$pValueTree.$getter", groupName) + (newIdx, q""" + $existingTree + $subTree + """) + } + } + + def createGroupName(): TermName = newTermName(ctx.fresh("group")) + + val rootGroupName = createGroupName() + + val (finalIdx, funcBody) = expandMethod(T.tpe, q"t", rootGroupName) + + if (finalIdx == 0) + ctx.abort(ctx.enclosingPosition, "Didn't consume any elements in the tuple, possibly empty case class?") + + val writeFunction: Tree = q""" + val writeFunc = (t: $T, rc: _root_.parquet.io.api.RecordConsumer, schema: _root_.parquet.schema.MessageType) => { + + var $rootGroupName: _root_.parquet.schema.GroupType = schema + rc.startMessage + $funcBody + rc.endMessage + } + writeFunc + """ + ctx.Expr[(T, RecordConsumer, MessageType) => Unit](q"$writeFunction") + } +} diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/TypedParquetTupleScheme.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/TypedParquetTupleScheme.scala index 60b5c665c2..261a705c22 100644 --- a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/TypedParquetTupleScheme.scala +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/TypedParquetTupleScheme.scala @@ -1,6 +1,6 @@ package com.twitter.scalding.parquet.tuple.scheme -import java.util.{ HashMap => JHashMap, List => JList, Map => JMap } +import java.util.{ HashMap => JHashMap, Map => JMap } import _root_.parquet.filter2.predicate.FilterPredicate import _root_.parquet.hadoop.api.ReadSupport.ReadContext @@ -60,24 +60,18 @@ trait ParquetReadSupport[T] extends ReadSupport[T] { /** * Parquet write support used by [[parquet.hadoop.ParquetOutputFormat]] to write values to parquet output. - * User must provide record schema and a function which permits to flat(at every level) a record to a index-value map. - * For example if we get: + * User must provide record schema and a function which permits to write a used defined case class to parquet store with + * the record consumer and schema definition. * - * case class SampleClassA(short: Short, int: Int) - * case class SampleClassB(bool: Boolean, a: SampleClassA, long: Long, float: Float) - * - * val b = SampleClassB(true, SampleClassA(1, 4), 6L, 5F) - * - * After flatting using the function , we should get a map like this: - * - * Map(0 -> true, 1 -> 1, 2 -> 4, 3 -> 6L, 4 -> 5F) - * - * For case class value types, we provide a macro to generate the field values function so that user + * For case class value types, we provide a macro to generate the write support function so that user * can define a ParquetWriteSupport like this: * * class SampleClassWriteSupport extends TupleWriteSupport[SampleClassB] { * import com.twitter.scalding.parquet.tuple.macros.Macros._ - * override val fieldValues: (SampleClassB) => Map[Int, Any] = caseClassFieldValues[SampleClassB] + * + * override def writeRecord(r: SampleClassB, rc: RecordConsumer, schema: MessageType):Unit = + * Macros.caseClassWriteSupport[SampleClassB](r, rc, schema) + * * override val rootSchema: String = caseClassParquetSchema[SampleClassB] * } * @@ -85,9 +79,6 @@ trait ParquetReadSupport[T] extends ReadSupport[T] { */ trait ParquetWriteSupport[T] extends WriteSupport[T] { - //function which permits to flat(at every level) a record to a index-value map. - val fieldValues: T => Map[Int, Any] - var recordConsumer: RecordConsumer = null val rootSchema: String @@ -99,57 +90,9 @@ trait ParquetWriteSupport[T] extends WriteSupport[T] { override def prepareForWrite(rc: RecordConsumer): Unit = recordConsumer = rc - override def write(record: T): Unit = { - val valuesMap = fieldValues(record) - recordConsumer.startMessage() - if (record != null) writeGroupType(0, valuesMap, rootType) - recordConsumer.endMessage() - } - - private def writeGroupType(outerFieldIdx: Int, valuesMap: Map[Int, Any], groupType: GroupType): Unit = { - val fields: JList[Type] = groupType.getFields - (0 until fields.size).map { i => - val field = fields.get(i) - val valueFieldIndex = outerFieldIdx + i - valuesMap.get(valueFieldIndex).map { v => - recordConsumer.startField(field.getName, i) - if (field.isPrimitive) - writePrimitiveType(v, field.asPrimitiveType()) - else { - recordConsumer.startGroup() - writeGroupType(valueFieldIndex, valuesMap, field.asGroupType()) - recordConsumer.endGroup() - } - recordConsumer.endField(field.getName, i) - } - } - } + override def write(record: T): Unit = writeRecord(record, recordConsumer, rootType) - private def writePrimitiveType(value: Any, field: PrimitiveType) = { - field.getPrimitiveTypeName match { - case PrimitiveType.PrimitiveTypeName.BINARY => - recordConsumer.addBinary(Binary.fromString(value.asInstanceOf[String])) - case PrimitiveType.PrimitiveTypeName.BOOLEAN => - recordConsumer.addBoolean(value.asInstanceOf[Boolean]) - case PrimitiveType.PrimitiveTypeName.INT32 => - value match { - case i: Int => - recordConsumer.addInteger(i) - case s: Short => - recordConsumer.addInteger(s.toInt) - case _ => - throw new UnsupportedOperationException(field.getName + " write to int not supported") - } - case PrimitiveType.PrimitiveTypeName.INT64 => - recordConsumer.addLong(value.asInstanceOf[Long]) - case PrimitiveType.PrimitiveTypeName.DOUBLE => - recordConsumer.addDouble(value.asInstanceOf[Double]) - case PrimitiveType.PrimitiveTypeName.FLOAT => - recordConsumer.addFloat(value.asInstanceOf[Float]) - case _ => - throw new UnsupportedOperationException(field.getName + " type not implemented") - } - } + def writeRecord(r: T, rc: RecordConsumer, schema: MessageType): Unit } /** diff --git a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala index 24eee042fa..2e617d60a2 100644 --- a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala +++ b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala @@ -10,7 +10,8 @@ import com.twitter.scalding.{ Args, Job, TypedTsv } import org.scalatest.{ Matchers, WordSpec } import parquet.filter2.predicate.FilterApi.binaryColumn import parquet.filter2.predicate.{FilterApi, FilterPredicate} -import parquet.io.api.Binary +import parquet.io.api.{RecordConsumer, Binary} +import parquet.schema.MessageType class TypedParquetTupleTest extends WordSpec with Matchers with HadoopPlatformTest { "TypedParquetTuple" should { @@ -30,7 +31,7 @@ class TypedParquetTupleTest extends WordSpec with Matchers with HadoopPlatformTe HadoopPlatformJobTest(new ReadWithFilterPredicateJob(_), cluster) .arg("input", tempParquet) - .sink[Int]("output") { _.toSet shouldBe values.filter(_.string == "B1").map(_.int).toSet } + .sink[Boolean]("output") { _.toSet shouldBe values.filter(_.string == "B1").map(_.a.bool).toSet } .run } finally { @@ -58,19 +59,26 @@ object TestValues { case class SampleClassA(bool: Boolean, short: Short, long: Long, float: Float) case class SampleClassB(string: String, int: Int, double: Option[Double], a: SampleClassA) +case class SampleClassC(string: String, a: SampleClassA) object SampleClassB { val schema: String = Macros.caseClassParquetSchema[SampleClassB] } -class ReadSupport extends ParquetReadSupport[SampleClassB] { +class BReadSupport extends ParquetReadSupport[SampleClassB] { override val tupleConverter: ParquetTupleConverter = Macros.caseClassParquetTupleConverter[SampleClassB] override val rootSchema: String = SampleClassB.schema } +class CReadSupport extends ParquetReadSupport[SampleClassC] { + override val tupleConverter: ParquetTupleConverter = Macros.caseClassParquetTupleConverter[SampleClassC] + override val rootSchema: String = Macros.caseClassParquetSchema[SampleClassC] +} + class WriteSupport extends ParquetWriteSupport[SampleClassB] { - override val fieldValues: (SampleClassB) => Map[Int, Any] = Macros.caseClassFieldValues[SampleClassB] override val rootSchema: String = SampleClassB.schema + override def writeRecord(r: SampleClassB, rc: RecordConsumer, schema: MessageType):Unit = + Macros.caseClassWriteSupport[SampleClassB](r, rc, schema) } /** @@ -94,21 +102,22 @@ class ReadFromTypedParquetTupleJob(args: Args) extends Job(args) { val inputPath = args.required("input") - val input = TypedParquet[SampleClassB, ReadSupport](Seq(inputPath)) + val input = TypedParquet[SampleClassB, BReadSupport](Seq(inputPath)) TypedPipe.from(input).map(_.a.float).write(TypedTsv[Float]("output")) } /** - * Test job read from a typed parquet source with filter predicate and write the mapped value into a typed tsv sink - * To test typed parquet tuple can bse used as source and apply filter predicate correctly + * Test job read from a typed parquet source with filter predicate and push down(SampleClassC takes only part of + * SampleClassB's data) + * To test typed parquet tuple can bse used as source and apply filter predicate and push down correctly */ class ReadWithFilterPredicateJob(args: Args) extends Job(args) { val fp: FilterPredicate = FilterApi.eq(binaryColumn("string"), Binary.fromString("B1")) val inputPath = args.required("input") - val input = TypedParquet[SampleClassB, ReadSupport](Seq(inputPath), Some(fp)) + val input = TypedParquet[SampleClassC, CReadSupport](Seq(inputPath), Some(fp)) - TypedPipe.from(input).map(_.int).write(TypedTsv[Int]("output")) + TypedPipe.from(input).map(_.a.bool).write(TypedTsv[Boolean]("output")) } diff --git a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/macros/MacroUnitTests.scala b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/macros/MacroUnitTests.scala index 020b7c3a6d..920b4f0fb1 100644 --- a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/macros/MacroUnitTests.scala +++ b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/macros/MacroUnitTests.scala @@ -1,7 +1,8 @@ package com.twitter.scalding.parquet.tuple.macros -import org.scalatest.{ Matchers, WordSpec } -import parquet.io.api.Binary +import org.scalatest.mock.MockitoSugar +import org.scalatest.{Matchers, WordSpec} +import parquet.io.api.{Binary, RecordConsumer} import parquet.schema.MessageTypeParser case class SampleClassA(x: Int, y: String) @@ -16,11 +17,11 @@ case class SampleClassE(a: Int, b: Long, c: Short, d: Boolean, e: Float, f: Doub case class SampleClassF(a: Option[SampleClassA]) -case class SampleClassG(a: Int, b: Option[SampleClassF], c: Double) +case class SampleClassG(a: Int, b: Option[SampleClassB], c: Double) -class MacroUnitTests extends WordSpec with Matchers { +class MacroUnitTests extends WordSpec with Matchers with MockitoSugar { - "Macro generated case class parquet schema generator" should { + "Macro case class parquet schema generator" should { "Generate parquet schema for SampleClassA" in { val schema = MessageTypeParser.parseMessageType(Macros.caseClassParquetSchema[SampleClassA]) @@ -103,74 +104,7 @@ class MacroUnitTests extends WordSpec with Matchers { } - "Macro generated case class field values generator" should { - "Generate field values for SampleClassA" in { - val a = SampleClassA(1, "foo") - val values = Macros.caseClassFieldValues[SampleClassA] - values(a) shouldEqual Map(0 -> 1, 1 -> "foo") - } - - "Generate field values for SampleClassB with nested case class" in { - val a = SampleClassA(1, "foo") - val b = SampleClassB(a, "b") - val values = Macros.caseClassFieldValues[SampleClassB] - - values(b) shouldEqual Map(0 -> 1, 1 -> "foo", 2 -> "b") - } - - "Generate field values for SampleClassC with two nested case classes" in { - - val a = SampleClassA(1, "foo") - - val b = SampleClassB(a, "b") - - val c = SampleClassC(a, b) - - val values = Macros.caseClassFieldValues[SampleClassC] - - values(c) shouldEqual Map(0 -> 1, 1 -> "foo", 2 -> 1, 3 -> "foo", 4 -> "b") - } - - "Generate field values for SampleClassD with option values" in { - val d = SampleClassD("toto", b = true, Some(2), 1, 2L, 3F, Some(5D)) - val values = Macros.caseClassFieldValues[SampleClassD] - values(d) shouldEqual Map(0 -> "toto", 1 -> true, 2 -> 2, 3 -> 1, 4 -> 2L, 5 -> 3F, 6 -> 5D) - - val d2 = SampleClassD("toto", b = true, None, 1, 2L, 3F, None) - val values2 = Macros.caseClassFieldValues[SampleClassD] - values2(d2) shouldEqual Map(0 -> "toto", 1 -> true, 3 -> 1, 4 -> 2L, 5 -> 3F) - } - - "Generate field values for SampleClassF with optional nested case class " in { - val a = SampleClassA(1, "foo") - val f1 = SampleClassF(Some(a)) - val values1 = Macros.caseClassFieldValues[SampleClassF] - values1(f1) shouldEqual Map(0 -> 1, 1 -> "foo") - - val f2 = SampleClassF(None) - val values2 = Macros.caseClassFieldValues[SampleClassF] - values2(f2) shouldEqual Map.empty - } - - "Generate field values for SampleClassG with nested case class containing optional fields" in { - val a = SampleClassA(1, "foo") - val f1 = SampleClassF(Some(a)) - val g1 = SampleClassG(0, Some(f1), 1D) - val values1 = Macros.caseClassFieldValues[SampleClassG] - values1(g1) shouldEqual Map(0 -> 0, 1 -> 1, 2 -> "foo", 3 -> 1D) - - val f2 = SampleClassF(None) - val g2 = SampleClassG(1, Some(f2), 2D) - val values2 = Macros.caseClassFieldValues[SampleClassG] - values2(g2) shouldEqual Map(0 -> 1, 3 -> 2D) - - val g3 = SampleClassG(1, None, 3D) - val values3 = Macros.caseClassFieldValues[SampleClassG] - values3(g3) shouldEqual Map(0 -> 1, 3 -> 3D) - } - } - - "Macro generated case class converters generator" should { + "Macro case class converters generator" should { "Generate converters for all primitive types" in { val converter = Macros.caseClassParquetTupleConverter[SampleClassE] @@ -232,12 +166,137 @@ class MacroUnitTests extends WordSpec with Matchers { val baString = ba.getConverter(1).asPrimitiveConverter() baString.addBinary(Binary.fromString("foo")) ba.end() + + val bString = b.getConverter(1).asPrimitiveConverter() + bString.addBinary(Binary.fromString("b1")) b.end() val c = converter.getConverter(2).asPrimitiveConverter() c.addDouble(4D) - converter.createValue() shouldEqual SampleClassG(0, Some(SampleClassF(Some(SampleClassA(2, "foo")))), 4D) + converter.createValue() shouldEqual SampleClassG(0, Some(SampleClassB(SampleClassA(2, "foo"), "b1")), 4D) } } + + + "Macro case class parquet write support generator" should { + "Generate write support for class with all the primitive type fields" in { + val writeSupportFn = Macros.caseClassWriteSupport[SampleClassE] + val e = SampleClassE(0, 1L, 2, d = true, 3F, 4D, "foo") + val schema = Macros.caseClassParquetSchema[SampleClassE] + val rc = new StringBuilderRecordConsumer + writeSupportFn(e, rc, MessageTypeParser.parseMessageType(schema)) + + rc.writeScenario shouldEqual """start message + |start field a at 0 + |write INT32 0 + |end field a at 0 + |start field b at 1 + |write INT64 1 + |start message + |start field a at 0 + |write INT32 0 + |end field a at 0 + |start field b at 1 + |write INT64 1 + |end field b at 1 + |start field c at 2 + |write INT32 2 + |end field c at 2 + |start field d at 3 + |write BOOLEAN true + |end field d at 3 + |start field e at 4 + |write FLOAT 3.0 + |end field e at 4 + |start field f at 5 + |write DOUBLE 4.0 + |end field f at 5 + |start field g at 6 + |write BINARY foo + |end field g at 6 + |end Message""".stripMargin + + } + + "Generate write support for nested case class and optinal fields" in { + val writeSupportFn = Macros.caseClassWriteSupport[SampleClassG] + + val g = SampleClassG(0, Some(SampleClassB(SampleClassA(2, "foo"), "b1")), 4D) + + val schema = MessageTypeParser.parseMessageType(Macros.caseClassParquetSchema[SampleClassG]) + val rc = new StringBuilderRecordConsumer + writeSupportFn(g, rc, schema) + + rc.writeScenario shouldEqual """start message + |start field a at 0 + |write INT32 0 + |end field a at 0 + |start group + |start field b at 1 + |start group + |start field a at 0 + |start field x at 0 + |write INT32 2 + |end field x at 0 + |start field y at 1 + |write BINARY foo + |end field y at 1 + |end field a at 0 + |end group + |start field y at 1 + |write BINARY b1 + |end field y at 1 + |end field b at 1 + |end group + |start field c at 2 + |write DOUBLE 4.0 + |end field c at 2 + |end Message""".stripMargin + + //test write tuple with optional field = None + val g2 = SampleClassG(0, None, 4D) + val rc2 = new StringBuilderRecordConsumer + writeSupportFn(g2, rc2, schema) + rc2.writeScenario shouldEqual """start message + |start field a at 0 + |write INT32 0 + |end field a at 0 + |start field c at 2 + |write DOUBLE 4.0 + |end field c at 2 + |end Message""".stripMargin + } + } +} + +//class to simulate record consumer for testing +class StringBuilderRecordConsumer extends RecordConsumer { + val sb = new StringBuilder + + override def startMessage(): Unit = sb.append("start message\n") + + override def endMessage(): Unit = sb.append("end Message") + + override def addFloat(v: Float): Unit = sb.append(s"write FLOAT $v\n") + + override def addBinary(binary: Binary): Unit = sb.append(s"write BINARY ${binary.toStringUsingUTF8}\n") + + override def addDouble(v: Double): Unit = sb.append(s"write DOUBLE $v\n") + + override def endGroup(): Unit = sb.append("end group\n") + + override def endField(s: String, i: Int): Unit = sb.append(s"end field $s at $i\n") + + override def startGroup(): Unit = sb.append("start group\n") + + override def startField(s: String, i: Int): Unit = sb.append(s"start field $s at $i\n") + + override def addBoolean(b: Boolean): Unit = sb.append(s"write BOOLEAN $b\n") + + override def addLong(l: Long): Unit = sb.append(sb.append(s"write INT64 $l\n")) + + override def addInteger(i: Int): Unit = sb.append(s"write INT32 $i\n") + + def writeScenario = sb.toString() } \ No newline at end of file From 2782288c44e78c274e6477ffb24c155d1a5ea112 Mon Sep 17 00:00:00 2001 From: beethere Date: Fri, 10 Apr 2015 23:16:38 +0200 Subject: [PATCH 094/177] Typed Parquet Tuple #1198 *Add Byte type support --- project/Build.scala | 5 +++-- .../parquet/tuple/macros/Macros.scala | 5 +++++ .../macros/impl/ParquetSchemaProvider.scala | 2 +- .../impl/ParquetTupleConverterProvider.scala | 2 ++ .../macros/impl/WriteSupportProvider.scala | 8 +++++--- .../tuple/scheme/ParquetTupleConverter.scala | 5 +++++ .../parquet/tuple/TypedParquetTupleTest.scala | 10 +++++----- .../parquet/tuple/macros/MacroUnitTests.scala | 19 +++++++++++++------ 8 files changed, 39 insertions(+), 17 deletions(-) diff --git a/project/Build.scala b/project/Build.scala index 991c764c8a..89aa7ebb43 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -35,6 +35,7 @@ object ScaldingBuild extends Build { val json4SVersion = "3.2.11" val parquetVersion = "1.6.0rc4" val protobufVersion = "2.4.1" + val quasiquotesVersion = "2.0.1" val scalaCheckVersion = "1.12.2" val scalaTestVersion = "2.2.4" val scalameterVersion = "0.6" @@ -314,7 +315,7 @@ object ScaldingBuild extends Build { "org.apache.hadoop" % "hadoop-core" % hadoopVersion % "provided", "org.scala-lang" % "scala-reflect" % scalaVersion, "com.twitter" %% "bijection-macros" % bijectionVersion - ) ++ (if(isScala210x(scalaVersion)) Seq("org.scalamacros" %% "quasiquotes" % "2.0.1") else Seq()) + ) ++ (if(isScala210x(scalaVersion)) Seq("org.scalamacros" %% "quasiquotes" % quasiquotesVersion) else Seq()) }, addCompilerPlugin("org.scalamacros" % "paradise" % "2.0.1" cross CrossVersion.full)) .dependsOn(scaldingCore, scaldingHadoopTest) @@ -426,7 +427,7 @@ object ScaldingBuild extends Build { "org.scala-lang" % "scala-library" % scalaVersion, "org.scala-lang" % "scala-reflect" % scalaVersion, "com.twitter" %% "bijection-macros" % bijectionVersion - ) ++ (if(isScala210x(scalaVersion)) Seq("org.scalamacros" %% "quasiquotes" % "2.0.1") else Seq()) + ) ++ (if(isScala210x(scalaVersion)) Seq("org.scalamacros" %% "quasiquotes" % quasiquotesVersion) else Seq()) }, addCompilerPlugin("org.scalamacros" % "paradise" % "2.0.1" cross CrossVersion.full) ).dependsOn(scaldingCore, scaldingHadoopTest) diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/Macros.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/Macros.scala index f055afa23b..acedb7a4d9 100644 --- a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/Macros.scala +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/Macros.scala @@ -7,6 +7,11 @@ import parquet.schema.MessageType import scala.language.experimental.macros +/** + * Macros used to generate parquet tuple read/write support. + * For right now these macros support only case class that contains only primitive fields or nested case classes. + * Option field is also supported. But collection types like List are not supported yet. + */ object Macros { /** * Macro used to generate parquet schema for a given case class that contains only primitive fields. diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/ParquetSchemaProvider.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/ParquetSchemaProvider.scala index 61878eb723..a322d251a0 100644 --- a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/ParquetSchemaProvider.scala +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/ParquetSchemaProvider.scala @@ -26,7 +26,7 @@ object ParquetSchemaProvider { createPrimitiveTypeField(q"_root_.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY") case tpe if tpe =:= typeOf[Boolean] => createPrimitiveTypeField(q"_root_.parquet.schema.PrimitiveType.PrimitiveTypeName.BOOLEAN") - case tpe if tpe =:= typeOf[Short] || tpe =:= typeOf[Int] => + case tpe if tpe =:= typeOf[Short] || tpe =:= typeOf[Int] || tpe =:= typeOf[Byte] => createPrimitiveTypeField(q"_root_.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32") case tpe if tpe =:= typeOf[Long] => createPrimitiveTypeField(q"_root_.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64") diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/ParquetTupleConverterProvider.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/ParquetTupleConverterProvider.scala index 90462036c1..c9f98fc925 100644 --- a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/ParquetTupleConverterProvider.scala +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/ParquetTupleConverterProvider.scala @@ -38,6 +38,8 @@ object ParquetTupleConverterProvider { List(createConverter(q"new _root_.com.twitter.scalding.parquet.tuple.scheme.StringConverter($idx, this, $isOption)")) case tpe if tpe =:= typeOf[Boolean] => List(createConverter(q"new _root_.com.twitter.scalding.parquet.tuple.scheme.BooleanConverter($idx, this, $isOption)")) + case tpe if tpe =:= typeOf[Byte] => + List(createConverter(q"new _root_.com.twitter.scalding.parquet.tuple.scheme.ByteConverter($idx, this, $isOption)")) case tpe if tpe =:= typeOf[Short] => List(createConverter(q"new _root_.com.twitter.scalding.parquet.tuple.scheme.ShortConverter($idx, this, $isOption)")) case tpe if tpe =:= typeOf[Int] => diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/WriteSupportProvider.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/WriteSupportProvider.scala index b4a74ab435..3c77e2531a 100644 --- a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/WriteSupportProvider.scala +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/WriteSupportProvider.scala @@ -24,11 +24,11 @@ object WriteSupportProvider { rc.endField($groupName.getFieldName($idx), $idx)""") def writeGroupField(subTree: Tree) = - q"""rc.startGroup() - rc.startField($groupName.getFieldName($idx), $idx) + q"""rc.startField($groupName.getFieldName($idx), $idx) + rc.startGroup() $subTree - rc.endField($groupName.getFieldName($idx), $idx) rc.endGroup() + rc.endField($groupName.getFieldName($idx), $idx) """ fieldType match { case tpe if tpe =:= typeOf[String] => @@ -45,6 +45,8 @@ object WriteSupportProvider { writePrimitiveField(q"rc.addFloat($fValue)") case tpe if tpe =:= typeOf[Double] => writePrimitiveField(q"rc.addDouble($fValue)") + case tpe if tpe =:= typeOf[Byte] => + writePrimitiveField(q"rc.addInteger($fValue.toInt)") case tpe if tpe.erasure =:= typeOf[Option[Any]] => val cacheName = newTermName(ctx.fresh(s"optionIndex")) val innerType = tpe.asInstanceOf[TypeRefApi].args.head diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/ParquetTupleConverter.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/ParquetTupleConverter.scala index 602c83ff79..5dea340985 100644 --- a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/ParquetTupleConverter.scala +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/ParquetTupleConverter.scala @@ -83,6 +83,11 @@ class ShortConverter(index: Int, parent: ParquetTupleConverter, isOption: Boolea override def addInt(value: Int) = appendValue(Try(value.toShort).getOrElse(null)) } +class ByteConverter(index: Int, parent: ParquetTupleConverter, isOption: Boolean = false) + extends PrimitiveTypeConverter(index, parent, isOption) { + override def addInt(value: Int) = appendValue(Try(value.toByte).getOrElse(null)) +} + class BooleanConverter(index: Int, parent: ParquetTupleConverter, isOption: Boolean = false) extends PrimitiveTypeConverter(index, parent, isOption) { override def addBoolean(value: Boolean) = appendValue(value) diff --git a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala index 2e617d60a2..a9dd13d8bc 100644 --- a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala +++ b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala @@ -51,13 +51,13 @@ class TypedParquetTupleTest extends WordSpec with Matchers with HadoopPlatformTe } object TestValues { - val values = Seq(SampleClassB("B1", 1, Some(4.0D), SampleClassA(bool = true, 5, 1L, 1.2F)), - SampleClassB("B2", 3, Some(3.0D), SampleClassA(bool = false, 4, 2L, 2.3F)), - SampleClassB("B3", 9, None, SampleClassA(bool = true, 6, 3L, 3.4F)), - SampleClassB("B4", 8, Some(5.0D), SampleClassA(bool = false, 7, 4L, 4.5F))) + val values = Seq(SampleClassB("B1", 1, Some(4.0D), SampleClassA(bool = true, 5, 1L, 1.2F, 1)), + SampleClassB("B2", 3, Some(3.0D), SampleClassA(bool = false, 4, 2L, 2.3F, 2)), + SampleClassB("B3", 9, None, SampleClassA(bool = true, 6, 3L, 3.4F, 3)), + SampleClassB("B4", 8, Some(5.0D), SampleClassA(bool = false, 7, 4L, 4.5F, 4))) } -case class SampleClassA(bool: Boolean, short: Short, long: Long, float: Float) +case class SampleClassA(bool: Boolean, short: Short, long: Long, float: Float, byte: Byte) case class SampleClassB(string: String, int: Int, double: Option[Double], a: SampleClassA) case class SampleClassC(string: String, a: SampleClassA) diff --git a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/macros/MacroUnitTests.scala b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/macros/MacroUnitTests.scala index 920b4f0fb1..242f256b45 100644 --- a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/macros/MacroUnitTests.scala +++ b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/macros/MacroUnitTests.scala @@ -13,7 +13,7 @@ case class SampleClassC(a: SampleClassA, b: SampleClassB) case class SampleClassD(a: String, b: Boolean, c: Option[Short], d: Int, e: Long, f: Float, g: Option[Double]) -case class SampleClassE(a: Int, b: Long, c: Short, d: Boolean, e: Float, f: Double, g: String) +case class SampleClassE(a: Int, b: Long, c: Short, d: Boolean, e: Float, f: Double, g: String, h: Byte) case class SampleClassF(a: Option[SampleClassA]) @@ -97,6 +97,7 @@ class MacroUnitTests extends WordSpec with Matchers with MockitoSugar { | required float e; | required double f; | required binary g; + | required int32 h; |} """.stripMargin) schema shouldEqual expectedSchema @@ -130,7 +131,10 @@ class MacroUnitTests extends WordSpec with Matchers with MockitoSugar { val string = converter.getConverter(6).asPrimitiveConverter() string.addBinary(Binary.fromString("foo")) - converter.createValue shouldEqual SampleClassE(0, 1L, 2, d = true, 3F, 4D, "foo") + val byte = converter.getConverter(7).asPrimitiveConverter() + byte.addInt(1) + + converter.createValue shouldEqual SampleClassE(0, 1L, 2, d = true, 3F, 4D, "foo", 1) } "Generate converters for case class with nested class" in { @@ -182,7 +186,7 @@ class MacroUnitTests extends WordSpec with Matchers with MockitoSugar { "Macro case class parquet write support generator" should { "Generate write support for class with all the primitive type fields" in { val writeSupportFn = Macros.caseClassWriteSupport[SampleClassE] - val e = SampleClassE(0, 1L, 2, d = true, 3F, 4D, "foo") + val e = SampleClassE(0, 1L, 2, d = true, 3F, 4D, "foo", 1) val schema = Macros.caseClassParquetSchema[SampleClassE] val rc = new StringBuilderRecordConsumer writeSupportFn(e, rc, MessageTypeParser.parseMessageType(schema)) @@ -215,6 +219,9 @@ class MacroUnitTests extends WordSpec with Matchers with MockitoSugar { |start field g at 6 |write BINARY foo |end field g at 6 + |start field h at 7 + |write INT32 1 + |end field h at 7 |end Message""".stripMargin } @@ -232,23 +239,23 @@ class MacroUnitTests extends WordSpec with Matchers with MockitoSugar { |start field a at 0 |write INT32 0 |end field a at 0 - |start group |start field b at 1 |start group |start field a at 0 + |start group |start field x at 0 |write INT32 2 |end field x at 0 |start field y at 1 |write BINARY foo |end field y at 1 - |end field a at 0 |end group + |end field a at 0 |start field y at 1 |write BINARY b1 |end field y at 1 - |end field b at 1 |end group + |end field b at 1 |start field c at 2 |write DOUBLE 4.0 |end field c at 2 From 22cbb6364ee538018fdb5ae31debac8fbf8ee1bf Mon Sep 17 00:00:00 2001 From: Kevin Lin Date: Sun, 12 Apr 2015 22:02:57 -0700 Subject: [PATCH 095/177] Add MustHasReducers trait. --- .../scala/com/twitter/scalding/typed/Sketched.scala | 4 ++-- .../com/twitter/scalding/typed/WithReducers.scala | 12 ++++++++++++ 2 files changed, 14 insertions(+), 2 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/Sketched.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/Sketched.scala index 47783a71b6..b1b59f214b 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/Sketched.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/Sketched.scala @@ -53,7 +53,7 @@ case class Sketched[K, V](pipe: TypedPipe[(K, V)], eps: Double, seed: Int)(implicit serialization: K => Array[Byte], ordering: Ordering[K]) - extends HasReducers { + extends MustHasReducers { import Sketched._ def serialize(k: K): Array[Byte] = serialization(k) @@ -91,7 +91,7 @@ case class Sketched[K, V](pipe: TypedPipe[(K, V)], case class SketchJoined[K: Ordering, V, V2, R](left: Sketched[K, V], right: TypedPipe[(K, V2)], numReducers: Int)(joiner: (K, V, Iterable[V2]) => Iterator[R]) - extends HasReducers { + extends MustHasReducers { val reducers = Some(numReducers) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/WithReducers.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/WithReducers.scala index 8b42ff30dc..310a1c8ba9 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/WithReducers.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/WithReducers.scala @@ -23,6 +23,18 @@ trait HasReducers { def reducers: Option[Int] } +/** + * used for types that must know how many reducers they need + * e.g. Sketched + */ +trait MustHasReducers { + def reducers: Option[Int] + if (!reducers.isDefined) { + def className = this.getClass.getSimpleName + throw new IllegalArgumentException(s"Number of reducers must be specified for class ${className}.") + } +} + /** * used for objects that may _set_ how many reducers they need * e.g. CoGrouped, Grouped, SortedGrouped, UnsortedGrouped From ba405d2276673967c7128ef2ae721ad6aba53fba Mon Sep 17 00:00:00 2001 From: Kevin Lin Date: Mon, 13 Apr 2015 00:14:51 -0700 Subject: [PATCH 096/177] make reducers a def in Sketched. --- .../src/main/scala/com/twitter/scalding/typed/Sketched.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/Sketched.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/Sketched.scala index b1b59f214b..a0f323df31 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/Sketched.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/Sketched.scala @@ -58,7 +58,7 @@ case class Sketched[K, V](pipe: TypedPipe[(K, V)], def serialize(k: K): Array[Byte] = serialization(k) - val reducers = Some(numReducers) + override def reducers = Some(numReducers) private lazy implicit val cms = CMS.monoid[Array[Byte]](eps, delta, seed) lazy val sketch: TypedPipe[CMS[Array[Byte]]] = @@ -93,7 +93,7 @@ case class SketchJoined[K: Ordering, V, V2, R](left: Sketched[K, V], numReducers: Int)(joiner: (K, V, Iterable[V2]) => Iterator[R]) extends MustHasReducers { - val reducers = Some(numReducers) + override def reducers = Some(numReducers) //the most of any one reducer we want to try to take up with a single key private val maxReducerFraction = 0.1 From 48c5c26446c4af4aebdcabe8c95ec6f53b0f847e Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Mon, 13 Apr 2015 08:55:42 -1000 Subject: [PATCH 097/177] Clean up some implementations --- .../com/twitter/scalding/Execution.scala | 35 +++++++------ .../twitter/scalding/typed/TypedPipe.scala | 49 +++++++------------ 2 files changed, 38 insertions(+), 46 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala b/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala index e09cf8241e..51b2ecb1e7 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala @@ -45,7 +45,7 @@ import cascading.flow.{ FlowDef, Flow } * zip to flatMap if you want to run two Executions in parallel. */ sealed trait Execution[+T] extends java.io.Serializable { - import Execution.{ EvalCache, FlatMapped, MapCounters, Mapped, OnComplete, RecoverWith, Zipped } + import Execution.{ EvalCache, FlatMapped, GetCounters, ResetCounters, Mapped, OnComplete, RecoverWith, Zipped } /** * Scala uses the filter method in for syntax for pattern matches that can fail. @@ -83,7 +83,7 @@ sealed trait Execution[+T] extends java.io.Serializable { * You may want .getAndResetCounters. */ def getCounters: Execution[(T, ExecutionCounters)] = - MapCounters[T, (T, ExecutionCounters)](this, { case tc @ (t, c) => (tc, c) }) + GetCounters(this) /** * Reads the counters and resets them to zero. Probably what @@ -122,7 +122,7 @@ sealed trait Execution[+T] extends java.io.Serializable { * you want to reset before a zip or a call to flatMap */ def resetCounters: Execution[T] = - MapCounters[T, T](this, { case (t, c) => (t, ExecutionCounters.empty) }) + ResetCounters(this) /** * This causes the Execution to occur. The result is not cached, so each call @@ -249,8 +249,7 @@ object Execution { val promise = Promise[JobStats]() messageQueue.put(RunFlowDef(conf, mode, fd, promise)) promise.future - } - catch { + } catch { case NonFatal(e) => Future.failed(e) } @@ -308,11 +307,15 @@ object Execution { prev.runStats(conf, mode, cache) .map { case (s, stats) => (fn(s), stats) }) } - private case class MapCounters[T, U](prev: Execution[T], - fn: ((T, ExecutionCounters)) => (U, ExecutionCounters)) extends Execution[U] { + private case class GetCounters[T](prev: Execution[T]) extends Execution[(T, ExecutionCounters)] { + def runStats(conf: Config, mode: Mode, cache: EvalCache)(implicit cec: ConcurrentExecutionContext) = + cache.getOrElseInsert(this, + prev.runStats(conf, mode, cache).map { case tc @ (t, c) => (tc, c) }) + } + private case class ResetCounters[T](prev: Execution[T]) extends Execution[T] { def runStats(conf: Config, mode: Mode, cache: EvalCache)(implicit cec: ConcurrentExecutionContext) = cache.getOrElseInsert(this, - prev.runStats(conf, mode, cache).map(fn)) + prev.runStats(conf, mode, cache).map { case (t, _) => (t, ExecutionCounters.empty) }) } private case class OnComplete[T](prev: Execution[T], fn: Try[T] => Unit) extends Execution[T] { @@ -327,8 +330,7 @@ object Execution { res.onComplete { tryT => try { fn(tryT.map(_._1)) - } - finally { + } finally { // Do our best to signal when we are done finished.complete(tryT) } @@ -439,9 +441,6 @@ object Execution { /** Returns a constant Execution[Unit] */ val unit: Execution[Unit] = from(()) - private[scalding] def factory[T](fn: (Config, Mode) => Execution[T]): Execution[T] = - ReaderExecution.flatMap { case (c, m) => fn(c, m) } - /** * This converts a function into an Execution monad. The flowDef returned * is never mutated. @@ -455,17 +454,21 @@ object Execution { private[scalding] def write[T](pipe: TypedPipe[T], sink: TypedSink[T]): Execution[Unit] = WriteExecution(ToWrite(pipe, sink), Nil) + /** + * Convenience method to get the Args + */ + def getArgs: Execution[Args] = ReaderExecution.map(_._1.getArgs) /** * Use this to read the configuration, which may contain Args or options * which describe input on which to run */ - def getConfig: Execution[Config] = factory { case (conf, _) => from(conf) } + def getConfig: Execution[Config] = ReaderExecution.map(_._1) /** Use this to get the mode, which may contain the job conf */ - def getMode: Execution[Mode] = factory { case (_, mode) => from(mode) } + def getMode: Execution[Mode] = ReaderExecution.map(_._2) /** Use this to get the config and mode. */ - def getConfigMode: Execution[(Config, Mode)] = factory { case (conf, mode) => from((conf, mode)) } + def getConfigMode: Execution[(Config, Mode)] = ReaderExecution /** * This is convenience method only here to make it slightly cleaner diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipe.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipe.scala index 79c2afc0b6..a4eb623e8c 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipe.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipe.scala @@ -841,12 +841,13 @@ class TypedPipeFactory[T] private (@transient val next: NoStackAndThen[(FlowDef, // unwrap in a loop, without recursing unwrap(this).toPipe[U](fieldNames)(flowDef, mode, setter) - override def toIterableExecution: Execution[Iterable[T]] = Execution.factory { (conf, mode) => - // This can only terminate in TypedPipeInst, which will - // keep the reference to this flowDef - val flowDef = new FlowDef - val nextPipe = unwrap(this)(flowDef, mode) - nextPipe.toIterableExecution + override def toIterableExecution: Execution[Iterable[T]] = Execution.getConfigMode.flatMap { + case (conf, mode) => + // This can only terminate in TypedPipeInst, which will + // keep the reference to this flowDef + val flowDef = new FlowDef + val nextPipe = unwrap(this)(flowDef, mode) + nextPipe.toIterableExecution } @annotation.tailrec @@ -918,10 +919,7 @@ class TypedPipeInst[T] private[scalding] (@transient inpipe: Pipe, RichPipe(inpipe).flatMapTo[TupleEntry, U](fields -> fieldNames)(flatMapFn) } - override def toIterableExecution: Execution[Iterable[T]] = Execution.factory { (conf, m) => - // To convert from java iterator to scala below - import scala.collection.JavaConverters._ - checkMode(m) + override def toIterableExecution: Execution[Iterable[T]] = openIfHead match { // TODO: it might be good to apply flatMaps locally, // since we obviously need to iterate all, @@ -929,12 +927,18 @@ class TypedPipeInst[T] private[scalding] (@transient inpipe: Pipe, // for us. So unwind until you hit the first filter, snapshot, // then apply the unwound functions case Some((tap, fields, Converter(conv))) => - Execution.from(new Iterable[T] { - def iterator = m.openForRead(conf, tap).asScala.map(tup => conv(tup.selectEntry(fields))) - }) + // To convert from java iterator to scala below + import scala.collection.JavaConverters._ + Execution.getConfigMode.map { + case (conf, m) => + // Verify the mode has not changed due to invalid TypedPipe DAG construction + checkMode(m) + new Iterable[T] { + def iterator = m.openForRead(conf, tap).asScala.map(tup => conv(tup.selectEntry(fields))) + } + } case _ => forceToDiskExecution.flatMap(_.toIterableExecution) } - } } final case class MergedTypedPipe[T](left: TypedPipe[T], right: TypedPipe[T]) extends TypedPipe[T] { @@ -966,14 +970,6 @@ final case class MergedTypedPipe[T](left: TypedPipe[T], right: TypedPipe[T]) ext override def fork: TypedPipe[T] = MergedTypedPipe(left.fork, right.fork) - /* - * This relies on the fact that two executions that are zipped will run in the - * same cascading flow, so we don't have to worry about it here. - */ - override def forceToDiskExecution = - left.forceToDiskExecution.zip(right.forceToDiskExecution) - .map { case (l, r) => l ++ r } - @annotation.tailrec private def flattenMerge(toFlatten: List[TypedPipe[T]], acc: List[TypedPipe[T]])(implicit fd: FlowDef, m: Mode): List[TypedPipe[T]] = toFlatten match { @@ -1010,14 +1006,7 @@ final case class MergedTypedPipe[T](left: TypedPipe[T], right: TypedPipe[T]) ext } } - /** - * This relies on the fact that two executions that are zipped will run in the - * same cascading flow, so we don't have to worry about it here. - */ - override def toIterableExecution: Execution[Iterable[T]] = - left.toIterableExecution.zip(right.toIterableExecution) - .map { case (l, r) => l ++ r } - + override def toIterableExecution: Execution[Iterable[T]] = forceToDiskExecution.flatMap(_.toIterableExecution) override def hashCogroup[K, V, W, R](smaller: HashJoinable[K, W])(joiner: (K, V, Iterable[W]) => Iterator[R])(implicit ev: TypedPipe[T] <:< TypedPipe[(K, V)]): TypedPipe[(K, R)] = MergedTypedPipe(left.hashCogroup(smaller)(joiner), right.hashCogroup(smaller)(joiner)) } From 4d7944854b1254f43d79d82ec40a1ad5b0c9428d Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Mon, 13 Apr 2015 09:27:35 -1000 Subject: [PATCH 098/177] Make fromTry parameter lazy --- .../src/main/scala/com/twitter/scalding/Execution.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala b/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala index 51b2ecb1e7..4747036386 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala @@ -427,7 +427,7 @@ object Execution { * time run is called. */ def from[T](t: => T): Execution[T] = fromTry(Try(t)) - def fromTry[T](t: Try[T]): Execution[T] = fromFuture { _ => toFuture(t) } + def fromTry[T](t: => Try[T]): Execution[T] = fromFuture { _ => toFuture(t) } /** * The call to fn will happen when the run method on the result is called. From 67d9be2a1a422cada52473a13d89e708a6518b6e Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Mon, 13 Apr 2015 10:34:31 -1000 Subject: [PATCH 099/177] Fix Config.getUniqueIDs --- .../src/main/scala/com/twitter/scalding/Config.scala | 6 ++++-- .../test/scala/com/twitter/scalding/ConfigTest.scala | 11 +++++++++++ 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala index 29137a2747..e7057c54bc 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala @@ -209,8 +209,10 @@ trait Config { // This is setting a property for cascading/driven (AppProps.APP_FRAMEWORKS -> ("scalding:" + scaldingVersion.toString))) - def getUniqueId: Option[UniqueID] = - get(UniqueID.UNIQUE_JOB_ID).map(UniqueID(_)) + def getUniqueIds: Set[UniqueID] = + get(UniqueID.UNIQUE_JOB_ID) + .map { str => str.split(",").toSet[String].map(UniqueID(_)) } + .getOrElse(Set.empty) /** * The serialization of your data will be smaller if any classes passed between tasks in your job diff --git a/scalding-core/src/test/scala/com/twitter/scalding/ConfigTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/ConfigTest.scala index a7543f991c..39a1552ae2 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/ConfigTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/ConfigTest.scala @@ -45,6 +45,11 @@ class ConfigTest extends WordSpec with Matchers { stillOld should contain (date) new2 shouldBe newConf } + "adding UniqueIDs works" in { + assert(Config.empty.getUniqueIds.size === 0) + val (id, conf) = Config.empty.ensureUniqueId + assert(conf.getUniqueIds === (Set(id))) + } "Default serialization should have tokens" in { Config.default.getCascadingSerializationTokens should not be empty Config.default.getCascadingSerializationTokens @@ -90,4 +95,10 @@ object ConfigProps extends Properties("Config") { val testKeys = c1.toMap.keySet | c2.toMap.keySet ++ keys testKeys.forall { k => merged.get(k) == c2.get(k).orElse(c1.get(k)) } } + property("adding many UniqueIDs works") = forAll { (l: List[String]) => + val uids = l.filterNot { s => s.isEmpty || s.contains(",") }.map(UniqueID(_)) + (uids.foldLeft(Config.empty) { (conf, id) => + conf.addUniqueId(id) + }.getUniqueIds == uids.toSet) + } } From 5a7332c21624dd67b65033a58e40a8dc78c5d85d Mon Sep 17 00:00:00 2001 From: Kevin Lin Date: Mon, 13 Apr 2015 17:22:08 -0700 Subject: [PATCH 100/177] Use compile time check. --- .../main/scala/com/twitter/scalding/typed/Sketched.scala | 4 ++-- .../scala/com/twitter/scalding/typed/WithReducers.scala | 6 +----- 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/Sketched.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/Sketched.scala index a0f323df31..ef7dddd7fd 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/Sketched.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/Sketched.scala @@ -58,7 +58,7 @@ case class Sketched[K, V](pipe: TypedPipe[(K, V)], def serialize(k: K): Array[Byte] = serialization(k) - override def reducers = Some(numReducers) + def reducers = Some(numReducers) private lazy implicit val cms = CMS.monoid[Array[Byte]](eps, delta, seed) lazy val sketch: TypedPipe[CMS[Array[Byte]]] = @@ -93,7 +93,7 @@ case class SketchJoined[K: Ordering, V, V2, R](left: Sketched[K, V], numReducers: Int)(joiner: (K, V, Iterable[V2]) => Iterator[R]) extends MustHasReducers { - override def reducers = Some(numReducers) + def reducers = Some(numReducers) //the most of any one reducer we want to try to take up with a single key private val maxReducerFraction = 0.1 diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/WithReducers.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/WithReducers.scala index 310a1c8ba9..321cd80bc3 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/WithReducers.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/WithReducers.scala @@ -28,11 +28,7 @@ trait HasReducers { * e.g. Sketched */ trait MustHasReducers { - def reducers: Option[Int] - if (!reducers.isDefined) { - def className = this.getClass.getSimpleName - throw new IllegalArgumentException(s"Number of reducers must be specified for class ${className}.") - } + def reducers: Some[Int] } /** From 5cf01f63b63612303e22376691a9803221498d5d Mon Sep 17 00:00:00 2001 From: Kevin Lin Date: Mon, 13 Apr 2015 23:48:54 -0700 Subject: [PATCH 101/177] MustHaveReducers extends HasReducers. --- .../src/main/scala/com/twitter/scalding/typed/Sketched.scala | 4 ++-- .../main/scala/com/twitter/scalding/typed/WithReducers.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/Sketched.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/Sketched.scala index ef7dddd7fd..2a374bd428 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/Sketched.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/Sketched.scala @@ -53,7 +53,7 @@ case class Sketched[K, V](pipe: TypedPipe[(K, V)], eps: Double, seed: Int)(implicit serialization: K => Array[Byte], ordering: Ordering[K]) - extends MustHasReducers { + extends MustHaveReducers[Sketched[K, V]] { import Sketched._ def serialize(k: K): Array[Byte] = serialization(k) @@ -91,7 +91,7 @@ case class Sketched[K, V](pipe: TypedPipe[(K, V)], case class SketchJoined[K: Ordering, V, V2, R](left: Sketched[K, V], right: TypedPipe[(K, V2)], numReducers: Int)(joiner: (K, V, Iterable[V2]) => Iterator[R]) - extends MustHasReducers { + extends MustHaveReducers[SketchJoined[K, V, V2, R]] { def reducers = Some(numReducers) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/WithReducers.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/WithReducers.scala index 321cd80bc3..dea42b36ba 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/WithReducers.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/WithReducers.scala @@ -27,7 +27,7 @@ trait HasReducers { * used for types that must know how many reducers they need * e.g. Sketched */ -trait MustHasReducers { +trait MustHaveReducers[+This <: MustHaveReducers[This]] extends HasReducers { def reducers: Some[Int] } From f146f78812fd75faf55a67cb4b87701b18c252da Mon Sep 17 00:00:00 2001 From: Kevin Lin Date: Tue, 14 Apr 2015 11:19:51 -0700 Subject: [PATCH 102/177] Remove F-bounding. --- .../src/main/scala/com/twitter/scalding/typed/Sketched.scala | 4 ++-- .../main/scala/com/twitter/scalding/typed/WithReducers.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/Sketched.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/Sketched.scala index 2a374bd428..407a586cbd 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/Sketched.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/Sketched.scala @@ -53,7 +53,7 @@ case class Sketched[K, V](pipe: TypedPipe[(K, V)], eps: Double, seed: Int)(implicit serialization: K => Array[Byte], ordering: Ordering[K]) - extends MustHaveReducers[Sketched[K, V]] { + extends MustHaveReducers { import Sketched._ def serialize(k: K): Array[Byte] = serialization(k) @@ -91,7 +91,7 @@ case class Sketched[K, V](pipe: TypedPipe[(K, V)], case class SketchJoined[K: Ordering, V, V2, R](left: Sketched[K, V], right: TypedPipe[(K, V2)], numReducers: Int)(joiner: (K, V, Iterable[V2]) => Iterator[R]) - extends MustHaveReducers[SketchJoined[K, V, V2, R]] { + extends MustHaveReducers { def reducers = Some(numReducers) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/WithReducers.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/WithReducers.scala index dea42b36ba..ff4c350236 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/WithReducers.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/WithReducers.scala @@ -27,7 +27,7 @@ trait HasReducers { * used for types that must know how many reducers they need * e.g. Sketched */ -trait MustHaveReducers[+This <: MustHaveReducers[This]] extends HasReducers { +trait MustHaveReducers extends HasReducers { def reducers: Some[Int] } From f7ad7a75112ca710f727d17c23a7c5790d611fa5 Mon Sep 17 00:00:00 2001 From: beethere Date: Sun, 12 Apr 2015 11:18:49 +0200 Subject: [PATCH 103/177] Typed Parquet Tuple #1198 *Improve tuple converter macro(delete unnecessary boxing) --- project/Build.scala | 5 +- .../impl/ParquetTupleConverterProvider.scala | 82 ++++++--- .../tuple/scheme/ParquetTupleConverter.scala | 159 ++++++++++++------ .../parquet/tuple/TypedParquetTupleTest.scala | 6 +- .../parquet/tuple/macros/MacroUnitTests.scala | 5 +- 5 files changed, 181 insertions(+), 76 deletions(-) diff --git a/project/Build.scala b/project/Build.scala index 89aa7ebb43..ec0f6c2ec5 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -33,6 +33,7 @@ object ScaldingBuild extends Build { val hravenVersion = "0.9.13" val jacksonVersion = "2.4.2" val json4SVersion = "3.2.11" + val paradiseVersion = "2.0.1" val parquetVersion = "1.6.0rc4" val protobufVersion = "2.4.1" val quasiquotesVersion = "2.0.1" @@ -316,7 +317,7 @@ object ScaldingBuild extends Build { "org.scala-lang" % "scala-reflect" % scalaVersion, "com.twitter" %% "bijection-macros" % bijectionVersion ) ++ (if(isScala210x(scalaVersion)) Seq("org.scalamacros" %% "quasiquotes" % quasiquotesVersion) else Seq()) - }, addCompilerPlugin("org.scalamacros" % "paradise" % "2.0.1" cross CrossVersion.full)) + }, addCompilerPlugin("org.scalamacros" % "paradise" % paradiseVersion cross CrossVersion.full)) .dependsOn(scaldingCore, scaldingHadoopTest) def scaldingParquetScroogeDeps(version: String) = { @@ -429,7 +430,7 @@ object ScaldingBuild extends Build { "com.twitter" %% "bijection-macros" % bijectionVersion ) ++ (if(isScala210x(scalaVersion)) Seq("org.scalamacros" %% "quasiquotes" % quasiquotesVersion) else Seq()) }, - addCompilerPlugin("org.scalamacros" % "paradise" % "2.0.1" cross CrossVersion.full) + addCompilerPlugin("org.scalamacros" % "paradise" % paradiseVersion cross CrossVersion.full) ).dependsOn(scaldingCore, scaldingHadoopTest) // This one uses a different naming convention diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/ParquetTupleConverterProvider.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/ParquetTupleConverterProvider.scala index c9f98fc925..19928078eb 100644 --- a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/ParquetTupleConverterProvider.scala +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/ParquetTupleConverterProvider.scala @@ -1,7 +1,7 @@ package com.twitter.scalding.parquet.tuple.macros.impl import com.twitter.bijection.macros.impl.IsCaseClassImpl -import com.twitter.scalding.parquet.tuple.scheme.ParquetTupleConverter +import com.twitter.scalding.parquet.tuple.scheme._ import scala.reflect.macros.Context @@ -15,57 +15,90 @@ object ParquetTupleConverterProvider { either it is not a case class or this macro call is possibly enclosed in a class. This will mean the macro is operating on a non-resolved type.""") - def buildGroupConverter(tpe: Type, parentTree: Tree, isOption: Boolean, idx: Int, converterBodyTree: Tree): Tree = { - q"""new _root_.com.twitter.scalding.parquet.tuple.scheme.ParquetTupleConverter($parentTree, $isOption, $idx){ - override def newConverter(i: Int): _root_.parquet.io.api.Converter = { + def buildGroupConverter(tpe: Type, parentTree: Tree, isOption: Boolean, idx: Int, converterBodyTree: Tree, + valueBuilder: Tree): Tree = { + q"""new _root_.com.twitter.scalding.parquet.tuple.scheme.ParquetTupleConverter($parentTree){ + override def newConverter(i: Int): _root_.com.twitter.scalding.parquet.tuple.scheme.TupleFieldConverter[Any] = { $converterBodyTree throw new RuntimeException("invalid index: " + i) } override def createValue(): Any = { - if(fieldValues.isEmpty) null - else classOf[$tpe].getConstructors()(0).newInstance(fieldValues.toSeq.map(_.asInstanceOf[AnyRef]): _*) + $valueBuilder } }""" } - def matchField(idx: Int, fieldType: Type, isOption: Boolean): List[Tree] = { + def matchField(idx: Int, fieldType: Type, isOption: Boolean): (Tree, Tree) = { def createConverter(converter: Tree): Tree = q"if($idx == i) return $converter" + def primitiveFieldValue(converterType: Type): Tree = if (isOption) { + val cachedRes = newTermName(ctx.fresh(s"fieldValue")) + q""" + { + val $cachedRes = converters($idx).asInstanceOf[$converterType] + if($cachedRes.hasValue) Some($cachedRes.currentValue) else _root_.scala.Option.empty[$fieldType] + } + """ + } else { + q"converters($idx).asInstanceOf[$converterType].currentValue" + } + + def primitiveFieldConverterAndFieldValue(converterType: Type): (Tree, Tree) = { + val companion = converterType.typeSymbol.companionSymbol + (createConverter(q"$companion(this)"), primitiveFieldValue(converterType)) + } + + def caseClassFieldValue: Tree = if (isOption) { + val cachedRes = newTermName(ctx.fresh(s"fieldValue")) + q""" + { + val $cachedRes = converters($idx) + if($cachedRes.hasValue) Some($cachedRes.currentValue.asInstanceOf[$fieldType]) + else _root_.scala.Option.empty[$fieldType] + } + """ + } else { + q"converters($idx).currentValue.asInstanceOf[$fieldType]" + } + fieldType match { case tpe if tpe =:= typeOf[String] => - List(createConverter(q"new _root_.com.twitter.scalding.parquet.tuple.scheme.StringConverter($idx, this, $isOption)")) + primitiveFieldConverterAndFieldValue(typeOf[StringConverter]) case tpe if tpe =:= typeOf[Boolean] => - List(createConverter(q"new _root_.com.twitter.scalding.parquet.tuple.scheme.BooleanConverter($idx, this, $isOption)")) + primitiveFieldConverterAndFieldValue(typeOf[BooleanConverter]) case tpe if tpe =:= typeOf[Byte] => - List(createConverter(q"new _root_.com.twitter.scalding.parquet.tuple.scheme.ByteConverter($idx, this, $isOption)")) + primitiveFieldConverterAndFieldValue(typeOf[ByteConverter]) case tpe if tpe =:= typeOf[Short] => - List(createConverter(q"new _root_.com.twitter.scalding.parquet.tuple.scheme.ShortConverter($idx, this, $isOption)")) + primitiveFieldConverterAndFieldValue(typeOf[ShortConverter]) case tpe if tpe =:= typeOf[Int] => - List(createConverter(q"new _root_.com.twitter.scalding.parquet.tuple.scheme.IntConverter($idx, this, $isOption)")) + primitiveFieldConverterAndFieldValue(typeOf[IntConverter]) case tpe if tpe =:= typeOf[Long] => - List(createConverter(q"new _root_.com.twitter.scalding.parquet.tuple.scheme.LongConverter($idx, this, $isOption)")) + primitiveFieldConverterAndFieldValue(typeOf[LongConverter]) case tpe if tpe =:= typeOf[Float] => - List(createConverter(q"new _root_.com.twitter.scalding.parquet.tuple.scheme.FloatConverter($idx, this, $isOption)")) + primitiveFieldConverterAndFieldValue(typeOf[FloatConverter]) case tpe if tpe =:= typeOf[Double] => - List(createConverter(q"new _root_.com.twitter.scalding.parquet.tuple.scheme.DoubleConverter($idx, this, $isOption)")) + primitiveFieldConverterAndFieldValue(typeOf[DoubleConverter]) case tpe if tpe.erasure =:= typeOf[Option[Any]] => val innerType = tpe.asInstanceOf[TypeRefApi].args.head matchField(idx, innerType, isOption = true) case tpe if IsCaseClassImpl.isCaseClassType(ctx)(tpe) => - val innerConverterTrees = buildConverterBody(tpe, expandMethod(tpe)) - List(createConverter(buildGroupConverter(tpe, q"Option(this)", isOption, idx, innerConverterTrees))) + val (innerConverters, innerFieldValues) = expandMethod(tpe).unzip + val innerConverterTree = buildConverterBody(tpe, innerConverters) + val innerValueBuilderTree = buildTupleValue(tpe, innerFieldValues) + val innerGroupConverter = createConverter(buildGroupConverter(tpe, q"Option(this)", isOption, idx, innerConverterTree, innerValueBuilderTree)) + (innerGroupConverter, caseClassFieldValue) case _ => ctx.abort(ctx.enclosingPosition, s"Case class $T is not pure primitives or nested case classes") } } - def expandMethod(outerTpe: Type): List[Tree] = { + def expandMethod(outerTpe: Type): List[(Tree, Tree)] = { outerTpe .declarations .collect { case m: MethodSymbol if m.isCaseAccessor => m } .zipWithIndex - .flatMap { + .map { case (accessorMethod, idx) => val fieldType = accessorMethod.returnType matchField(idx, fieldType, isOption = false) @@ -82,7 +115,16 @@ object ParquetTupleConverterProvider { } } - val groupConverter = buildGroupConverter(T.tpe, q"None", isOption = false, -1, buildConverterBody(T.tpe, expandMethod(T.tpe))) + def buildTupleValue(tpe: Type, fieldValueBuilders: List[Tree]): Tree = { + if (fieldValueBuilders.isEmpty) + ctx.abort(ctx.enclosingPosition, s"Case class $tpe has no primitive types we were able to extract") + val companion = tpe.typeSymbol.companionSymbol + q"$companion(..$fieldValueBuilders)" + } + + val (converters, fieldValues) = expandMethod(T.tpe).unzip + val groupConverter = buildGroupConverter(T.tpe, q"None", isOption = false, -1, buildConverterBody(T.tpe, converters), + buildTupleValue(T.tpe, fieldValues)) ctx.Expr[ParquetTupleConverter](q""" $groupConverter diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/ParquetTupleConverter.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/ParquetTupleConverter.scala index 5dea340985..43615de045 100644 --- a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/ParquetTupleConverter.scala +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/ParquetTupleConverter.scala @@ -1,26 +1,44 @@ package com.twitter.scalding.parquet.tuple.scheme import parquet.io.api.{ Binary, Converter, GroupConverter, PrimitiveConverter } - -import scala.collection.mutable import scala.util.Try +/** + * Parquet tuple field converter used by parquet read support to read tuple field value from parquet. + * @tparam T field value type. + */ +trait TupleFieldConverter[+T] extends Converter { + /** + * Current value read from parquet column + */ + def currentValue: T + + /** + * Used to check if a optional field has value stored or not. + */ + var hasValue: Boolean = false + + /** + *reset the converter state, make it ready for reading next column value. + */ + def reset(): Unit +} + /** * Parquet tuple converter used to create user defined tuple value from parquet column values * @param parent parent parquet tuple converter - * @param isOption is the field optional - * @param outerIndex field index in parent tuple schema */ -abstract class ParquetTupleConverter(val parent: Option[ParquetTupleConverter] = None, val isOption: Boolean = false, - val outerIndex: Int = -1) extends GroupConverter { - var converters: Map[Int, Converter] = Map() - val fieldValues: mutable.ArrayBuffer[Any] = mutable.ArrayBuffer() +abstract class ParquetTupleConverter(val parent: Option[ParquetTupleConverter] = None) extends GroupConverter + with TupleFieldConverter[Any] { + var converters: Map[Int, TupleFieldConverter[Any]] = Map() + + var value: Any = null - var currentValue: Any = null + override def currentValue: Any = value def createValue(): Any - def newConverter(i: Int): Converter + def newConverter(i: Int): TupleFieldConverter[Any] override def getConverter(i: Int) = { val converter = converters.get(i) @@ -33,62 +51,107 @@ abstract class ParquetTupleConverter(val parent: Option[ParquetTupleConverter] = } override def end(): Unit = { - currentValue = createValue() - fieldValues.remove(0, fieldValues.size) - parent.map(_.addFieldValue(outerIndex, currentValue, isOption)) + if (hasValue) { + value = createValue() + parent.map(p => p.hasValue = true) + } } - override def start(): Unit = () - - def addFieldValue(index: Int, value: Any, isOpt: Boolean) = { - val currentSize = fieldValues.size - //insert none for these optional fields that has non value written for given row - (currentSize until index).map(fieldValues.insert(_, None)) - if (isOpt) fieldValues.insert(index, Option(value)) else fieldValues.insert(index, value) + override def reset(): Unit = { + value = null + converters.values.map(v => v.reset()) + hasValue = false } + + override def start(): Unit = reset() } -class PrimitiveTypeConverter(val index: Int, val parent: ParquetTupleConverter, val isOption: Boolean) - extends PrimitiveConverter { - def appendValue(value: Any) = parent.addFieldValue(index, value, isOption) +sealed trait PrimitiveTupleFieldConverter[T] extends TupleFieldConverter[T] { + val parent: ParquetTupleConverter + val defaultValue: T + var value: T = defaultValue + + override def currentValue: T = value + + protected def valueAdded(): Unit = { + hasValue = true + parent.hasValue = true + } + + override def reset(): Unit = { + value = defaultValue + hasValue = false + } } -class StringConverter(index: Int, parent: ParquetTupleConverter, isOption: Boolean = false) - extends PrimitiveTypeConverter(index, parent, isOption) { - override def addBinary(value: Binary): Unit = appendValue(value.toStringUsingUTF8) +case class StringConverter(parent: ParquetTupleConverter) extends PrimitiveConverter with PrimitiveTupleFieldConverter[String] { + override val defaultValue: String = null + + override def addBinary(binary: Binary): Unit = { + value = binary.toStringUsingUTF8 + valueAdded() + } } -class DoubleConverter(index: Int, parent: ParquetTupleConverter, isOption: Boolean = false) - extends PrimitiveTypeConverter(index, parent, isOption) { - override def addDouble(value: Double): Unit = appendValue(value) +case class DoubleConverter(parent: ParquetTupleConverter) extends PrimitiveConverter with PrimitiveTupleFieldConverter[Double] { + override val defaultValue: Double = 0D + + override def addDouble(v: Double): Unit = { + value = v + valueAdded() + } } -class FloatConverter(index: Int, parent: ParquetTupleConverter, isOption: Boolean = false) - extends PrimitiveTypeConverter(index, parent, isOption) { - override def addFloat(value: Float): Unit = appendValue(value) +case class FloatConverter(parent: ParquetTupleConverter) extends PrimitiveConverter with PrimitiveTupleFieldConverter[Float] { + override val defaultValue: Float = 0F + + override def addFloat(v: Float): Unit = { + value = v + valueAdded() + } } -class LongConverter(index: Int, parent: ParquetTupleConverter, isOption: Boolean = false) - extends PrimitiveTypeConverter(index, parent, isOption) { - override def addLong(value: Long) = appendValue(value) +case class LongConverter(parent: ParquetTupleConverter) extends PrimitiveConverter with PrimitiveTupleFieldConverter[Long] { + override val defaultValue: Long = 0L + + override def addLong(v: Long): Unit = { + value = v + valueAdded() + } } -class IntConverter(index: Int, parent: ParquetTupleConverter, isOption: Boolean = false) - extends PrimitiveTypeConverter(index, parent, isOption) { - override def addInt(value: Int) = appendValue(value) +case class IntConverter(parent: ParquetTupleConverter) extends PrimitiveConverter with PrimitiveTupleFieldConverter[Int] { + override val defaultValue: Int = 0 + + override def addInt(v: Int): Unit = { + value = v + valueAdded() + } } -class ShortConverter(index: Int, parent: ParquetTupleConverter, isOption: Boolean = false) - extends PrimitiveTypeConverter(index, parent, isOption) { - override def addInt(value: Int) = appendValue(Try(value.toShort).getOrElse(null)) +case class ShortConverter(parent: ParquetTupleConverter) extends PrimitiveConverter with PrimitiveTupleFieldConverter[Short] { + override val defaultValue: Short = 0 + + override def addInt(v: Int): Unit = { + value = Try(v.toShort).getOrElse(0) + valueAdded() + } } -class ByteConverter(index: Int, parent: ParquetTupleConverter, isOption: Boolean = false) - extends PrimitiveTypeConverter(index, parent, isOption) { - override def addInt(value: Int) = appendValue(Try(value.toByte).getOrElse(null)) +case class ByteConverter(parent: ParquetTupleConverter) extends PrimitiveConverter with PrimitiveTupleFieldConverter[Byte] { + override val defaultValue: Byte = 0 + + override def addInt(v: Int): Unit = { + value = Try(v.toByte).getOrElse(0) + valueAdded() + } } -class BooleanConverter(index: Int, parent: ParquetTupleConverter, isOption: Boolean = false) - extends PrimitiveTypeConverter(index, parent, isOption) { - override def addBoolean(value: Boolean) = appendValue(value) -} \ No newline at end of file +case class BooleanConverter(parent: ParquetTupleConverter) extends PrimitiveConverter with PrimitiveTupleFieldConverter[Boolean] { + override val defaultValue: Boolean = false + + override def addBoolean(v: Boolean): Unit = { + value = v + valueAdded() + } +} diff --git a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala index a9dd13d8bc..ce7d611af0 100644 --- a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala +++ b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala @@ -9,8 +9,8 @@ import com.twitter.scalding.typed.TypedPipe import com.twitter.scalding.{ Args, Job, TypedTsv } import org.scalatest.{ Matchers, WordSpec } import parquet.filter2.predicate.FilterApi.binaryColumn -import parquet.filter2.predicate.{FilterApi, FilterPredicate} -import parquet.io.api.{RecordConsumer, Binary} +import parquet.filter2.predicate.{ FilterApi, FilterPredicate } +import parquet.io.api.{ RecordConsumer, Binary } import parquet.schema.MessageType class TypedParquetTupleTest extends WordSpec with Matchers with HadoopPlatformTest { @@ -77,7 +77,7 @@ class CReadSupport extends ParquetReadSupport[SampleClassC] { class WriteSupport extends ParquetWriteSupport[SampleClassB] { override val rootSchema: String = SampleClassB.schema - override def writeRecord(r: SampleClassB, rc: RecordConsumer, schema: MessageType):Unit = + override def writeRecord(r: SampleClassB, rc: RecordConsumer, schema: MessageType): Unit = Macros.caseClassWriteSupport[SampleClassB](r, rc, schema) } diff --git a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/macros/MacroUnitTests.scala b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/macros/MacroUnitTests.scala index 242f256b45..06843438ea 100644 --- a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/macros/MacroUnitTests.scala +++ b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/macros/MacroUnitTests.scala @@ -1,8 +1,8 @@ package com.twitter.scalding.parquet.tuple.macros import org.scalatest.mock.MockitoSugar -import org.scalatest.{Matchers, WordSpec} -import parquet.io.api.{Binary, RecordConsumer} +import org.scalatest.{ Matchers, WordSpec } +import parquet.io.api.{ Binary, RecordConsumer } import parquet.schema.MessageTypeParser case class SampleClassA(x: Int, y: String) @@ -182,7 +182,6 @@ class MacroUnitTests extends WordSpec with Matchers with MockitoSugar { } } - "Macro case class parquet write support generator" should { "Generate write support for class with all the primitive type fields" in { val writeSupportFn = Macros.caseClassWriteSupport[SampleClassE] From e2528da9e96550678a3009df89baee68ded0f0dc Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Tue, 14 Apr 2015 18:08:25 -0700 Subject: [PATCH 104/177] add cache for addFlow --- .../scala/com/twitter/scalding/Stats.scala | 24 ++++++++++--------- 1 file changed, 13 insertions(+), 11 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Stats.scala b/scalding-core/src/main/scala/com/twitter/scalding/Stats.scala index 1d538d17f4..4c91e70a28 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Stats.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Stats.scala @@ -122,18 +122,20 @@ object RuntimeStats extends java.io.Serializable { } } + private[this] var prevFP: FlowProcess[_] = null def addFlowProcess(fp: FlowProcess[_]) { - val uniqueJobIdObj = fp.getProperty(UniqueID.UNIQUE_JOB_ID) - if (uniqueJobIdObj != null) { - // for speed concern, use a while loop instead of foreach here - val splitted = StringUtility.fastSplit(uniqueJobIdObj.asInstanceOf[String], ",") - val size = splitted.length - var i = 0 - while (i < size) { - val uniqueId = splitted(i) - logger.debug("Adding flow process id: " + uniqueId) - flowMappingStore.getOrElse(uniqueId, new WeakReference(fp)) - i += 1 + if (prevFP != fp) { + val uniqueJobIdObj = fp.getProperty(UniqueID.UNIQUE_JOB_ID) + if (uniqueJobIdObj != null) { + // for speed concern, use a while loop instead of foreach here + var splitted = StringUtility.fastSplit(uniqueJobIdObj.asInstanceOf[String], ",") + while (!splitted.isEmpty) { + val uniqueId = splitted.head + splitted = splitted.tail + logger.debug("Adding flow process id: " + uniqueId) + flowMappingStore.put(uniqueId, new WeakReference(fp)) + prevFP = fp + } } } } From 5f8f3f2874d762f5b2e5b53e0375067d2f8921dd Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Tue, 14 Apr 2015 18:08:45 -0700 Subject: [PATCH 105/177] Deal with multiple characters separator --- .../scala/com/twitter/scalding/StringUtility.scala | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/StringUtility.scala b/scalding-core/src/main/scala/com/twitter/scalding/StringUtility.scala index 11a4ece041..1e421e0990 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/StringUtility.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/StringUtility.scala @@ -1,25 +1,21 @@ package com.twitter.scalding -import java.util - -import scala.collection.mutable.ArrayBuffer - object StringUtility { - private def fastSplitHelper(text: String, key: String, from: Int, textLength: Int): List[String] = { + private def fastSplitHelper(text: String, key: String, from: Int, textLength: Int, keyLength: Int): List[String] = { val firstIndex = text.indexOf(key, from) if (firstIndex == -1) { if (from < textLength) { List(text.substring(from)) } else { - List() + List("") } } else { // the text till the separator should be kept in any case - text.substring(from, firstIndex) :: fastSplitHelper(text, key, firstIndex + 1, textLength) + text.substring(from, firstIndex) :: fastSplitHelper(text, key, firstIndex + keyLength, textLength, keyLength) } } - def fastSplit(text: String, key: String): Seq[String] = { - fastSplitHelper(text, key, 0, text.length) + def fastSplit(text: String, key: String): List[String] = { + fastSplitHelper(text, key, 0, text.length, key.length) } } \ No newline at end of file From c0a7e2f54764a59f758d424df80c5280319fe3f4 Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Tue, 14 Apr 2015 18:09:43 -0700 Subject: [PATCH 106/177] Change to property test --- .../twitter/scalding/StringUtilityTest.scala | 42 ++++++++++++++++--- 1 file changed, 36 insertions(+), 6 deletions(-) diff --git a/scalding-core/src/test/scala/com/twitter/scalding/StringUtilityTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/StringUtilityTest.scala index 49acd4feed..8a5caadff6 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/StringUtilityTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/StringUtilityTest.scala @@ -1,6 +1,12 @@ package com.twitter.scalding -import org.scalatest.{ Matchers, WordSpec } +import org.scalatest.{ PropSpec, Matchers, WordSpec } +import org.scalacheck.{ Arbitrary, Properties } +import org.scalacheck.Prop.forAll +import org.scalatest.prop.Checkers +import org.scalacheck.Gen + +import scala.collection.mutable.ListBuffer class StringUtilityTest extends WordSpec with Matchers { "fastSplitTest" should { @@ -16,14 +22,14 @@ class StringUtilityTest extends WordSpec with Matchers { val text2 = "a:b:c:d:" val res2 = StringUtility.fastSplit(text2, ":") res2 should be { - Seq("a", "b", "c", "d") + Seq("a", "b", "c", "d", "") } } "be able to split only one separators" in { val text2 = "a@" val res2 = StringUtility.fastSplit(text2, "@") res2 should be { - Seq("a") + Seq("a", "") } } "be able to split when separator doesn't show up" in { @@ -33,6 +39,7 @@ class StringUtilityTest extends WordSpec with Matchers { Seq("a") } } + /* "be able to be faster than java's split function" in { // helper function to time def time[R](block: => R): Double = { @@ -88,8 +95,31 @@ class StringUtilityTest extends WordSpec with Matchers { (mean, std, median, s) } - // assert that total time for fastSplit is really faster here? - println("mean, std, median, and total time for running java's split" + meanAndStd(javaRunTimeList)) - println("mean, std, median, and total time for running java's split" + meanAndStd(fastSplitRunTimeList)) } + */ +} + +class StringUtilityPropertyTest extends PropSpec with Checkers { + val randomStringGen = for { + s <- Gen.pick(5, List.fill(100)(List("k", "l", "m", "x", "//.")).flatten) + + } yield s + + // test for one separator and two + val randomSeparator = for { + s <- Gen.oneOf("@@", "@", "x", "//.") + } yield s + + property("blah") { + check { + forAll(randomStringGen, randomSeparator) { + (str, separator) => + val t = str.mkString("") + val r1 = t.split(separator, -1).toList + val r2 = StringUtility.fastSplit(t, separator) + r1 == r2 + } + } + } + } From 1578d285ef70e2d0f718ab176342cbf08507ce39 Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Tue, 14 Apr 2015 18:13:57 -0700 Subject: [PATCH 107/177] Delete profiling --- .../twitter/scalding/StringUtilityTest.scala | 58 ------------------- 1 file changed, 58 deletions(-) diff --git a/scalding-core/src/test/scala/com/twitter/scalding/StringUtilityTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/StringUtilityTest.scala index 8a5caadff6..82ba7ccac2 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/StringUtilityTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/StringUtilityTest.scala @@ -39,64 +39,6 @@ class StringUtilityTest extends WordSpec with Matchers { Seq("a") } } - /* - "be able to be faster than java's split function" in { - // helper function to time - def time[R](block: => R): Double = { - val t0 = System.nanoTime() - val result = block // call-by-name - val t1 = System.nanoTime() - val timeDiff = (t1 - t0) - timeDiff - } - - def randomString(length: Int) = { - val possibleChars = "abcdefg|" - val nPossibleChar = possibleChars.length - val r = new scala.util.Random - val sb = new StringBuilder - for (i <- 1 to length) { - sb.append(possibleChars(r.nextInt(nPossibleChar))) - } - sb.toString - } - - // randomly test - // for loop is to run the functions multiple times - var javaRunTimeList = List[Double]() - var fastSplitRunTimeList = List[Double]() - for (i <- 1 to 100) { - val randomStrings: List[String] = (1 to 100000).map { - x => - randomString(50) - }.toList - val randomSeparatorIndex = scala.util.Random.nextInt(1) - val separator = "|"(randomSeparatorIndex).toString - - val fastSplitRunTime = time { - val splittedByFastSpliter = randomStrings.map { s => StringUtility.fastSplit(s, separator).toList } - } - fastSplitRunTimeList = fastSplitRunTime :: fastSplitRunTimeList - - val javaRunTime = time { - val splittedByRegex = randomStrings.map { s => s.split(separator).toList } - } - - javaRunTimeList = javaRunTime :: javaRunTimeList - - } - - def meanAndStd(list: List[Double]): (Double, Double, Double, Double) = { - val s = list.sum - val mean = s / list.size - val std = math.sqrt(list.map{ x => x * x }.sum / list.size - mean * mean) - val sorted = list.sorted - val median = sorted(list.length / 2) - (mean, std, median, s) - } - - } - */ } class StringUtilityPropertyTest extends PropSpec with Checkers { From d9a7acb97c9d82c35de84f702f187f4f1b48a99f Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Wed, 15 Apr 2015 08:03:22 -1000 Subject: [PATCH 108/177] Use NonFatal instead of Throwable --- .../src/main/scala/com/twitter/scalding/Execution.scala | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala b/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala index 4747036386..f8c28f4b15 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala @@ -232,8 +232,7 @@ object Execution { try { ExecutionContext.newContext(conf)(fd, mode).run } catch { - // Try our best to complete the future - case e: Throwable => Future.failed(e) + case NonFatal(e) => Future.failed(e) }) // Loop go() @@ -418,8 +417,7 @@ object Execution { /** * This creates a definitely failed Execution. */ - def failed(t: Throwable): Execution[Nothing] = - fromFuture(_ => Future.failed(t)) + def failed(t: Throwable): Execution[Nothing] = fromTry(Failure(t)) /** * This makes a constant execution that runs no job. From b468e4fda09d2c9871728c20ec4fd697c13cc2cd Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Wed, 15 Apr 2015 12:57:23 -0700 Subject: [PATCH 109/177] Move preFP out if statement. Use a proper name for property test --- README.md | 1 + scalding-core/src/main/scala/com/twitter/scalding/Stats.scala | 2 +- .../src/test/scala/com/twitter/scalding/StringUtilityTest.scala | 2 +- 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 7f86144067..70e656f1e0 100644 --- a/README.md +++ b/README.md @@ -136,6 +136,7 @@ Thanks for assistance and contributions: * Kevin Lin * Josh Attenberg * Juliet Hougland +* Eddie Xie A full list of [contributors](https://github.com/twitter/scalding/graphs/contributors) can be found on GitHub. diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Stats.scala b/scalding-core/src/main/scala/com/twitter/scalding/Stats.scala index 4c91e70a28..91aeac847e 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Stats.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Stats.scala @@ -134,9 +134,9 @@ object RuntimeStats extends java.io.Serializable { splitted = splitted.tail logger.debug("Adding flow process id: " + uniqueId) flowMappingStore.put(uniqueId, new WeakReference(fp)) - prevFP = fp } } + prevFP = fp } } diff --git a/scalding-core/src/test/scala/com/twitter/scalding/StringUtilityTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/StringUtilityTest.scala index 82ba7ccac2..d7be43c3d6 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/StringUtilityTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/StringUtilityTest.scala @@ -52,7 +52,7 @@ class StringUtilityPropertyTest extends PropSpec with Checkers { s <- Gen.oneOf("@@", "@", "x", "//.") } yield s - property("blah") { + property("fastSplit(s, sep) should match s.split(sep, -1) for non-regex sep") { check { forAll(randomStringGen, randomSeparator) { (str, separator) => From 0494a87d76e489f0e57a517a5f94e7a6e3e9f4fd Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Wed, 15 Apr 2015 12:59:34 -0700 Subject: [PATCH 110/177] Add @ as separator in test for completeness --- .../src/test/scala/com/twitter/scalding/StringUtilityTest.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scalding-core/src/test/scala/com/twitter/scalding/StringUtilityTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/StringUtilityTest.scala index d7be43c3d6..3961825fb8 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/StringUtilityTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/StringUtilityTest.scala @@ -43,7 +43,7 @@ class StringUtilityTest extends WordSpec with Matchers { class StringUtilityPropertyTest extends PropSpec with Checkers { val randomStringGen = for { - s <- Gen.pick(5, List.fill(100)(List("k", "l", "m", "x", "//.")).flatten) + s <- Gen.pick(5, List.fill(100)(List("k", "l", "m", "x", "//.", "@")).flatten) } yield s From 3fc4e45a6fab5285576aafc745f6587174b65a09 Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Wed, 15 Apr 2015 13:04:30 -0700 Subject: [PATCH 111/177] Use reference equality instead of structure equality --- scalding-core/src/main/scala/com/twitter/scalding/Stats.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Stats.scala b/scalding-core/src/main/scala/com/twitter/scalding/Stats.scala index 91aeac847e..5da2045889 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Stats.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Stats.scala @@ -124,7 +124,7 @@ object RuntimeStats extends java.io.Serializable { private[this] var prevFP: FlowProcess[_] = null def addFlowProcess(fp: FlowProcess[_]) { - if (prevFP != fp) { + if (!(prevFP eq fp)) { val uniqueJobIdObj = fp.getProperty(UniqueID.UNIQUE_JOB_ID) if (uniqueJobIdObj != null) { // for speed concern, use a while loop instead of foreach here From cd05d272a58756b826b58f3177d4fcef1477c85c Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Wed, 15 Apr 2015 11:06:03 -1000 Subject: [PATCH 112/177] Make sure the flow starting thread is a daemon --- .../com/twitter/scalding/Execution.scala | 38 +++++++++++++------ 1 file changed, 26 insertions(+), 12 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala b/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala index f8c28f4b15..9430887f67 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala @@ -225,15 +225,23 @@ object Execution { def run() { @annotation.tailrec def go(): Unit = messageQueue.take match { - case Stop => - () + case Stop => () case RunFlowDef(conf, mode, fd, promise) => - promise.completeWith( - try { - ExecutionContext.newContext(conf)(fd, mode).run - } catch { - case NonFatal(e) => Future.failed(e) - }) + try { + promise.completeWith( + ExecutionContext.newContext(conf)(fd, mode).run) + } catch { + case t: Throwable => + // something bad happened, but this thread is a daemon + // that should only stop if all others have stopped or + // we have received the stop message. + // Stopping this thread prematurely can deadlock + // futures from the promise we have. + // In a sense, this thread does not exist logically and + // must forward all exceptions to threads that requested + // this work be started. + promise.tryFailure(t) + } // Loop go() } @@ -243,18 +251,24 @@ object Execution { } }) - def runFlowDef(conf: Config, mode: Mode, fd: FlowDef): Future[JobStats] = { + def runFlowDef(conf: Config, mode: Mode, fd: FlowDef): Future[JobStats] = try { val promise = Promise[JobStats]() + val fut = promise.future messageQueue.put(RunFlowDef(conf, mode, fd, promise)) - promise.future + // Don't do any work after the .put call, we want no chance for exception + // after the put + fut } catch { case NonFatal(e) => Future.failed(e) } - } - def start(): Unit = thread.start() + def start(): Unit = { + // Make sure this thread can't keep us running if all others are gone + thread.setDaemon(true) + thread.start() + } /* * This is called after we are done submitting all jobs */ From e1e31970927a9db197eec320c7862ecd4e6298fc Mon Sep 17 00:00:00 2001 From: Ruban Monu Date: Thu, 16 Apr 2015 11:58:25 -0700 Subject: [PATCH 113/177] more helper methods in TimePathedSource --- .../twitter/scalding/TimePathedSource.scala | 45 ++++++++++++------- 1 file changed, 28 insertions(+), 17 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/TimePathedSource.scala b/scalding-core/src/main/scala/com/twitter/scalding/TimePathedSource.scala index 1379a66d65..cba5312786 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/TimePathedSource.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/TimePathedSource.scala @@ -31,6 +31,31 @@ object TimePathedSource { "%1$tm" -> Months(1)(tz), "%1$tY" -> Years(1)(tz)) .find { unitDur: (String, Duration) => pattern.contains(unitDur._1) } .map(_._2) + + def allPathsFor(pattern: String, duration: Option[Duration], dateRange: DateRange, tz: TimeZone): Iterable[String] = + duration.map { dur => + // This method is exhaustive, but too expensive for Cascading's JobConf writing. + dateRange.each(dur) + .map { dr: DateRange => + this.toPath(pattern, dr.start, tz) + } + }.getOrElse(Nil) + + // picks all read paths in the given daterange + def readPathsFor(pattern: String, dateRange: DateRange, tz: TimeZone): Iterable[String] = { + val stepSize = TimePathedSource.stepSize(pattern, DateOps.UTC) + this.allPathsFor(pattern, stepSize, dateRange, DateOps.UTC) + } + + // picks the write path based on daterange end + def writePathFor(pattern: String, dateRange: DateRange, tz: TimeZone): String = { + // TODO this should be required everywhere but works on read without it + // maybe in 0.9.0 be more strict + assert(pattern.takeRight(2) == "/*", "Pattern must end with /* " + pattern) + val lastSlashPos = pattern.lastIndexOf('/') + val stripped = pattern.slice(0, lastSlashPos) + this.toPath(stripped, dateRange.end, tz) + } } abstract class TimeSeqPathedSource(val patterns: Seq[String], val dateRange: DateRange, val tz: TimeZone) extends FileSource { @@ -48,15 +73,7 @@ abstract class TimeSeqPathedSource(val patterns: Seq[String], val dateRange: Dat TimePathedSource.stepSize(pattern, tz) protected def allPathsFor(pattern: String): Iterable[String] = - defaultDurationFor(pattern) - .map { dur => - // This method is exhaustive, but too expensive for Cascading's JobConf writing. - dateRange.each(dur) - .map { dr: DateRange => - TimePathedSource.toPath(pattern, dr.start, tz) - } - } - .getOrElse(Nil) + TimePathedSource.allPathsFor(pattern, defaultDurationFor(pattern), dateRange, tz) /** These are all the paths we will read for this data completely enumerated */ def allPaths: Iterable[String] = @@ -104,14 +121,8 @@ abstract class TimePathedSource(val pattern: String, tz: TimeZone) extends TimeSeqPathedSource(Seq(pattern), dateRange, tz) { //Write to the path defined by the end time: - override def hdfsWritePath = { - // TODO this should be required everywhere but works on read without it - // maybe in 0.9.0 be more strict - assert(pattern.takeRight(2) == "/*", "Pattern must end with /* " + pattern) - val lastSlashPos = pattern.lastIndexOf('/') - val stripped = pattern.slice(0, lastSlashPos) - TimePathedSource.toPath(stripped, dateRange.end, tz) - } + override def hdfsWritePath = TimePathedSource.writePathFor(pattern, dateRange, tz) + override def localPath = pattern } From 8a074f1e0a2fa671bb50b65dcf4264301a4adce5 Mon Sep 17 00:00:00 2001 From: Ruban Monu Date: Thu, 16 Apr 2015 12:07:59 -0700 Subject: [PATCH 114/177] make InputSizeReducerEstimator work for any CompositeTap --- .../reducer_estimation/InputSizeReducerEstimator.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/InputSizeReducerEstimator.scala b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/InputSizeReducerEstimator.scala index edfce5c246..221451c3a0 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/InputSizeReducerEstimator.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/InputSizeReducerEstimator.scala @@ -2,7 +2,7 @@ package com.twitter.scalding.reducer_estimation import scala.collection.JavaConverters._ import cascading.flow.FlowStep -import cascading.tap.{ Tap, MultiSourceTap } +import cascading.tap.{ Tap, CompositeTap } import cascading.tap.hadoop.Hfs import org.apache.hadoop.mapred.JobConf import org.slf4j.LoggerFactory @@ -26,7 +26,7 @@ class InputSizeReducerEstimator extends ReducerEstimator { private def unrollTaps(taps: Seq[Tap[_, _, _]]): Seq[Tap[_, _, _]] = taps.flatMap { - case multi: MultiSourceTap[_, _, _] => + case multi: CompositeTap[_] => unrollTaps(multi.getChildTaps.asScala.toSeq) case t => Seq(t) } From d1d3c50ac56bc27b911c39dea3bc470b260ae87c Mon Sep 17 00:00:00 2001 From: Ruban Monu Date: Thu, 16 Apr 2015 12:39:47 -0700 Subject: [PATCH 115/177] more comments --- .../com/twitter/scalding/TimePathedSource.scala | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/TimePathedSource.scala b/scalding-core/src/main/scala/com/twitter/scalding/TimePathedSource.scala index cba5312786..9d39a9d172 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/TimePathedSource.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/TimePathedSource.scala @@ -32,7 +32,10 @@ object TimePathedSource { .find { unitDur: (String, Duration) => pattern.contains(unitDur._1) } .map(_._2) - def allPathsFor(pattern: String, duration: Option[Duration], dateRange: DateRange, tz: TimeZone): Iterable[String] = + /** + * Gives all paths in the given daterange with windows based on the provided duration. + */ + def allPathsWithDuration(pattern: String, duration: Option[Duration], dateRange: DateRange, tz: TimeZone): Iterable[String] = duration.map { dur => // This method is exhaustive, but too expensive for Cascading's JobConf writing. dateRange.each(dur) @@ -41,13 +44,17 @@ object TimePathedSource { } }.getOrElse(Nil) - // picks all read paths in the given daterange + /** + * Gives all read paths in the given daterange. + */ def readPathsFor(pattern: String, dateRange: DateRange, tz: TimeZone): Iterable[String] = { val stepSize = TimePathedSource.stepSize(pattern, DateOps.UTC) - this.allPathsFor(pattern, stepSize, dateRange, DateOps.UTC) + this.allPathsWithDuration(pattern, stepSize, dateRange, DateOps.UTC) } - // picks the write path based on daterange end + /** + * Gives the write path based on daterange end. + */ def writePathFor(pattern: String, dateRange: DateRange, tz: TimeZone): String = { // TODO this should be required everywhere but works on read without it // maybe in 0.9.0 be more strict @@ -73,7 +80,7 @@ abstract class TimeSeqPathedSource(val patterns: Seq[String], val dateRange: Dat TimePathedSource.stepSize(pattern, tz) protected def allPathsFor(pattern: String): Iterable[String] = - TimePathedSource.allPathsFor(pattern, defaultDurationFor(pattern), dateRange, tz) + TimePathedSource.allPathsWithDuration(pattern, defaultDurationFor(pattern), dateRange, tz) /** These are all the paths we will read for this data completely enumerated */ def allPaths: Iterable[String] = From 0acde4839423e6941a39fec83090e060dfee1ea5 Mon Sep 17 00:00:00 2001 From: Ruban Monu Date: Thu, 16 Apr 2015 14:11:04 -0700 Subject: [PATCH 116/177] address review comments --- .../twitter/scalding/TimePathedSource.scala | 29 ++++++++++--------- 1 file changed, 15 insertions(+), 14 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/TimePathedSource.scala b/scalding-core/src/main/scala/com/twitter/scalding/TimePathedSource.scala index 9d39a9d172..8c438212d5 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/TimePathedSource.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/TimePathedSource.scala @@ -35,33 +35,31 @@ object TimePathedSource { /** * Gives all paths in the given daterange with windows based on the provided duration. */ - def allPathsWithDuration(pattern: String, duration: Option[Duration], dateRange: DateRange, tz: TimeZone): Iterable[String] = - duration.map { dur => - // This method is exhaustive, but too expensive for Cascading's JobConf writing. - dateRange.each(dur) - .map { dr: DateRange => - this.toPath(pattern, dr.start, tz) - } - }.getOrElse(Nil) + def allPathsWithDuration(pattern: String, duration: Duration, dateRange: DateRange, tz: TimeZone): Iterable[String] = + // This method is exhaustive, but too expensive for Cascading's JobConf writing. + dateRange.each(duration) + .map { dr: DateRange => + toPath(pattern, dr.start, tz) + } /** * Gives all read paths in the given daterange. */ def readPathsFor(pattern: String, dateRange: DateRange, tz: TimeZone): Iterable[String] = { - val stepSize = TimePathedSource.stepSize(pattern, DateOps.UTC) - this.allPathsWithDuration(pattern, stepSize, dateRange, DateOps.UTC) + TimePathedSource.stepSize(pattern, DateOps.UTC) match { + case Some(duration) => allPathsWithDuration(pattern, duration, dateRange, DateOps.UTC) + case None => sys.error(s"No suitable step size for pattern: $pattern") + } } /** * Gives the write path based on daterange end. */ def writePathFor(pattern: String, dateRange: DateRange, tz: TimeZone): String = { - // TODO this should be required everywhere but works on read without it - // maybe in 0.9.0 be more strict assert(pattern.takeRight(2) == "/*", "Pattern must end with /* " + pattern) val lastSlashPos = pattern.lastIndexOf('/') val stripped = pattern.slice(0, lastSlashPos) - this.toPath(stripped, dateRange.end, tz) + toPath(stripped, dateRange.end, tz) } } @@ -80,7 +78,10 @@ abstract class TimeSeqPathedSource(val patterns: Seq[String], val dateRange: Dat TimePathedSource.stepSize(pattern, tz) protected def allPathsFor(pattern: String): Iterable[String] = - TimePathedSource.allPathsWithDuration(pattern, defaultDurationFor(pattern), dateRange, tz) + defaultDurationFor(pattern) match { + case Some(duration) => TimePathedSource.allPathsWithDuration(pattern, duration, dateRange, tz) + case None => sys.error(s"No suitable step size for pattern: $pattern") + } /** These are all the paths we will read for this data completely enumerated */ def allPaths: Iterable[String] = From d7bbb1864820bb73f781e3d2c270eca91eb16123 Mon Sep 17 00:00:00 2001 From: Mansur Ashraf Date: Fri, 17 Apr 2015 16:35:20 -0700 Subject: [PATCH 117/177] Revert "Change RichDate to value class (AnyVal)" --- .../test/scala/com/twitter/scalding/KryoTest.scala | 6 +----- .../main/scala/com/twitter/scalding/RichDate.scala | 12 +++++++++++- .../test/scala/com/twitter/scalding/DateTest.scala | 5 +++++ 3 files changed, 17 insertions(+), 6 deletions(-) diff --git a/scalding-core/src/test/scala/com/twitter/scalding/KryoTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/KryoTest.scala index 036edaa0c6..a0c97a8112 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/KryoTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/KryoTest.scala @@ -149,11 +149,7 @@ class KryoTest extends WordSpec with Matchers { val myDate: RichDate = "1999-12-30T14" val simpleDate: java.util.Date = myDate.value val myDateRange = DateRange("2012-01-02", "2012-06-09") - - // to work with serialization - val newMyDate = myDate.asInstanceOf[AnyRef] - - singleRT(newMyDate) shouldBe myDate + singleRT(myDate) shouldBe myDate singleRT(simpleDate) shouldBe simpleDate singleRT(myDateRange) shouldBe myDateRange } diff --git a/scalding-date/src/main/scala/com/twitter/scalding/RichDate.scala b/scalding-date/src/main/scala/com/twitter/scalding/RichDate.scala index cbc1284165..a06a91d815 100644 --- a/scalding-date/src/main/scala/com/twitter/scalding/RichDate.scala +++ b/scalding-date/src/main/scala/com/twitter/scalding/RichDate.scala @@ -66,7 +66,7 @@ object RichDate { /** * A value class wrapper for milliseconds since the epoch */ -case class RichDate(val timestamp: Long) extends AnyVal with Ordered[RichDate] { +case class RichDate(val timestamp: Long) extends Ordered[RichDate] { // these are mutable, don't keep them around def value: Date = new java.util.Date(timestamp) @@ -79,6 +79,14 @@ case class RichDate(val timestamp: Long) extends AnyVal with Ordered[RichDate] { override def compare(that: RichDate): Int = Ordering[Long].compare(timestamp, that.timestamp) + //True of the other is a RichDate with equal value, or a Date equal to value + override def equals(that: Any) = + that match { + case d: Date => d.getTime == timestamp + case RichDate(ts) => ts == timestamp + case _ => false + } + /** * Use String.format to format the date, as opposed to toString with uses SimpleDateFormat */ @@ -89,6 +97,8 @@ case class RichDate(val timestamp: Long) extends AnyVal with Ordered[RichDate] { * to make them equal. this is the same as what java does (and only sane thing): * http://grepcode.com/file/repository.grepcode.com/java/root/jdk/openjdk/6-b14/java/util/Date.java#989 */ + override def hashCode = + (timestamp.toInt) ^ ((timestamp >> 32).toInt) def toCalendar(implicit tz: TimeZone) = { val cal = Calendar.getInstance(tz) diff --git a/scalding-date/src/test/scala/com/twitter/scalding/DateTest.scala b/scalding-date/src/test/scala/com/twitter/scalding/DateTest.scala index 79723d7952..543300d7df 100644 --- a/scalding-date/src/test/scala/com/twitter/scalding/DateTest.scala +++ b/scalding-date/src/test/scala/com/twitter/scalding/DateTest.scala @@ -66,6 +66,11 @@ class DateTest extends WordSpec { val rd2: RichDate = " 2010-10-02 T 00:00:01 " assert(rd1 === rd2) } + "Have same equals & hashCode as Date (crazy?)" in { + val rd1: RichDate = "2011-10-20" + assert(rd1 === rd1.value) + assert(rd1.hashCode === rd1.value.hashCode) + } "be well ordered" in { val rd1: RichDate = "2011-10-20" val rd2: RichDate = "2011-10-21" From 5bf642dd303d1d49b6189098e3402b713b5b255d Mon Sep 17 00:00:00 2001 From: Mansur Ashraf Date: Fri, 17 Apr 2015 16:40:59 -0700 Subject: [PATCH 118/177] added comments --- .../src/main/scala/com/twitter/scalding/RichDate.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/scalding-date/src/main/scala/com/twitter/scalding/RichDate.scala b/scalding-date/src/main/scala/com/twitter/scalding/RichDate.scala index a06a91d815..debc735d85 100644 --- a/scalding-date/src/main/scala/com/twitter/scalding/RichDate.scala +++ b/scalding-date/src/main/scala/com/twitter/scalding/RichDate.scala @@ -64,7 +64,8 @@ object RichDate { } /** - * A value class wrapper for milliseconds since the epoch + * A value class wrapper for milliseconds since the epoch. Its tempting to extend + * this with AnyVal but this causes problem with Java code. */ case class RichDate(val timestamp: Long) extends Ordered[RichDate] { // these are mutable, don't keep them around From 182cc1db7f6051cfc1dc723868818f931d973c07 Mon Sep 17 00:00:00 2001 From: Mansur Ashraf Date: Fri, 17 Apr 2015 16:52:07 -0700 Subject: [PATCH 119/177] switched to java.lang.Long.compare --- .../src/main/scala/com/twitter/scalding/RichDate.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scalding-date/src/main/scala/com/twitter/scalding/RichDate.scala b/scalding-date/src/main/scala/com/twitter/scalding/RichDate.scala index debc735d85..ebe1c77c68 100644 --- a/scalding-date/src/main/scala/com/twitter/scalding/RichDate.scala +++ b/scalding-date/src/main/scala/com/twitter/scalding/RichDate.scala @@ -78,7 +78,7 @@ case class RichDate(val timestamp: Long) extends Ordered[RichDate] { def -(that: RichDate) = AbsoluteDuration.fromMillisecs(timestamp - that.timestamp) override def compare(that: RichDate): Int = - Ordering[Long].compare(timestamp, that.timestamp) + java.lang.Long.compare(timestamp, that.timestamp) //True of the other is a RichDate with equal value, or a Date equal to value override def equals(that: Any) = From 5f06144e00b37143418b37c10ed41fd8212520ce Mon Sep 17 00:00:00 2001 From: beethere Date: Sat, 18 Apr 2015 18:08:25 +0200 Subject: [PATCH 120/177] Improve tuple converter macro *use two different classes for modeling required and optional field converter *delete unnecessary class cast, type all the field converter class --- .../parquet/tuple/macros/Macros.scala | 2 +- .../impl/ParquetTupleConverterProvider.scala | 131 ++++++++------- .../tuple/scheme/ParquetTupleConverter.scala | 159 ++++++++---------- .../scheme/TypedParquetTupleScheme.scala | 6 +- .../parquet/tuple/TypedParquetTupleTest.scala | 4 +- .../parquet/tuple/macros/MacroUnitTests.scala | 18 +- 6 files changed, 160 insertions(+), 160 deletions(-) diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/Macros.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/Macros.scala index acedb7a4d9..47f7b03097 100644 --- a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/Macros.scala +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/Macros.scala @@ -44,7 +44,7 @@ object Macros { * @tparam T Case class type that contains only primitive fields or nested case class. * @return Generated parquet converter */ - def caseClassParquetTupleConverter[T]: ParquetTupleConverter = macro ParquetTupleConverterProvider.toParquetTupleConverterImpl[T] + def caseClassParquetTupleConverter[T]: ParquetTupleConverter[T] = macro ParquetTupleConverterProvider.toParquetTupleConverterImpl[T] /** * Macro used to generate case class write support to parquet. diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/ParquetTupleConverterProvider.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/ParquetTupleConverterProvider.scala index 19928078eb..f226f6d6c4 100644 --- a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/ParquetTupleConverterProvider.scala +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/ParquetTupleConverterProvider.scala @@ -6,7 +6,7 @@ import com.twitter.scalding.parquet.tuple.scheme._ import scala.reflect.macros.Context object ParquetTupleConverterProvider { - def toParquetTupleConverterImpl[T](ctx: Context)(implicit T: ctx.WeakTypeTag[T]): ctx.Expr[ParquetTupleConverter] = { + def toParquetTupleConverterImpl[T](ctx: Context)(implicit T: ctx.WeakTypeTag[T]): ctx.Expr[ParquetTupleConverter[T]] = { import ctx.universe._ if (!IsCaseClassImpl.isCaseClassType(ctx)(T.tpe)) @@ -15,85 +15,101 @@ object ParquetTupleConverterProvider { either it is not a case class or this macro call is possibly enclosed in a class. This will mean the macro is operating on a non-resolved type.""") - def buildGroupConverter(tpe: Type, parentTree: Tree, isOption: Boolean, idx: Int, converterBodyTree: Tree, - valueBuilder: Tree): Tree = { - q"""new _root_.com.twitter.scalding.parquet.tuple.scheme.ParquetTupleConverter($parentTree){ - override def newConverter(i: Int): _root_.com.twitter.scalding.parquet.tuple.scheme.TupleFieldConverter[Any] = { - $converterBodyTree + def buildGroupConverter(tpe: Type, isOption: Boolean, converters: List[Tree], converterGetters: List[Tree], + converterResetCalls: List[Tree], valueBuilder: Tree): Tree = { + q"""new _root_.com.twitter.scalding.parquet.tuple.scheme.ParquetTupleConverter[$tpe]{ + ..$converters + + override def currentValue: $tpe = $valueBuilder + + override def getConverter(i: Int): _root_.parquet.io.api.Converter = { + ..$converterGetters throw new RuntimeException("invalid index: " + i) } - override def createValue(): Any = { - $valueBuilder + override def reset(): Unit = { + ..$converterResetCalls } + }""" } - def matchField(idx: Int, fieldType: Type, isOption: Boolean): (Tree, Tree) = { + def matchField(idx: Int, fieldType: Type, isOption: Boolean): (Tree, Tree, Tree, Tree) = { + + def createPrimitiveConverter(converterName: TermName, converterType: Type): Tree = { + if (isOption) { + q""" + val $converterName = new _root_.com.twitter.scalding.parquet.tuple.scheme.OptionalPrimitiveFieldConverter[$fieldType] { + override val delegate: _root_.com.twitter.scalding.parquet.tuple.scheme.PrimitiveFieldConverter[$fieldType] = new $converterType() + } + """ + } else { + q"val $converterName = new $converterType()" + } + } - def createConverter(converter: Tree): Tree = q"if($idx == i) return $converter" + def createCaseClassFieldConverter(converterName: TermName, groupConverter: Tree): Tree = { + if (isOption) { + q""" + val $converterName = new _root_.com.twitter.scalding.parquet.tuple.scheme.OptionalParquetTupleConverter[$fieldType] { + override val delegate: _root_.com.twitter.scalding.parquet.tuple.scheme.ParquetTupleConverter[$fieldType] = $groupConverter + } + """ + } else { + q"val $converterName = $groupConverter" + } + } - def primitiveFieldValue(converterType: Type): Tree = if (isOption) { - val cachedRes = newTermName(ctx.fresh(s"fieldValue")) - q""" - { - val $cachedRes = converters($idx).asInstanceOf[$converterType] - if($cachedRes.hasValue) Some($cachedRes.currentValue) else _root_.scala.Option.empty[$fieldType] - } - """ - } else { - q"converters($idx).asInstanceOf[$converterType].currentValue" + def createFieldMatchResult(converterName: TermName, converter: Tree): (Tree, Tree, Tree, Tree) = { + val converterGetter: Tree = q"if($idx == i) return $converterName" + val converterResetCall: Tree = q"$converterName.reset()" + val converterFieldValue: Tree = q"$converterName.currentValue" + (converter, converterGetter, converterResetCall, converterFieldValue) } - def primitiveFieldConverterAndFieldValue(converterType: Type): (Tree, Tree) = { - val companion = converterType.typeSymbol.companionSymbol - (createConverter(q"$companion(this)"), primitiveFieldValue(converterType)) + def matchPrimitiveField(converterType: Type): (Tree, Tree, Tree, Tree) = { + val converterName = newTermName(ctx.fresh(s"fieldConverter")) + val converter: Tree = createPrimitiveConverter(converterName, converterType) + createFieldMatchResult(converterName, converter) } - def caseClassFieldValue: Tree = if (isOption) { - val cachedRes = newTermName(ctx.fresh(s"fieldValue")) - q""" - { - val $cachedRes = converters($idx) - if($cachedRes.hasValue) Some($cachedRes.currentValue.asInstanceOf[$fieldType]) - else _root_.scala.Option.empty[$fieldType] - } - """ - } else { - q"converters($idx).currentValue.asInstanceOf[$fieldType]" + def matchCaseClassField(groupConverter: Tree): (Tree, Tree, Tree, Tree) = { + val converterName = newTermName(ctx.fresh(s"fieldConverter")) + val converter: Tree = createCaseClassFieldConverter(converterName, groupConverter) + createFieldMatchResult(converterName, converter) } fieldType match { case tpe if tpe =:= typeOf[String] => - primitiveFieldConverterAndFieldValue(typeOf[StringConverter]) + matchPrimitiveField(typeOf[StringConverter]) case tpe if tpe =:= typeOf[Boolean] => - primitiveFieldConverterAndFieldValue(typeOf[BooleanConverter]) + matchPrimitiveField(typeOf[BooleanConverter]) case tpe if tpe =:= typeOf[Byte] => - primitiveFieldConverterAndFieldValue(typeOf[ByteConverter]) + matchPrimitiveField(typeOf[ByteConverter]) case tpe if tpe =:= typeOf[Short] => - primitiveFieldConverterAndFieldValue(typeOf[ShortConverter]) + matchPrimitiveField(typeOf[ShortConverter]) case tpe if tpe =:= typeOf[Int] => - primitiveFieldConverterAndFieldValue(typeOf[IntConverter]) + matchPrimitiveField(typeOf[IntConverter]) case tpe if tpe =:= typeOf[Long] => - primitiveFieldConverterAndFieldValue(typeOf[LongConverter]) + matchPrimitiveField(typeOf[LongConverter]) case tpe if tpe =:= typeOf[Float] => - primitiveFieldConverterAndFieldValue(typeOf[FloatConverter]) + matchPrimitiveField(typeOf[FloatConverter]) case tpe if tpe =:= typeOf[Double] => - primitiveFieldConverterAndFieldValue(typeOf[DoubleConverter]) + matchPrimitiveField(typeOf[DoubleConverter]) case tpe if tpe.erasure =:= typeOf[Option[Any]] => val innerType = tpe.asInstanceOf[TypeRefApi].args.head matchField(idx, innerType, isOption = true) case tpe if IsCaseClassImpl.isCaseClassType(ctx)(tpe) => - val (innerConverters, innerFieldValues) = expandMethod(tpe).unzip - val innerConverterTree = buildConverterBody(tpe, innerConverters) + val (innerConverters, innerConvertersGetters, innerConvertersResetCalls, innerFieldValues) = unzip(expandMethod(tpe)) val innerValueBuilderTree = buildTupleValue(tpe, innerFieldValues) - val innerGroupConverter = createConverter(buildGroupConverter(tpe, q"Option(this)", isOption, idx, innerConverterTree, innerValueBuilderTree)) - (innerGroupConverter, caseClassFieldValue) + val converterTree: Tree = buildGroupConverter(tpe, isOption, innerConverters, + innerConvertersGetters, innerConvertersResetCalls, innerValueBuilderTree) + matchCaseClassField(converterTree) case _ => ctx.abort(ctx.enclosingPosition, s"Case class $T is not pure primitives or nested case classes") } } - def expandMethod(outerTpe: Type): List[(Tree, Tree)] = { + def expandMethod(outerTpe: Type): List[(Tree, Tree, Tree, Tree)] = outerTpe .declarations .collect { case m: MethodSymbol if m.isCaseAccessor => m } @@ -103,15 +119,12 @@ object ParquetTupleConverterProvider { val fieldType = accessorMethod.returnType matchField(idx, fieldType, isOption = false) }.toList - } - def buildConverterBody(tpe: Type, trees: List[Tree]): Tree = { - if (trees.isEmpty) - ctx.abort(ctx.enclosingPosition, s"Case class $tpe has no primitive types we were able to extract") - trees.foldLeft(q"") { - case (existingTree, t) => - q"""$existingTree - $t""" + def unzip(treeTuples: List[(Tree, Tree, Tree, Tree)]): (List[Tree], List[Tree], List[Tree], List[Tree]) = { + val emptyTreeList = List[Tree]() + treeTuples.foldRight(emptyTreeList, emptyTreeList, emptyTreeList, emptyTreeList) { + case ((t1, t2, t3, t4), (l1, l2, l3, l4)) => + (t1 :: l1, t2 :: l2, t3 :: l3, t4 :: l4) } } @@ -122,11 +135,11 @@ object ParquetTupleConverterProvider { q"$companion(..$fieldValueBuilders)" } - val (converters, fieldValues) = expandMethod(T.tpe).unzip - val groupConverter = buildGroupConverter(T.tpe, q"None", isOption = false, -1, buildConverterBody(T.tpe, converters), - buildTupleValue(T.tpe, fieldValues)) + val (converters, converterGetters, convertersResetCalls, fieldValues) = unzip(expandMethod(T.tpe)) + val groupConverter = buildGroupConverter(T.tpe, isOption = false, converters, converterGetters, + convertersResetCalls, buildTupleValue(T.tpe, fieldValues)) - ctx.Expr[ParquetTupleConverter](q""" + ctx.Expr[ParquetTupleConverter[T]](q""" $groupConverter """) } diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/ParquetTupleConverter.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/ParquetTupleConverter.scala index 43615de045..87a1dc216d 100644 --- a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/ParquetTupleConverter.scala +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/ParquetTupleConverter.scala @@ -3,10 +3,6 @@ package com.twitter.scalding.parquet.tuple.scheme import parquet.io.api.{ Binary, Converter, GroupConverter, PrimitiveConverter } import scala.util.Try -/** - * Parquet tuple field converter used by parquet read support to read tuple field value from parquet. - * @tparam T field value type. - */ trait TupleFieldConverter[+T] extends Converter { /** * Current value read from parquet column @@ -14,144 +10,135 @@ trait TupleFieldConverter[+T] extends Converter { def currentValue: T /** - * Used to check if a optional field has value stored or not. - */ - var hasValue: Boolean = false - - /** - *reset the converter state, make it ready for reading next column value. + * reset the converter state, make it ready for reading next column value. */ def reset(): Unit } /** * Parquet tuple converter used to create user defined tuple value from parquet column values - * @param parent parent parquet tuple converter */ -abstract class ParquetTupleConverter(val parent: Option[ParquetTupleConverter] = None) extends GroupConverter - with TupleFieldConverter[Any] { - var converters: Map[Int, TupleFieldConverter[Any]] = Map() +abstract class ParquetTupleConverter[T] extends GroupConverter with TupleFieldConverter[T] { + override def start(): Unit = reset() + override def end(): Unit = () +} - var value: Any = null +abstract class OptionalParquetTupleConverter[T] extends GroupConverter with TupleFieldConverter[Option[T]] { + var value: Option[T] = None + val delegate: ParquetTupleConverter[T] - override def currentValue: Any = value + def currentValue: Option[T] = value - def createValue(): Any + override def start(): Unit = reset() - def newConverter(i: Int): TupleFieldConverter[Any] + override def getConverter(i: Int): Converter = delegate.getConverter(i) - override def getConverter(i: Int) = { - val converter = converters.get(i) - if (converter.isDefined) converter.get - else { - val c = newConverter(i) - converters += i -> c - c - } + override def reset(): Unit = { + value = None + delegate.reset() } override def end(): Unit = { - if (hasValue) { - value = createValue() - parent.map(p => p.hasValue = true) - } - } - - override def reset(): Unit = { - value = null - converters.values.map(v => v.reset()) - hasValue = false + value = Option(delegate.currentValue) } - - override def start(): Unit = reset() } -sealed trait PrimitiveTupleFieldConverter[T] extends TupleFieldConverter[T] { - val parent: ParquetTupleConverter +trait PrimitiveFieldConverter[T] extends PrimitiveConverter with TupleFieldConverter[T] { val defaultValue: T var value: T = defaultValue override def currentValue: T = value - protected def valueAdded(): Unit = { - hasValue = true - parent.hasValue = true - } + override def reset(): Unit = value = defaultValue +} + +abstract class OptionalPrimitiveFieldConverter[T] extends PrimitiveConverter with TupleFieldConverter[Option[T]] { + var value: Option[T] = None + + val delegate: PrimitiveFieldConverter[T] override def reset(): Unit = { - value = defaultValue - hasValue = false + value = None + delegate.reset() + } + + override def currentValue: Option[T] = value + + override def addBinary(v: Binary) = { + delegate.addBinary(v) + value = Option(delegate.currentValue) + } + + override def addBoolean(v: Boolean) = { + delegate.addBoolean(v) + value = Option(delegate.currentValue) + } + + override def addDouble(v: Double) = { + delegate.addDouble(v) + value = Option(delegate.currentValue) + } + + override def addFloat(v: Float) = { + delegate.addFloat(v) + value = Option(delegate.currentValue) + } + + override def addInt(v: Int) = { + delegate.addInt(v) + value = Option(delegate.currentValue) + } + + override def addLong(v: Long) = { + delegate.addLong(v) + value = Option(delegate.currentValue) } } -case class StringConverter(parent: ParquetTupleConverter) extends PrimitiveConverter with PrimitiveTupleFieldConverter[String] { +class StringConverter extends PrimitiveFieldConverter[String] { override val defaultValue: String = null - override def addBinary(binary: Binary): Unit = { - value = binary.toStringUsingUTF8 - valueAdded() - } + override def addBinary(binary: Binary): Unit = value = binary.toStringUsingUTF8 } -case class DoubleConverter(parent: ParquetTupleConverter) extends PrimitiveConverter with PrimitiveTupleFieldConverter[Double] { +class DoubleConverter extends PrimitiveFieldConverter[Double] { override val defaultValue: Double = 0D - override def addDouble(v: Double): Unit = { - value = v - valueAdded() - } + override def addDouble(v: Double): Unit = value = v } -case class FloatConverter(parent: ParquetTupleConverter) extends PrimitiveConverter with PrimitiveTupleFieldConverter[Float] { +class FloatConverter extends PrimitiveFieldConverter[Float] { override val defaultValue: Float = 0F - override def addFloat(v: Float): Unit = { - value = v - valueAdded() - } + override def addFloat(v: Float): Unit = value = v } -case class LongConverter(parent: ParquetTupleConverter) extends PrimitiveConverter with PrimitiveTupleFieldConverter[Long] { +class LongConverter extends PrimitiveFieldConverter[Long] { override val defaultValue: Long = 0L - override def addLong(v: Long): Unit = { - value = v - valueAdded() - } + override def addLong(v: Long): Unit = value = v } -case class IntConverter(parent: ParquetTupleConverter) extends PrimitiveConverter with PrimitiveTupleFieldConverter[Int] { +class IntConverter extends PrimitiveFieldConverter[Int] { override val defaultValue: Int = 0 - override def addInt(v: Int): Unit = { - value = v - valueAdded() - } + override def addInt(v: Int): Unit = value = v } -case class ShortConverter(parent: ParquetTupleConverter) extends PrimitiveConverter with PrimitiveTupleFieldConverter[Short] { +class ShortConverter extends PrimitiveFieldConverter[Short] { override val defaultValue: Short = 0 - override def addInt(v: Int): Unit = { - value = Try(v.toShort).getOrElse(0) - valueAdded() - } + override def addInt(v: Int): Unit = value = Try(v.toShort).getOrElse(0) } -case class ByteConverter(parent: ParquetTupleConverter) extends PrimitiveConverter with PrimitiveTupleFieldConverter[Byte] { +class ByteConverter extends PrimitiveFieldConverter[Byte] { override val defaultValue: Byte = 0 - override def addInt(v: Int): Unit = { - value = Try(v.toByte).getOrElse(0) - valueAdded() - } + override def addInt(v: Int): Unit = value = Try(v.toByte).getOrElse(0) } -case class BooleanConverter(parent: ParquetTupleConverter) extends PrimitiveConverter with PrimitiveTupleFieldConverter[Boolean] { +class BooleanConverter extends PrimitiveFieldConverter[Boolean] { override val defaultValue: Boolean = false - override def addBoolean(v: Boolean): Unit = { - value = v - valueAdded() - } + override def addBoolean(v: Boolean): Unit = value = v } diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/TypedParquetTupleScheme.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/TypedParquetTupleScheme.scala index 261a705c22..be2d632398 100644 --- a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/TypedParquetTupleScheme.scala +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/TypedParquetTupleScheme.scala @@ -22,8 +22,8 @@ import parquet.schema._ * @param converter root converter * @tparam T User defined value type */ -class ParquetTupleMaterializer[T](val converter: ParquetTupleConverter) extends RecordMaterializer[T] { - override def getCurrentRecord: T = converter.currentValue.asInstanceOf[T] +class ParquetTupleMaterializer[T](val converter: ParquetTupleConverter[T]) extends RecordMaterializer[T] { + override def getCurrentRecord: T = converter.currentValue override def getRootConverter: GroupConverter = converter } @@ -45,7 +45,7 @@ class ParquetTupleMaterializer[T](val converter: ParquetTupleConverter) extends * @tparam T user defined value type */ trait ParquetReadSupport[T] extends ReadSupport[T] { - val tupleConverter: ParquetTupleConverter + val tupleConverter: ParquetTupleConverter[T] val rootSchema: String lazy val rootType: MessageType = MessageTypeParser.parseMessageType(rootSchema) diff --git a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala index ce7d611af0..520eefe548 100644 --- a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala +++ b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala @@ -66,12 +66,12 @@ object SampleClassB { } class BReadSupport extends ParquetReadSupport[SampleClassB] { - override val tupleConverter: ParquetTupleConverter = Macros.caseClassParquetTupleConverter[SampleClassB] + override val tupleConverter: ParquetTupleConverter[SampleClassB] = Macros.caseClassParquetTupleConverter[SampleClassB] override val rootSchema: String = SampleClassB.schema } class CReadSupport extends ParquetReadSupport[SampleClassC] { - override val tupleConverter: ParquetTupleConverter = Macros.caseClassParquetTupleConverter[SampleClassC] + override val tupleConverter: ParquetTupleConverter[SampleClassC] = Macros.caseClassParquetTupleConverter[SampleClassC] override val rootSchema: String = Macros.caseClassParquetSchema[SampleClassC] } diff --git a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/macros/MacroUnitTests.scala b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/macros/MacroUnitTests.scala index 06843438ea..66675f6f53 100644 --- a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/macros/MacroUnitTests.scala +++ b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/macros/MacroUnitTests.scala @@ -109,7 +109,7 @@ class MacroUnitTests extends WordSpec with Matchers with MockitoSugar { "Generate converters for all primitive types" in { val converter = Macros.caseClassParquetTupleConverter[SampleClassE] - + converter.start() val intConverter = converter.getConverter(0).asPrimitiveConverter() intConverter.addInt(0) @@ -133,13 +133,13 @@ class MacroUnitTests extends WordSpec with Matchers with MockitoSugar { val byte = converter.getConverter(7).asPrimitiveConverter() byte.addInt(1) - - converter.createValue shouldEqual SampleClassE(0, 1L, 2, d = true, 3F, 4D, "foo", 1) + converter.end() + converter.currentValue shouldEqual SampleClassE(0, 1L, 2, d = true, 3F, 4D, "foo", 1) } "Generate converters for case class with nested class" in { val converter = Macros.caseClassParquetTupleConverter[SampleClassB] - + converter.start() val a = converter.getConverter(0).asGroupConverter() a.start() @@ -151,13 +151,13 @@ class MacroUnitTests extends WordSpec with Matchers with MockitoSugar { val bString = converter.getConverter(1).asPrimitiveConverter() bString.addBinary(Binary.fromString("toto")) - - converter.createValue() shouldEqual SampleClassB(SampleClassA(2, "foo"), "toto") + converter.end() + converter.currentValue shouldEqual SampleClassB(SampleClassA(2, "foo"), "toto") } "Generate converters for case class with optional nested class" in { val converter = Macros.caseClassParquetTupleConverter[SampleClassG] - + converter.start() val a = converter.getConverter(0).asPrimitiveConverter() a.addInt(0) @@ -177,8 +177,8 @@ class MacroUnitTests extends WordSpec with Matchers with MockitoSugar { val c = converter.getConverter(2).asPrimitiveConverter() c.addDouble(4D) - - converter.createValue() shouldEqual SampleClassG(0, Some(SampleClassB(SampleClassA(2, "foo"), "b1")), 4D) + converter.end() + converter.currentValue shouldEqual SampleClassG(0, Some(SampleClassB(SampleClassA(2, "foo"), "b1")), 4D) } } From 536bd0c43981475d48922f303df22df8618de1ea Mon Sep 17 00:00:00 2001 From: beethere Date: Tue, 21 Apr 2015 22:09:13 +0200 Subject: [PATCH 121/177] Typed Parquet Tuple #1198 * Add macro support for collection types(LIST, SET, MAP) --- .../scalding/parquet/tuple/TypedParquet.scala | 6 +- .../parquet/tuple/macros/Macros.scala | 15 +- .../macros/impl/ParquetSchemaProvider.scala | 28 +- .../impl/ParquetTupleConverterProvider.scala | 131 +++++-- .../macros/impl/WriteSupportProvider.scala | 47 ++- .../tuple/scheme/ParquetTupleConverter.scala | 300 +++++++++++++--- .../parquet/tuple/TypedParquetTupleTest.scala | 80 ++--- .../parquet/tuple/macros/MacroUnitTests.scala | 340 +++++++++++++++++- 8 files changed, 761 insertions(+), 186 deletions(-) diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/TypedParquet.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/TypedParquet.scala index b9baff93bf..860c3cb71d 100644 --- a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/TypedParquet.scala +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/TypedParquet.scala @@ -21,7 +21,7 @@ object TypedParquet { * * class ReadSupport extends ParquetReadSupport[SampleClassB] { * import com.twitter.scalding.parquet.tuple.macros.Macros._ - * override val tupleConverter: ParquetTupleConverter = caseClassParquetTupleConverter[SampleClassB] + * override val tupleConverter: ParquetTupleConverter[SampleClassB] = caseClassParquetTupleConverter[SampleClassB] * override val rootSchema: String = caseClassParquetSchema[SampleClassB] * } * @@ -71,7 +71,9 @@ object TypedParquetSink { * * class WriteSupport extends ParquetWriteSupport[SampleClassB] { * import com.twitter.scalding.parquet.tuple.macros.Macros._ - * override val fieldValues: (SampleClassB) => Map[Int, Any] = caseClassFieldValues[SampleClassB] + * + * override def writeRecord(r: SampleClassB, rc: RecordConsumer, schema: MessageType): Unit = + * caseClassWriteSupport[SampleClassB](r, rc, schema) * override val rootSchema: String = caseClassParquetSchema[SampleClassB] * } * diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/Macros.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/Macros.scala index 47f7b03097..295fd5ccaa 100644 --- a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/Macros.scala +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/Macros.scala @@ -9,13 +9,13 @@ import scala.language.experimental.macros /** * Macros used to generate parquet tuple read/write support. - * For right now these macros support only case class that contains only primitive fields or nested case classes. - * Option field is also supported. But collection types like List are not supported yet. + * These macros support only case class that contains primitive fields or nested case classes and also collection fields + * like scala List, Set, and Map. + * @author Jian TANG */ object Macros { /** - * Macro used to generate parquet schema for a given case class that contains only primitive fields. - * Option field and nested group is supported. For example if we have: + * Macro used to generate parquet schema for a given case class. For example if we have: * * case class SampleClassA(x: Int, y: String) * case class SampleClassB(a: SampleClassA, y: String) @@ -32,16 +32,15 @@ object Macros { * } * """ * - * @tparam T Case class type that contains only primitive fields or nested case class. + * @tparam T Case class type that contains primitive fields or collection fields or nested case class. * @return Generated case class parquet message type string */ def caseClassParquetSchema[T]: String = macro ParquetSchemaProvider.toParquetSchemaImpl[T] /** - * Macro used to generate parquet tuple converter for a given case class that contains only primitive fields. - * Option field and nested group is supported. + * Macro used to generate parquet tuple converter for a given case class. * - * @tparam T Case class type that contains only primitive fields or nested case class. + * @tparam T Case class type that contains primitive or collection type fields or nested case class. * @return Generated parquet converter */ def caseClassParquetTupleConverter[T]: ParquetTupleConverter[T] = macro ParquetTupleConverterProvider.toParquetTupleConverterImpl[T] diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/ParquetSchemaProvider.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/ParquetSchemaProvider.scala index a322d251a0..012bd36d22 100644 --- a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/ParquetSchemaProvider.scala +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/ParquetSchemaProvider.scala @@ -12,14 +12,18 @@ object ParquetSchemaProvider { c.abort(c.enclosingPosition, s"""We cannot enforce ${T.tpe} is a case class, either it is not a case class or this macro call is possibly enclosed in a class. This will mean the macro is operating on a non-resolved type.""") - def matchField(fieldType: Type, fieldName: String, isOption: Boolean): List[Tree] = { + def matchField(fieldType: Type, fieldName: String, isOption: Boolean): Tree = { val REPETITION_REQUIRED = q"_root_.parquet.schema.Type.Repetition.REQUIRED" val REPETITION_OPTIONAL = q"_root_.parquet.schema.Type.Repetition.OPTIONAL" + val REPETITION_REPEATED = q"_root_.parquet.schema.Type.Repetition.REPEATED" def repetition: Tree = if (isOption) REPETITION_OPTIONAL else REPETITION_REQUIRED - def createPrimitiveTypeField(primitiveType: Tree): List[Tree] = - List(q"""new _root_.parquet.schema.PrimitiveType($repetition, $primitiveType, $fieldName)""") + def createPrimitiveTypeField(primitiveType: Tree): Tree = + q"""new _root_.parquet.schema.PrimitiveType($repetition, $primitiveType, $fieldName)""" + + def createListGroupType(innerFieldsType: Tree): Tree = + q"""new _root_.parquet.schema.GroupType($REPETITION_REPEATED, "list", $innerFieldsType)""" fieldType match { case tpe if tpe =:= typeOf[String] => @@ -37,10 +41,18 @@ object ParquetSchemaProvider { case tpe if tpe.erasure =:= typeOf[Option[Any]] => val innerType = tpe.asInstanceOf[TypeRefApi].args.head matchField(innerType, fieldName, isOption = true) + case tpe if tpe.erasure =:= typeOf[List[Any]] || tpe.erasure =:= typeOf[Set[_]] => + val innerType = tpe.asInstanceOf[TypeRefApi].args.head + val innerFieldsType = matchField(innerType, "element", isOption = false) + q"_root_.parquet.schema.ConversionPatterns.listType($repetition, $fieldName, ${createListGroupType(innerFieldsType)})" + case tpe if tpe.erasure =:= typeOf[Map[_, Any]] => + val List(keyType, valueType) = tpe.asInstanceOf[TypeRefApi].args + val keyFieldType = matchField(keyType, "key", isOption = false) + val valueFieldType = matchField(valueType, "value", isOption = false) + q"_root_.parquet.schema.ConversionPatterns.mapType($repetition, $fieldName, $keyFieldType, $valueFieldType)" case tpe if IsCaseClassImpl.isCaseClassType(c)(tpe) => - List(q"""new _root_.parquet.schema.GroupType($repetition, $fieldName, - _root_.scala.Array.apply[_root_.parquet.schema.Type](..${expandMethod(tpe)}):_*)""") - case _ => c.abort(c.enclosingPosition, s"Case class $T is not pure primitives or nested case classes") + q"new _root_.parquet.schema.GroupType($repetition, $fieldName, ..${expandMethod(tpe)})" + case _ => c.abort(c.enclosingPosition, s"Case class $T has unsupported field type : $fieldType ") } } @@ -48,7 +60,7 @@ object ParquetSchemaProvider { outerTpe .declarations .collect { case m: MethodSymbol if m.isCaseAccessor => m } - .flatMap { accessorMethod => + .map { accessorMethod => val fieldName = accessorMethod.name.toTermName.toString val fieldType = accessorMethod.returnType matchField(fieldType, fieldName, isOption = false) @@ -58,7 +70,7 @@ object ParquetSchemaProvider { val expanded = expandMethod(T.tpe) if (expanded.isEmpty) - c.abort(c.enclosingPosition, s"Case class $T.tpe has no primitive types we were able to extract") + c.abort(c.enclosingPosition, s"Case class $T.tpe has no fields we were able to extract") val messageTypeName = s"${T.tpe}".split("\\.").last val schema = q"""new _root_.parquet.schema.MessageType($messageTypeName, diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/ParquetTupleConverterProvider.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/ParquetTupleConverterProvider.scala index f226f6d6c4..aed795ae67 100644 --- a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/ParquetTupleConverterProvider.scala +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/ParquetTupleConverterProvider.scala @@ -6,6 +6,13 @@ import com.twitter.scalding.parquet.tuple.scheme._ import scala.reflect.macros.Context object ParquetTupleConverterProvider { + private[this] sealed trait CollectionType + private[this] case object NOT_A_COLLECTION extends CollectionType + private[this] case object OPTION extends CollectionType + private[this] case object LIST extends CollectionType + private[this] case object SET extends CollectionType + private[this] case object MAP extends CollectionType + def toParquetTupleConverterImpl[T](ctx: Context)(implicit T: ctx.WeakTypeTag[T]): ctx.Expr[ParquetTupleConverter[T]] = { import ctx.universe._ @@ -15,8 +22,8 @@ object ParquetTupleConverterProvider { either it is not a case class or this macro call is possibly enclosed in a class. This will mean the macro is operating on a non-resolved type.""") - def buildGroupConverter(tpe: Type, isOption: Boolean, converters: List[Tree], converterGetters: List[Tree], - converterResetCalls: List[Tree], valueBuilder: Tree): Tree = { + def buildGroupConverter(tpe: Type, converters: List[Tree], converterGetters: List[Tree], + converterResetCalls: List[Tree], valueBuilder: Tree): Tree = q"""new _root_.com.twitter.scalding.parquet.tuple.scheme.ParquetTupleConverter[$tpe]{ ..$converters @@ -31,35 +38,65 @@ object ParquetTupleConverterProvider { ..$converterResetCalls } - }""" - } - - def matchField(idx: Int, fieldType: Type, isOption: Boolean): (Tree, Tree, Tree, Tree) = { - - def createPrimitiveConverter(converterName: TermName, converterType: Type): Tree = { - if (isOption) { - q""" - val $converterName = new _root_.com.twitter.scalding.parquet.tuple.scheme.OptionalPrimitiveFieldConverter[$fieldType] { - override val delegate: _root_.com.twitter.scalding.parquet.tuple.scheme.PrimitiveFieldConverter[$fieldType] = new $converterType() - } - """ - } else { - q"val $converterName = new $converterType()" + }""" + + def matchField(idx: Int, fieldType: Type, collectionType: CollectionType): (Tree, Tree, Tree, Tree) = { + def fieldConverter(converterName: TermName, converter: Tree, isPrimitive: Boolean = false): Tree = { + def primitiveCollectionElementConverter: Tree = + q"""override val child: _root_.com.twitter.scalding.parquet.tuple.scheme.TupleFieldConverter[$fieldType] = + new _root_.com.twitter.scalding.parquet.tuple.scheme.CollectionElementPrimitiveConverter[$fieldType](this) { + override val delegate: _root_.com.twitter.scalding.parquet.tuple.scheme.PrimitiveFieldConverter[$fieldType] = $converter + } + """ + + def caseClassFieldCollectionElementConverter: Tree = + q"""override val child: _root_.com.twitter.scalding.parquet.tuple.scheme.TupleFieldConverter[$fieldType] = + new _root_.com.twitter.scalding.parquet.tuple.scheme.CollectionElementGroupConverter[$fieldType](this) { + override val delegate: _root_.com.twitter.scalding.parquet.tuple.scheme.TupleFieldConverter[$fieldType] = $converter + } + """ + + collectionType match { + case OPTION => + val child = if (isPrimitive) primitiveCollectionElementConverter else caseClassFieldCollectionElementConverter + q""" + val $converterName = new _root_.com.twitter.scalding.parquet.tuple.scheme.OptionConverter[$fieldType] { + $child + } + """ + case LIST => + val child = if (isPrimitive) primitiveCollectionElementConverter else caseClassFieldCollectionElementConverter + q""" + val $converterName = new _root_.com.twitter.scalding.parquet.tuple.scheme.ListConverter[$fieldType] { + $child + } + """ + case SET => + val child = if (isPrimitive) primitiveCollectionElementConverter else caseClassFieldCollectionElementConverter + + q""" + val $converterName = new _root_.com.twitter.scalding.parquet.tuple.scheme.SetConverter[$fieldType] { + $child + } + """ + case MAP => converter + case _ => q"val $converterName = $converter" } - } - def createCaseClassFieldConverter(converterName: TermName, groupConverter: Tree): Tree = { - if (isOption) { - q""" - val $converterName = new _root_.com.twitter.scalding.parquet.tuple.scheme.OptionalParquetTupleConverter[$fieldType] { - override val delegate: _root_.com.twitter.scalding.parquet.tuple.scheme.ParquetTupleConverter[$fieldType] = $groupConverter - } - """ - } else { - q"val $converterName = $groupConverter" - } } + def createMapFieldConverter(converterName: TermName, K: Type, V: Type, keyConverter: Tree, + valueConverter: Tree): Tree = + q"""val $converterName = new _root_.com.twitter.scalding.parquet.tuple.scheme.MapConverter[$K, $V] { + + override val child: _root_.com.twitter.scalding.parquet.tuple.scheme.TupleFieldConverter[($K, $V)] = + new _root_.com.twitter.scalding.parquet.tuple.scheme.MapKeyValueConverter[$K, $V](this) { + override val keyConverter: _root_.com.twitter.scalding.parquet.tuple.scheme.TupleFieldConverter[$K] = $keyConverter + override val valueConverter: _root_.com.twitter.scalding.parquet.tuple.scheme.TupleFieldConverter[$V] = $valueConverter + } + } + """ + def createFieldMatchResult(converterName: TermName, converter: Tree): (Tree, Tree, Tree, Tree) = { val converterGetter: Tree = q"if($idx == i) return $converterName" val converterResetCall: Tree = q"$converterName.reset()" @@ -69,16 +106,23 @@ object ParquetTupleConverterProvider { def matchPrimitiveField(converterType: Type): (Tree, Tree, Tree, Tree) = { val converterName = newTermName(ctx.fresh(s"fieldConverter")) - val converter: Tree = createPrimitiveConverter(converterName, converterType) + val innerConverter: Tree = q"new $converterType()" + val converter: Tree = fieldConverter(converterName, innerConverter, isPrimitive = true) createFieldMatchResult(converterName, converter) } def matchCaseClassField(groupConverter: Tree): (Tree, Tree, Tree, Tree) = { val converterName = newTermName(ctx.fresh(s"fieldConverter")) - val converter: Tree = createCaseClassFieldConverter(converterName, groupConverter) + val converter: Tree = fieldConverter(converterName, groupConverter) createFieldMatchResult(converterName, converter) } + def matchMapField(K: Type, V: Type, keyConverter: Tree, valueConverter: Tree): (Tree, Tree, Tree, Tree) = { + val converterName = newTermName(ctx.fresh(s"fieldConverter")) + val mapConverter = createMapFieldConverter(converterName, K, V, keyConverter, valueConverter) + createFieldMatchResult(converterName, mapConverter) + } + fieldType match { case tpe if tpe =:= typeOf[String] => matchPrimitiveField(typeOf[StringConverter]) @@ -98,14 +142,25 @@ object ParquetTupleConverterProvider { matchPrimitiveField(typeOf[DoubleConverter]) case tpe if tpe.erasure =:= typeOf[Option[Any]] => val innerType = tpe.asInstanceOf[TypeRefApi].args.head - matchField(idx, innerType, isOption = true) + matchField(idx, innerType, OPTION) + case tpe if tpe.erasure =:= typeOf[List[Any]] => + val innerType = tpe.asInstanceOf[TypeRefApi].args.head + matchField(idx, innerType, LIST) + case tpe if tpe.erasure =:= typeOf[Set[_]] => + val innerType = tpe.asInstanceOf[TypeRefApi].args.head + matchField(idx, innerType, SET) + case tpe if tpe.erasure =:= typeOf[Map[_, Any]] => + val List(keyType, valueType) = tpe.asInstanceOf[TypeRefApi].args + val (keyConverter, _, _, _) = matchField(0, keyType, MAP) + val (valueConverter, _, _, _) = matchField(0, valueType, MAP) + matchMapField(keyType, valueType, keyConverter, valueConverter) case tpe if IsCaseClassImpl.isCaseClassType(ctx)(tpe) => val (innerConverters, innerConvertersGetters, innerConvertersResetCalls, innerFieldValues) = unzip(expandMethod(tpe)) val innerValueBuilderTree = buildTupleValue(tpe, innerFieldValues) - val converterTree: Tree = buildGroupConverter(tpe, isOption, innerConverters, - innerConvertersGetters, innerConvertersResetCalls, innerValueBuilderTree) + val converterTree: Tree = buildGroupConverter(tpe, innerConverters, innerConvertersGetters, + innerConvertersResetCalls, innerValueBuilderTree) matchCaseClassField(converterTree) - case _ => ctx.abort(ctx.enclosingPosition, s"Case class $T is not pure primitives or nested case classes") + case _ => ctx.abort(ctx.enclosingPosition, s"Case class $T has unsupported field type : $fieldType ") } } @@ -117,7 +172,7 @@ object ParquetTupleConverterProvider { .map { case (accessorMethod, idx) => val fieldType = accessorMethod.returnType - matchField(idx, fieldType, isOption = false) + matchField(idx, fieldType, NOT_A_COLLECTION) }.toList def unzip(treeTuples: List[(Tree, Tree, Tree, Tree)]): (List[Tree], List[Tree], List[Tree], List[Tree]) = { @@ -136,11 +191,9 @@ object ParquetTupleConverterProvider { } val (converters, converterGetters, convertersResetCalls, fieldValues) = unzip(expandMethod(T.tpe)) - val groupConverter = buildGroupConverter(T.tpe, isOption = false, converters, converterGetters, - convertersResetCalls, buildTupleValue(T.tpe, fieldValues)) + val groupConverter = buildGroupConverter(T.tpe, converters, converterGetters, convertersResetCalls, + buildTupleValue(T.tpe, fieldValues)) - ctx.Expr[ParquetTupleConverter[T]](q""" - $groupConverter - """) + ctx.Expr[ParquetTupleConverter[T]](groupConverter) } } diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/WriteSupportProvider.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/WriteSupportProvider.scala index 3c77e2531a..a1fadfcad3 100644 --- a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/WriteSupportProvider.scala +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/macros/impl/WriteSupportProvider.scala @@ -30,6 +30,13 @@ object WriteSupportProvider { rc.endGroup() rc.endField($groupName.getFieldName($idx), $idx) """ + def writeCollectionField(elementGroupName: TermName, subTree: Tree) = + writeGroupField(q"""if(!$fValue.isEmpty) { + val $elementGroupName = $groupName.getType($idx).asGroupType.getType(0).asGroupType + $subTree + } + """) + fieldType match { case tpe if tpe =:= typeOf[String] => writePrimitiveField(q"rc.addBinary(Binary.fromString($fValue))") @@ -51,13 +58,37 @@ object WriteSupportProvider { val cacheName = newTermName(ctx.fresh(s"optionIndex")) val innerType = tpe.asInstanceOf[TypeRefApi].args.head val (_, subTree) = matchField(idx, innerType, q"$cacheName", groupName) - (idx + 1, q""" - if($fValue.isDefined) { - val $cacheName = $fValue.get - $subTree - } + (idx + 1, q"""if($fValue.isDefined) { + val $cacheName = $fValue.get + $subTree + } """) - + case tpe if tpe.erasure =:= typeOf[List[Any]] || tpe.erasure =:= typeOf[Set[_]] => + val innerType = tpe.asInstanceOf[TypeRefApi].args.head + val newGroupName = createGroupName() + val (_, subTree) = matchField(0, innerType, q"element", newGroupName) + (idx + 1, writeCollectionField(newGroupName, q""" + rc.startField("list", 0) + $fValue.foreach{ element => + rc.startGroup() + $subTree + rc.endGroup + } + rc.endField("list", 0)""")) + case tpe if tpe.erasure =:= typeOf[Map[_, Any]] => + val List(keyType, valueType) = tpe.asInstanceOf[TypeRefApi].args + val newGroupName = createGroupName() + val (_, keySubTree) = matchField(0, keyType, q"key", newGroupName) + val (_, valueSubTree) = matchField(1, valueType, q"value", newGroupName) + (idx + 1, writeCollectionField(newGroupName, q""" + rc.startField("map", 0) + $fValue.foreach{ case(key, value) => + rc.startGroup() + $keySubTree + $valueSubTree + rc.endGroup + } + rc.endField("map", 0)""")) case tpe if IsCaseClassImpl.isCaseClassType(ctx)(tpe) => val newGroupName = createGroupName() val (_, subTree) = expandMethod(tpe, fValue, newGroupName) @@ -66,7 +97,7 @@ object WriteSupportProvider { val $newGroupName = $groupName.getType($idx).asGroupType() ${writeGroupField(subTree)}""") - case _ => ctx.abort(ctx.enclosingPosition, s"Case class $T is not pure primitives or nested case classes") + case _ => ctx.abort(ctx.enclosingPosition, s"Case class $T has unsupported field type : $fieldType") } } @@ -103,6 +134,6 @@ object WriteSupportProvider { } writeFunc """ - ctx.Expr[(T, RecordConsumer, MessageType) => Unit](q"$writeFunction") + ctx.Expr[(T, RecordConsumer, MessageType) => Unit](writeFunction) } } diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/ParquetTupleConverter.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/ParquetTupleConverter.scala index 87a1dc216d..a407eb5d36 100644 --- a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/ParquetTupleConverter.scala +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/ParquetTupleConverter.scala @@ -23,122 +23,308 @@ abstract class ParquetTupleConverter[T] extends GroupConverter with TupleFieldCo override def end(): Unit = () } -abstract class OptionalParquetTupleConverter[T] extends GroupConverter with TupleFieldConverter[Option[T]] { - var value: Option[T] = None - val delegate: ParquetTupleConverter[T] +/** + * Primitive fields converter + * @tparam T primitive types (String, Double, Float, Long, Int, Short, Byte, Boolean) + */ +trait PrimitiveFieldConverter[T] extends PrimitiveConverter with TupleFieldConverter[T] { + val defaultValue: T + var value: T = defaultValue - def currentValue: Option[T] = value + override def currentValue: T = value - override def start(): Unit = reset() + override def reset(): Unit = value = defaultValue +} + +class StringConverter extends PrimitiveFieldConverter[String] { + override val defaultValue: String = null - override def getConverter(i: Int): Converter = delegate.getConverter(i) + override def addBinary(binary: Binary): Unit = value = binary.toStringUsingUTF8 +} - override def reset(): Unit = { - value = None - delegate.reset() - } +class DoubleConverter extends PrimitiveFieldConverter[Double] { + override val defaultValue: Double = 0D - override def end(): Unit = { - value = Option(delegate.currentValue) - } + override def addDouble(v: Double): Unit = value = v } -trait PrimitiveFieldConverter[T] extends PrimitiveConverter with TupleFieldConverter[T] { - val defaultValue: T - var value: T = defaultValue +class FloatConverter extends PrimitiveFieldConverter[Float] { + override val defaultValue: Float = 0F - override def currentValue: T = value + override def addFloat(v: Float): Unit = value = v +} - override def reset(): Unit = value = defaultValue +class LongConverter extends PrimitiveFieldConverter[Long] { + override val defaultValue: Long = 0L + + override def addLong(v: Long): Unit = value = v } -abstract class OptionalPrimitiveFieldConverter[T] extends PrimitiveConverter with TupleFieldConverter[Option[T]] { - var value: Option[T] = None +class IntConverter extends PrimitiveFieldConverter[Int] { + override val defaultValue: Int = 0 - val delegate: PrimitiveFieldConverter[T] + override def addInt(v: Int): Unit = value = v +} - override def reset(): Unit = { - value = None - delegate.reset() - } +class ShortConverter extends PrimitiveFieldConverter[Short] { + override val defaultValue: Short = 0 - override def currentValue: Option[T] = value + override def addInt(v: Int): Unit = value = Try(v.toShort).getOrElse(0) +} + +class ByteConverter extends PrimitiveFieldConverter[Byte] { + override val defaultValue: Byte = 0 + + override def addInt(v: Int): Unit = value = Try(v.toByte).getOrElse(0) +} + +class BooleanConverter extends PrimitiveFieldConverter[Boolean] { + override val defaultValue: Boolean = false + + override def addBoolean(v: Boolean): Unit = value = v +} + +/** + * Collection field converter, such as list(Scala Option is also seen as a collection). + * @tparam T collection element type(can be primitive types or nested types) + */ +trait CollectionConverter[T] { + val child: TupleFieldConverter[T] + + def appendValue(v: T): Unit +} + +/** + * A wrapper of primitive converters for modeling primitive fields in a collection + * @tparam T primitive types (String, Double, Float, Long, Int, Short, Byte, Boolean) + */ +abstract class CollectionElementPrimitiveConverter[T](val parent: CollectionConverter[T]) extends PrimitiveConverter + with TupleFieldConverter[T] { + val delegate: PrimitiveFieldConverter[T] override def addBinary(v: Binary) = { delegate.addBinary(v) - value = Option(delegate.currentValue) + parent.appendValue(delegate.currentValue) } override def addBoolean(v: Boolean) = { delegate.addBoolean(v) - value = Option(delegate.currentValue) + parent.appendValue(delegate.currentValue) } override def addDouble(v: Double) = { delegate.addDouble(v) - value = Option(delegate.currentValue) + parent.appendValue(delegate.currentValue) } override def addFloat(v: Float) = { delegate.addFloat(v) - value = Option(delegate.currentValue) + parent.appendValue(delegate.currentValue) } override def addInt(v: Int) = { delegate.addInt(v) - value = Option(delegate.currentValue) + parent.appendValue(delegate.currentValue) } override def addLong(v: Long) = { delegate.addLong(v) - value = Option(delegate.currentValue) + parent.appendValue(delegate.currentValue) } -} -class StringConverter extends PrimitiveFieldConverter[String] { - override val defaultValue: String = null + override def currentValue: T = delegate.currentValue - override def addBinary(binary: Binary): Unit = value = binary.toStringUsingUTF8 + override def reset(): Unit = delegate.reset() } -class DoubleConverter extends PrimitiveFieldConverter[Double] { - override val defaultValue: Double = 0D +/** + * A wrapper of group converters for modeling group type element in a collection + * @tparam T group tuple type(can be a collection type, such as list) + */ +abstract class CollectionElementGroupConverter[T](val parent: CollectionConverter[T]) extends GroupConverter + with TupleFieldConverter[T] { - override def addDouble(v: Double): Unit = value = v + val delegate: TupleFieldConverter[T] + + override def getConverter(i: Int): Converter = delegate.asGroupConverter().getConverter(i) + + override def end(): Unit = { + parent.appendValue(delegate.currentValue) + delegate.asGroupConverter().end() + } + + override def start(): Unit = delegate.asGroupConverter().start() + + override def currentValue: T = delegate.currentValue + + override def reset(): Unit = delegate.reset() } -class FloatConverter extends PrimitiveFieldConverter[Float] { - override val defaultValue: Float = 0F +/** + * Option converter for modeling option field + * @tparam T option element type(can be primitive types or nested types) + */ +abstract class OptionConverter[T] extends TupleFieldConverter[Option[T]] with CollectionConverter[T] { + var value: Option[T] = None - override def addFloat(v: Float): Unit = value = v + override def appendValue(v: T): Unit = value = Option(v) + + override def currentValue: Option[T] = value + + override def reset(): Unit = { + value = None + child.reset() + } + + override def isPrimitive: Boolean = child.isPrimitive + + override def asGroupConverter: GroupConverter = child.asGroupConverter() + + override def asPrimitiveConverter: PrimitiveConverter = child.asPrimitiveConverter() } -class LongConverter extends PrimitiveFieldConverter[Long] { - override val defaultValue: Long = 0L +/** + * List in parquet is represented by 3-level structure. + * Check this https://github.com/apache/incubator-parquet-format/blob/master/LogicalTypes.md + * Helper class to wrap a converter for a list group converter + */ +object ListElement { + def wrapper(child: Converter): GroupConverter = new GroupConverter() { + override def getConverter(i: Int): Converter = { + if (i != 0) + throw new IllegalArgumentException("list have only one element field. can't reach " + i) + child + } - override def addLong(v: Long): Unit = value = v + override def end(): Unit = () + + override def start(): Unit = () + } } +/** + * List converter for modeling list field + * @tparam T list element type(can be primitive types or nested types) + */ +abstract class ListConverter[T] extends GroupConverter with TupleFieldConverter[List[T]] with CollectionConverter[T] { -class IntConverter extends PrimitiveFieldConverter[Int] { - override val defaultValue: Int = 0 + var value: List[T] = Nil - override def addInt(v: Int): Unit = value = v + def appendValue(v: T): Unit = value = value :+ v + + lazy val listElement: GroupConverter = new GroupConverter() { + override def getConverter(i: Int): Converter = { + if (i != 0) + throw new IllegalArgumentException("lists have only one element field. can't reach " + i) + child + } + + override def end(): Unit = () + + override def start(): Unit = () + } + + override def getConverter(i: Int): Converter = { + if (i != 0) + throw new IllegalArgumentException("lists have only one element field. can't reach " + i) + listElement + } + + override def end(): Unit = () + + override def start(): Unit = reset() + + override def currentValue: List[T] = value + + override def reset(): Unit = { + value = Nil + child.reset() + } } -class ShortConverter extends PrimitiveFieldConverter[Short] { - override val defaultValue: Short = 0 +/** + * Set converter for modeling set field + * @tparam T list element type(can be primitive types or nested types) + */ +abstract class SetConverter[T] extends GroupConverter with TupleFieldConverter[Set[T]] with CollectionConverter[T] { - override def addInt(v: Int): Unit = value = Try(v.toShort).getOrElse(0) + var value: Set[T] = Set() + + def appendValue(v: T): Unit = value = value + v + + //in the back end set is stored as list + lazy val listElement: GroupConverter = ListElement.wrapper(child) + + override def getConverter(i: Int): Converter = { + if (i != 0) + throw new IllegalArgumentException("sets have only one element field. can't reach " + i) + listElement + } + + override def end(): Unit = () + + override def start(): Unit = reset() + + override def currentValue: Set[T] = value + + override def reset(): Unit = { + value = Set() + child.reset() + } } -class ByteConverter extends PrimitiveFieldConverter[Byte] { - override val defaultValue: Byte = 0 +/** + * Map converter for modeling map field + * @tparam K map key type + * @tparam V map value type + */ +abstract class MapConverter[K, V] extends GroupConverter with TupleFieldConverter[Map[K, V]] with CollectionConverter[(K, V)] { - override def addInt(v: Int): Unit = value = Try(v.toByte).getOrElse(0) + var value: Map[K, V] = Map() + + def appendValue(v: (K, V)): Unit = value = value + v + + override def getConverter(i: Int): Converter = { + if (i != 0) + throw new IllegalArgumentException("maps have only one element type key_value(0). can't reach " + i) + child + } + + override def end(): Unit = () + + override def start(): Unit = reset() + + override def currentValue: Map[K, V] = value + + override def reset(): Unit = { + value = Map() + child.reset() + } } -class BooleanConverter extends PrimitiveFieldConverter[Boolean] { - override val defaultValue: Boolean = false +abstract class MapKeyValueConverter[K, V](parent: CollectionConverter[(K, V)]) + extends CollectionElementGroupConverter[(K, V)](parent) { - override def addBoolean(v: Boolean): Unit = value = v + val keyConverter: TupleFieldConverter[K] + + val valueConverter: TupleFieldConverter[V] + + override lazy val delegate: TupleFieldConverter[(K, V)] = new GroupConverter with TupleFieldConverter[(K, V)] { + override def currentValue: (K, V) = (keyConverter.currentValue, valueConverter.currentValue) + + override def reset(): Unit = { + keyConverter.reset() + valueConverter.reset() + } + + override def getConverter(i: Int): Converter = { + if (i == 0) keyConverter + else if (i == 1) valueConverter + else throw new IllegalArgumentException("key_value has only the key (0) and value (1) fields expected: " + i) + } + + override def end(): Unit = () + + override def start(): Unit = reset() + } } + diff --git a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala index 520eefe548..0b7483818f 100644 --- a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala +++ b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala @@ -1,7 +1,5 @@ package com.twitter.scalding.parquet.tuple -import java.io.File - import com.twitter.scalding.parquet.tuple.macros.Macros import com.twitter.scalding.parquet.tuple.scheme.{ ParquetTupleConverter, ParquetReadSupport, ParquetWriteSupport } import com.twitter.scalding.platform.{ HadoopPlatformJobTest, HadoopPlatformTest } @@ -18,48 +16,43 @@ class TypedParquetTupleTest extends WordSpec with Matchers with HadoopPlatformTe "read and write correctly" in { import com.twitter.scalding.parquet.tuple.TestValues._ - val tempParquet = java.nio.file.Files.createTempDirectory("parquet_tuple_test_parquet_").toAbsolutePath.toString - try { - HadoopPlatformJobTest(new WriteToTypedParquetTupleJob(_), cluster) - .arg("output", tempParquet) - .run - - HadoopPlatformJobTest(new ReadFromTypedParquetTupleJob(_), cluster) - .arg("input", tempParquet) - .sink[Float]("output") { _.toSet shouldBe values.map(_.a.float).toSet } - .run - - HadoopPlatformJobTest(new ReadWithFilterPredicateJob(_), cluster) - .arg("input", tempParquet) - .sink[Boolean]("output") { _.toSet shouldBe values.filter(_.string == "B1").map(_.a.bool).toSet } - .run - - } finally { - deletePath(tempParquet) - } - } - } + HadoopPlatformJobTest(new WriteToTypedParquetTupleJob(_), cluster) + .arg("output", "output1") + .sink[SampleClassB](TypedParquet[SampleClassB, BReadSupport](Seq("output1"))) { _.toSet shouldBe values.toSet } + .run - def deletePath(path: String) = { - val dir = new File(path) - for { - files <- Option(dir.listFiles) - file <- files - } file.delete() - dir.delete() + HadoopPlatformJobTest(new ReadWithFilterPredicateJob(_), cluster) + .arg("input", "output1") + .arg("output", "output2") + .sink[Boolean]("output2") { _.toSet shouldBe values.filter(_.string == "B1").map(_.a.bool).toSet } + .run + + } } } object TestValues { - val values = Seq(SampleClassB("B1", 1, Some(4.0D), SampleClassA(bool = true, 5, 1L, 1.2F, 1)), - SampleClassB("B2", 3, Some(3.0D), SampleClassA(bool = false, 4, 2L, 2.3F, 2)), - SampleClassB("B3", 9, None, SampleClassA(bool = true, 6, 3L, 3.4F, 3)), - SampleClassB("B4", 8, Some(5.0D), SampleClassA(bool = false, 7, 4L, 4.5F, 4))) + val values = Seq( + SampleClassB("B1", Some(4.0D), SampleClassA(bool = true, 5, 1L, 1.2F, 1), List(1, 2), + List(SampleClassD(1, "1"), SampleClassD(2, "2")), Set(1D, 2D), Set(SampleClassF(1, 1F)), Map(1 -> "foo")), + SampleClassB("B2", Some(3.0D), SampleClassA(bool = false, 4, 2L, 2.3F, 2), List(3, 4), Nil, Set(3, 4), Set(), + Map(2 -> "bar"), Map(SampleClassD(0, "z") -> SampleClassF(0, 3), SampleClassD(0, "y") -> SampleClassF(2, 6))), + SampleClassB("B3", None, SampleClassA(bool = true, 6, 3L, 3.4F, 3), List(5, 6), + List(SampleClassD(3, "3"), SampleClassD(4, "4")), Set(5, 6), Set(SampleClassF(2, 2F))), + SampleClassB("B4", Some(5.0D), SampleClassA(bool = false, 7, 4L, 4.5F, 4), Nil, + List(SampleClassD(5, "5"), SampleClassD(6, "6")), Set(), Set(SampleClassF(3, 3F), SampleClassF(5, 4F)), + Map(3 -> "foo2"), Map(SampleClassD(0, "q") -> SampleClassF(4, 3)))) } case class SampleClassA(bool: Boolean, short: Short, long: Long, float: Float, byte: Byte) -case class SampleClassB(string: String, int: Int, double: Option[Double], a: SampleClassA) + +case class SampleClassB(string: String, double: Option[Double], a: SampleClassA, intList: List[Int], + dList: List[SampleClassD], doubleSet: Set[Double], fSet: Set[SampleClassF], intStringMap: Map[Int, String] = Map(), + dfMap: Map[SampleClassD, SampleClassF] = Map()) + case class SampleClassC(string: String, a: SampleClassA) +case class SampleClassD(x: Int, y: String) +case class SampleClassF(w: Byte, z: Float) object SampleClassB { val schema: String = Macros.caseClassParquetSchema[SampleClassB] @@ -94,19 +87,6 @@ class WriteToTypedParquetTupleJob(args: Args) extends Job(args) { TypedPipe.from(values).write(sink) } -/** - * Test job read from a typed parquet tuple and write the mapped value into a typed tsv sink - * To test typed parquet tuple can bse used as source and read data correctly - */ -class ReadFromTypedParquetTupleJob(args: Args) extends Job(args) { - - val inputPath = args.required("input") - - val input = TypedParquet[SampleClassB, BReadSupport](Seq(inputPath)) - - TypedPipe.from(input).map(_.a.float).write(TypedTsv[Float]("output")) -} - /** * Test job read from a typed parquet source with filter predicate and push down(SampleClassC takes only part of * SampleClassB's data) @@ -116,8 +96,10 @@ class ReadWithFilterPredicateJob(args: Args) extends Job(args) { val fp: FilterPredicate = FilterApi.eq(binaryColumn("string"), Binary.fromString("B1")) val inputPath = args.required("input") + val outputPath = args.required("output") val input = TypedParquet[SampleClassC, CReadSupport](Seq(inputPath), Some(fp)) - TypedPipe.from(input).map(_.a.bool).write(TypedTsv[Boolean]("output")) + TypedPipe.from(input).map(_.a.bool).write(TypedTsv[Boolean](outputPath)) } + diff --git a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/macros/MacroUnitTests.scala b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/macros/MacroUnitTests.scala index 66675f6f53..d7be8f650f 100644 --- a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/macros/MacroUnitTests.scala +++ b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/macros/MacroUnitTests.scala @@ -15,9 +15,17 @@ case class SampleClassD(a: String, b: Boolean, c: Option[Short], d: Int, e: Long case class SampleClassE(a: Int, b: Long, c: Short, d: Boolean, e: Float, f: Double, g: String, h: Byte) -case class SampleClassF(a: Option[SampleClassA]) +case class SampleClassF(a: Int, b: Option[SampleClassB], c: Double) -case class SampleClassG(a: Int, b: Option[SampleClassB], c: Double) +case class SampleClassG(a: Int, b: Option[List[Double]]) + +case class SampleClassH(a: Int, b: List[SampleClassA]) + +case class SampleClassI(a: Int, b: List[Option[Double]]) + +case class SampleClassJ(a: Map[Int, String]) + +case class SampleClassK(a: String, b: Map[SampleClassA, SampleClassB]) class MacroUnitTests extends WordSpec with Matchers with MockitoSugar { @@ -103,6 +111,94 @@ class MacroUnitTests extends WordSpec with Matchers with MockitoSugar { schema shouldEqual expectedSchema } + "Generate parquet schema for SampleClassG" in { + val schema = MessageTypeParser.parseMessageType(Macros.caseClassParquetSchema[SampleClassG]) + val expectedSchema = MessageTypeParser.parseMessageType(""" + |message SampleClassG { + | required int32 a; + | optional group b (LIST) { + | repeated group list { + | required double element; + | } + | } + |} + | + """.stripMargin) + schema shouldEqual expectedSchema + } + + "Generate parquet schema for SampleClassH" in { + val schema = MessageTypeParser.parseMessageType(Macros.caseClassParquetSchema[SampleClassH]) + val expectedSchema = MessageTypeParser.parseMessageType(""" + |message SampleClassH { + | required int32 a; + | required group b (LIST) { + | repeated group list { + | required group element { + | required int32 x; + | required binary y; + | } + | } + | } + |} + """.stripMargin) + schema shouldEqual expectedSchema + } + + "Generate parquet schema for SampleClassI" in { + val schema = MessageTypeParser.parseMessageType(Macros.caseClassParquetSchema[SampleClassI]) + val expectedSchema = MessageTypeParser.parseMessageType(""" + |message SampleClassI { + | required int32 a; + | required group b (LIST) { + | repeated group list { + | optional double element; + | } + | } + |} + """.stripMargin) + schema shouldEqual expectedSchema + } + + "Generate parquet schema for SampleClassJ" in { + val schema = MessageTypeParser.parseMessageType(Macros.caseClassParquetSchema[SampleClassJ]) + val expectedSchema = MessageTypeParser.parseMessageType(""" + |message SampleClassJ { + | required group a (MAP) { + | repeated group map (MAP_KEY_VALUE) { + | required int32 key; + | required binary value; + | } + | } + |} + """.stripMargin) + schema shouldEqual expectedSchema + } + + "Generate parquet schema for SampleClassK" in { + val schema = MessageTypeParser.parseMessageType(Macros.caseClassParquetSchema[SampleClassK]) + val expectedSchema = MessageTypeParser.parseMessageType(""" + message SampleClassK { + | required binary a; + | required group b (MAP) { + | repeated group map (MAP_KEY_VALUE) { + | required group key { + | required int32 x; + | required binary y; + | } + | required group value { + | required group a { + | required int32 x; + | required binary y; + | } + | required binary y; + | } + | } + | } + |} + """.stripMargin) + schema shouldEqual expectedSchema + } } "Macro case class converters generator" should { @@ -156,7 +252,33 @@ class MacroUnitTests extends WordSpec with Matchers with MockitoSugar { } "Generate converters for case class with optional nested class" in { - val converter = Macros.caseClassParquetTupleConverter[SampleClassG] + val converter = Macros.caseClassParquetTupleConverter[SampleClassF] + converter.start() + val a = converter.getConverter(0).asPrimitiveConverter() + a.addInt(0) + + val b = converter.getConverter(1).asGroupConverter() + b.start() + val ba = b.getConverter(0).asGroupConverter() + ba.start() + val baInt = ba.getConverter(0).asPrimitiveConverter() + baInt.addInt(2) + val baString = ba.getConverter(1).asPrimitiveConverter() + baString.addBinary(Binary.fromString("foo")) + ba.end() + + val bString = b.getConverter(1).asPrimitiveConverter() + bString.addBinary(Binary.fromString("b1")) + b.end() + + val c = converter.getConverter(2).asPrimitiveConverter() + c.addDouble(4D) + converter.end() + converter.currentValue shouldEqual SampleClassF(0, Some(SampleClassB(SampleClassA(2, "foo"), "b1")), 4D) + } + + "Generate converters for case class with list fields" in { + val converter = Macros.caseClassParquetTupleConverter[SampleClassF] converter.start() val a = converter.getConverter(0).asPrimitiveConverter() a.addInt(0) @@ -178,7 +300,42 @@ class MacroUnitTests extends WordSpec with Matchers with MockitoSugar { val c = converter.getConverter(2).asPrimitiveConverter() c.addDouble(4D) converter.end() - converter.currentValue shouldEqual SampleClassG(0, Some(SampleClassB(SampleClassA(2, "foo"), "b1")), 4D) + converter.currentValue shouldEqual SampleClassF(0, Some(SampleClassB(SampleClassA(2, "foo"), "b1")), 4D) + } + + "Generate converters for case class with map fields" in { + val converter = Macros.caseClassParquetTupleConverter[SampleClassK] + converter.start() + val a = converter.getConverter(0).asPrimitiveConverter() + a.addBinary(Binary.fromString("foo")) + + val keyValue = converter.getConverter(1).asGroupConverter().getConverter(0).asGroupConverter() + keyValue.start() + val key = keyValue.getConverter(0).asGroupConverter() + key.start() + val keyInt = key.getConverter(0).asPrimitiveConverter() + keyInt.addInt(2) + val keyString = key.getConverter(1).asPrimitiveConverter() + keyString.addBinary(Binary.fromString("bar")) + key.end() + + val value = keyValue.getConverter(1).asGroupConverter() + value.start() + val valueA = value.getConverter(0).asGroupConverter() + valueA.start() + val valueAInt = valueA.getConverter(0).asPrimitiveConverter() + valueAInt.addInt(2) + val valueAString = valueA.getConverter(1).asPrimitiveConverter() + valueAString.addBinary(Binary.fromString("bar")) + valueA.end() + val valueString = value.getConverter(1).asPrimitiveConverter() + valueString.addBinary(Binary.fromString("b1")) + value.end() + keyValue.end() + converter.end() + + converter.currentValue shouldEqual SampleClassK("foo", + Map(SampleClassA(2, "bar") -> SampleClassB(SampleClassA(2, "bar"), "b1"))) } } @@ -221,18 +378,18 @@ class MacroUnitTests extends WordSpec with Matchers with MockitoSugar { |start field h at 7 |write INT32 1 |end field h at 7 - |end Message""".stripMargin + |end message""".stripMargin } - "Generate write support for nested case class and optinal fields" in { - val writeSupportFn = Macros.caseClassWriteSupport[SampleClassG] + "Generate write support for nested case class and optional fields" in { + val writeSupportFn = Macros.caseClassWriteSupport[SampleClassF] - val g = SampleClassG(0, Some(SampleClassB(SampleClassA(2, "foo"), "b1")), 4D) + val f = SampleClassF(0, Some(SampleClassB(SampleClassA(2, "foo"), "b1")), 4D) - val schema = MessageTypeParser.parseMessageType(Macros.caseClassParquetSchema[SampleClassG]) + val schema = MessageTypeParser.parseMessageType(Macros.caseClassParquetSchema[SampleClassF]) val rc = new StringBuilderRecordConsumer - writeSupportFn(g, rc, schema) + writeSupportFn(f, rc, schema) rc.writeScenario shouldEqual """start message |start field a at 0 @@ -258,12 +415,12 @@ class MacroUnitTests extends WordSpec with Matchers with MockitoSugar { |start field c at 2 |write DOUBLE 4.0 |end field c at 2 - |end Message""".stripMargin + |end message""".stripMargin //test write tuple with optional field = None - val g2 = SampleClassG(0, None, 4D) + val f2 = SampleClassF(0, None, 4D) val rc2 = new StringBuilderRecordConsumer - writeSupportFn(g2, rc2, schema) + writeSupportFn(f2, rc2, schema) rc2.writeScenario shouldEqual """start message |start field a at 0 |write INT32 0 @@ -271,7 +428,160 @@ class MacroUnitTests extends WordSpec with Matchers with MockitoSugar { |start field c at 2 |write DOUBLE 4.0 |end field c at 2 - |end Message""".stripMargin + |end message""".stripMargin + } + + "Generate write support for case class with LIST fields" in { + //test write tuple with list of primitive fields + val writeSupportFn = Macros.caseClassWriteSupport[SampleClassI] + val i = SampleClassI(0, List(None, Some(2))) + val schema = MessageTypeParser.parseMessageType(Macros.caseClassParquetSchema[SampleClassI]) + val rc = new StringBuilderRecordConsumer + writeSupportFn(i, rc, schema) + + rc.writeScenario shouldEqual """start message + |start field a at 0 + |write INT32 0 + |end field a at 0 + |start field b at 1 + |start group + |start field list at 0 + |start group + |end group + |start group + |start field element at 0 + |write DOUBLE 2.0 + |end field element at 0 + |end group + |end field list at 0 + |end group + |end field b at 1 + |end message""".stripMargin + //test write list of nested class field + val writeSupportFn2 = Macros.caseClassWriteSupport[SampleClassH] + val h = SampleClassH(0, List(SampleClassA(2, "foo"), SampleClassA(2, "bar"))) + val schema2 = MessageTypeParser.parseMessageType(Macros.caseClassParquetSchema[SampleClassH]) + val rc2 = new StringBuilderRecordConsumer + writeSupportFn2(h, rc2, schema2) + + rc2.writeScenario shouldEqual """start message + |start field a at 0 + |write INT32 0 + |end field a at 0 + |start field b at 1 + |start group + |start field list at 0 + |start group + |start field element at 0 + |start group + |start field x at 0 + |write INT32 2 + |end field x at 0 + |start field y at 1 + |write BINARY foo + |end field y at 1 + |end group + |end field element at 0 + |end group + |start group + |start field element at 0 + |start group + |start field x at 0 + |write INT32 2 + |end field x at 0 + |start field y at 1 + |write BINARY bar + |end field y at 1 + |end group + |end field element at 0 + |end group + |end field list at 0 + |end group + |end field b at 1 + |end message""".stripMargin + + } + + "Generate write support for case class with MAP fields" in { + //test write tuple with map of primitive fields + val writeSupportFn = Macros.caseClassWriteSupport[SampleClassJ] + val j = SampleClassJ(Map(1 -> "foo", 2 -> "bar")) + val schema = MessageTypeParser.parseMessageType(Macros.caseClassParquetSchema[SampleClassJ]) + val rc = new StringBuilderRecordConsumer + writeSupportFn(j, rc, schema) + rc.writeScenario shouldEqual """start message + |start field a at 0 + |start group + |start field map at 0 + |start group + |start field key at 0 + |write INT32 1 + |end field key at 0 + |start field value at 1 + |write BINARY foo + |end field value at 1 + |end group + |start group + |start field key at 0 + |write INT32 2 + |end field key at 0 + |start field value at 1 + |write BINARY bar + |end field value at 1 + |end group + |end field map at 0 + |end group + |end field a at 0 + |end message""".stripMargin + + //test write Map of case class field + val writeSupportFn2 = Macros.caseClassWriteSupport[SampleClassK] + val k = SampleClassK("foo", Map(SampleClassA(2, "foo") -> SampleClassB(SampleClassA(2, "foo"), "bar"))) + val schema2 = MessageTypeParser.parseMessageType(Macros.caseClassParquetSchema[SampleClassK]) + val rc2 = new StringBuilderRecordConsumer + writeSupportFn2(k, rc2, schema2) + + rc2.writeScenario shouldEqual """start message + |start field a at 0 + |write BINARY foo + |end field a at 0 + |start field b at 1 + |start group + |start field map at 0 + |start group + |start field key at 0 + |start group + |start field x at 0 + |write INT32 2 + |end field x at 0 + |start field y at 1 + |write BINARY foo + |end field y at 1 + |end group + |end field key at 0 + |start field value at 1 + |start group + |start field a at 0 + |start group + |start field x at 0 + |write INT32 2 + |end field x at 0 + |start field y at 1 + |write BINARY foo + |end field y at 1 + |end group + |end field a at 0 + |start field y at 1 + |write BINARY bar + |end field y at 1 + |end group + |end field value at 1 + |end group + |end field map at 0 + |end group + |end field b at 1 + |end message""".stripMargin + } } } @@ -282,7 +592,7 @@ class StringBuilderRecordConsumer extends RecordConsumer { override def startMessage(): Unit = sb.append("start message\n") - override def endMessage(): Unit = sb.append("end Message") + override def endMessage(): Unit = sb.append("end message") override def addFloat(v: Float): Unit = sb.append(s"write FLOAT $v\n") From eb32954a22368afeefe9113a2cbdaffe291550f2 Mon Sep 17 00:00:00 2001 From: Sriram Krishnan Date: Thu, 23 Apr 2015 16:53:26 -0700 Subject: [PATCH 122/177] Adding a test for reducer estimation with groupAll (1 reducer) which is currently failing --- .../ReducerEstimatorTest.scala | 37 ++++++++++++++++++- 1 file changed, 35 insertions(+), 2 deletions(-) diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala index 5bba7fc79b..027299750a 100644 --- a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala +++ b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala @@ -9,8 +9,8 @@ object HipJob { val inSrc = TextLine(getClass.getResource("/hipster.txt").toString) val inScores = TypedTsv[(String, Double)](getClass.getResource("/scores.tsv").toString) val out = TypedTsv[Double]("output") - val countsPath = "counts.tsv" - val counts = TypedTsv[(String, Int)](countsPath) + val counts = TypedTsv[(String, Int)]("counts.tsv") + val size = TypedTsv[Long]("size.tsv") val correct = Map("hello" -> 1, "goodbye" -> 1, "world" -> 2) } @@ -53,6 +53,16 @@ class SimpleJob(args: Args) extends Job(args) { .write(counts) } +class GroupAllJob(args: Args) extends Job(args) { + import HipJob._ + TypedPipe.from(inSrc) + .flatMap(_.split("[^\\w]+")) + .groupAll + .size + .values + .write(size) +} + class ReducerEstimatorTestSingle extends WordSpec with Matchers with HadoopPlatformTest { import HipJob._ @@ -74,6 +84,29 @@ class ReducerEstimatorTestSingle extends WordSpec with Matchers with HadoopPlatf } } } + +class ReducerEstimatorTestGroupAll extends WordSpec with Matchers with HadoopPlatformTest { + import HipJob._ + + override def initialize() = cluster.initialize(Config.empty + .addReducerEstimator(classOf[InputSizeReducerEstimator]) + + (InputSizeReducerEstimator.BytesPerReducer -> (1L << 10).toString)) + + "Group-all job with reducer estimator" should { + "run with correct number of reducers (i.e. 1)" in { + HadoopPlatformJobTest(new SimpleJob(_), cluster) + .inspectCompletedFlow { flow => + val steps = flow.getFlowSteps.asScala + steps should have size 1 + + val conf = Config.fromHadoop(steps.head.getConfig) + conf.getNumReducers should contain (1) + } + .run + } + } +} + class ReducerEstimatorTestMulti extends WordSpec with Matchers with HadoopPlatformTest { import HipJob._ From 3c03d196dabf468f7fcbf5f77bd68aec3a47daa1 Mon Sep 17 00:00:00 2001 From: Sriram Krishnan Date: Thu, 23 Apr 2015 18:13:00 -0700 Subject: [PATCH 123/177] Now figuring out if withReducers() is being used explicitly via job conf --- .../src/main/scala/com/twitter/scalding/Config.scala | 3 +++ .../src/main/scala/com/twitter/scalding/RichPipe.scala | 2 ++ .../com/twitter/scalding/reducer_estimation/Common.scala | 8 ++------ .../reducer_estimation/ReducerEstimatorTest.scala | 2 +- 4 files changed, 8 insertions(+), 7 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala index cfffb2be44..fb779e92f9 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala @@ -325,6 +325,9 @@ object Config { /** Whether estimator should override manually-specified reducers. */ val ReducerEstimatorOverride = "scalding.reducer.estimator.override" + /** Whether the number of reducers has been set explicitly using a `withReducers` */ + val WithReducersSetExplicitly = "scalding.with.reducers.set.explicitly" + val empty: Config = Config(Map.empty) /* diff --git a/scalding-core/src/main/scala/com/twitter/scalding/RichPipe.scala b/scalding-core/src/main/scala/com/twitter/scalding/RichPipe.scala index 1d3d29b2fa..62ad3c3ce0 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/RichPipe.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/RichPipe.scala @@ -45,6 +45,8 @@ object RichPipe extends java.io.Serializable { if (reducers > 0) { p.getStepConfigDef() .setProperty(REDUCER_KEY, reducers.toString) + p.getStepConfigDef() + .setProperty(Config.WithReducersSetExplicitly, "true") } else if (reducers != -1) { throw new IllegalArgumentException(s"Number of reducers must be non-negative. Got: ${reducers}") } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala index 60fc54d1c1..2e1cf359dc 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala @@ -72,12 +72,8 @@ object ReducerEstimatorStepStrategy extends FlowStepStrategy[JobConf] { val flowNumReducers = flow.getConfig.get(Config.HadoopNumReducers) val stepNumReducers = conf.get(Config.HadoopNumReducers) - // assuming that if the step's reducers is different than the default for the flow, - // it was probably set by `withReducers` explicitly. This isn't necessarily true -- - // Cascading may have changed it for its own reasons. - // TODO: disambiguate this by setting something in JobConf when `withReducers` is called - // (will be addressed by https://github.com/twitter/scalding/pull/973) - val setExplicitly = flowNumReducers != stepNumReducers + // whether the reducers have been set explicitly with `withReducers` + val setExplicitly = conf.getBoolean(Config.WithReducersSetExplicitly, false) // log in JobConf what was explicitly set by 'withReducers' if (setExplicitly) conf.set(EstimatorConfig.originalNumReducers, stepNumReducers) diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala index 027299750a..866f6ccb78 100644 --- a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala +++ b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala @@ -94,7 +94,7 @@ class ReducerEstimatorTestGroupAll extends WordSpec with Matchers with HadoopPla "Group-all job with reducer estimator" should { "run with correct number of reducers (i.e. 1)" in { - HadoopPlatformJobTest(new SimpleJob(_), cluster) + HadoopPlatformJobTest(new GroupAllJob(_), cluster) .inspectCompletedFlow { flow => val steps = flow.getFlowSteps.asScala steps should have size 1 From 29921adefce7b3cbde8b81a5dfb71a20b7edf2fa Mon Sep 17 00:00:00 2001 From: Sriram Krishnan Date: Fri, 24 Apr 2015 13:37:48 -0700 Subject: [PATCH 124/177] Fixing the ReducerEstimatorTestMulti test that was broken (groupAll was expecting > 1 reducers) --- .../com/twitter/scalding/reducer_estimation/Common.scala | 2 -- .../reducer_estimation/ReducerEstimatorTest.scala | 9 ++++----- 2 files changed, 4 insertions(+), 7 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala index 2e1cf359dc..6c17c701a4 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala @@ -68,8 +68,6 @@ object ReducerEstimatorStepStrategy extends FlowStepStrategy[JobConf] { preds: JList[FlowStep[JobConf]], step: FlowStep[JobConf]): Unit = { val conf = step.getConfig - - val flowNumReducers = flow.getConfig.get(Config.HadoopNumReducers) val stepNumReducers = conf.get(Config.HadoopNumReducers) // whether the reducers have been set explicitly with `withReducers` diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala index 866f6ccb78..316bf50f2c 100644 --- a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala +++ b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala @@ -32,12 +32,11 @@ class HipJob(args: Args) extends Job(args) { wordCounts.leftJoin(scores) .mapValues{ case (count, score) => (count, score.getOrElse(0.0)) } - - // force another M/R step + // force another M/R step - should use reducer estimation .toTypedPipe .map{ case (word, (count, score)) => (count, score) } .group.sum - + // force another M/R step - this should force 1 reducer because it is essentially a groupAll .toTypedPipe.values.sum .write(out) @@ -112,7 +111,7 @@ class ReducerEstimatorTestMulti extends WordSpec with Matchers with HadoopPlatfo override def initialize() = cluster.initialize(Config.empty .addReducerEstimator(classOf[InputSizeReducerEstimator]) + - (InputSizeReducerEstimator.BytesPerReducer -> (1L << 16).toString)) + (InputSizeReducerEstimator.BytesPerReducer -> (1L << 10).toString)) "Multi-step job with reducer estimator" should { "run with correct number of reducers in each step" in { @@ -121,7 +120,7 @@ class ReducerEstimatorTestMulti extends WordSpec with Matchers with HadoopPlatfo .inspectCompletedFlow { flow => val steps = flow.getFlowSteps.asScala val reducers = steps.map(_.getConfig.getInt(Config.HadoopNumReducers, 0)).toList - reducers shouldBe List(1, 1, 2) + reducers shouldBe List(3, 1, 1) } .run } From 3fe283cee1c30630531ae36130c02c30257fc686 Mon Sep 17 00:00:00 2001 From: Sriram Krishnan Date: Fri, 24 Apr 2015 14:00:04 -0700 Subject: [PATCH 125/177] Bonus test - to verify that the reducer estimator override actually works --- .../ReducerEstimatorTest.scala | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala index 316bf50f2c..d141587077 100644 --- a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala +++ b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala @@ -48,6 +48,8 @@ class SimpleJob(args: Args) extends Job(args) { .flatMap(_.split("[^\\w]+")) .map(_.toLowerCase -> 1) .group + // force the number of reducers to two, to test with/without estimation + .withReducers(2) .sum .write(counts) } @@ -69,6 +71,29 @@ class ReducerEstimatorTestSingle extends WordSpec with Matchers with HadoopPlatf .addReducerEstimator(classOf[InputSizeReducerEstimator]) + (InputSizeReducerEstimator.BytesPerReducer -> (1L << 10).toString)) + "Single-step job with reducer estimator" should { + "run with correct number of reducers" in { + HadoopPlatformJobTest(new SimpleJob(_), cluster) + .inspectCompletedFlow { flow => + val steps = flow.getFlowSteps.asScala + steps should have size 1 + + val conf = Config.fromHadoop(steps.head.getConfig) + conf.getNumReducers should contain (2) + } + .run + } + } +} + +class ReducerEstimatorTestSingleOverride extends WordSpec with Matchers with HadoopPlatformTest { + import HipJob._ + + override def initialize() = cluster.initialize(Config.empty + .addReducerEstimator(classOf[InputSizeReducerEstimator]) + + (InputSizeReducerEstimator.BytesPerReducer -> (1L << 10).toString) + + (Config.ReducerEstimatorOverride -> "true")) + "Single-step job with reducer estimator" should { "run with correct number of reducers" in { HadoopPlatformJobTest(new SimpleJob(_), cluster) From ac0869adc4dd816b1fb316a4d845629d3614987a Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Wed, 29 Apr 2015 10:48:55 -1000 Subject: [PATCH 126/177] Fix SimpleDateFormat caching by default --- .../main/scala/com/twitter/scalding/DateOps.scala | 9 ++++++--- .../main/scala/com/twitter/scalding/DateParser.scala | 12 +++++++++++- 2 files changed, 17 insertions(+), 4 deletions(-) diff --git a/scalding-date/src/main/scala/com/twitter/scalding/DateOps.scala b/scalding-date/src/main/scala/com/twitter/scalding/DateOps.scala index 9ce0c87d97..442862af86 100644 --- a/scalding-date/src/main/scala/com/twitter/scalding/DateOps.scala +++ b/scalding-date/src/main/scala/com/twitter/scalding/DateOps.scala @@ -51,10 +51,13 @@ object DateOps extends java.io.Serializable { /** * Return the guessed format for this datestring */ - def getFormat(s: String): Option[String] = { - DATE_FORMAT_VALIDATORS.find{ _._2.findFirstIn(prepare(s)).isDefined }.map(_._1) - } + def getFormat(s: String): Option[String] = + DATE_FORMAT_VALIDATORS.find { _._2.findFirstIn(prepare(s)).isDefined }.map(_._1) + /** + * The DateParser returned here is based on SimpleDateFormat, which is not thread-safe. + * Do not share the result across threads. + */ def getDateParser(s: String): Option[DateParser] = getFormat(s).map { fmt => DateParser.from(new SimpleDateFormat(fmt)).contramap(prepare) } } diff --git a/scalding-date/src/main/scala/com/twitter/scalding/DateParser.scala b/scalding-date/src/main/scala/com/twitter/scalding/DateParser.scala index 2236cfa060..5a20c344b6 100644 --- a/scalding-date/src/main/scala/com/twitter/scalding/DateParser.scala +++ b/scalding-date/src/main/scala/com/twitter/scalding/DateParser.scala @@ -41,8 +41,11 @@ object DateParser { /** * This is scalding's default date parser. You can choose this * by setting an implicit val DateParser. + * Note that DateParsers using SimpleDateFormat from Java are + * not thread-safe, thus the def here. You can cache the result + * if you are sure */ - val default: DateParser = new DateParser { + def default: DateParser = new DateParser { def parse(s: String)(implicit tz: TimeZone) = DateOps.getDateParser(s) .map { p => p.parse(s) } @@ -56,6 +59,10 @@ object DateParser { /** Using the type-class pattern */ def parse(s: String)(implicit tz: TimeZone, p: DateParser): Try[RichDate] = p.parse(s)(tz) + /** + * Note that DateFormats in Java are generally not thread-safe, + * so you should not share the result here across threads + */ implicit def from(df: DateFormat): DateParser = new DateParser { def parse(s: String)(implicit tz: TimeZone) = Try { df.setTimeZone(tz) @@ -63,6 +70,9 @@ object DateParser { } } + /** + * This ignores the time-zone assuming it must be in the String + */ def from(fn: String => RichDate) = new DateParser { def parse(s: String)(implicit tz: TimeZone) = Try(fn(s)) } From 372420ae6d67ddda19f61f680d57542de7e8e3f5 Mon Sep 17 00:00:00 2001 From: Alex Levenson Date: Thu, 30 Apr 2015 01:06:02 -0700 Subject: [PATCH 127/177] Add TypedPipeDiff for comparing typed pipes --- .../typed/HashEqualsArrayWrapper.scala | 319 ++++++++++++++++++ .../scalding/typed/TypedPipeDiff.scala | 158 +++++++++ .../typed/HashEqualsArrayWrapperTest.scala | 72 ++++ .../scalding/typed/TypedPipeDiffTest.scala | 240 +++++++++++++ 4 files changed, 789 insertions(+) create mode 100644 scalding-core/src/main/scala/com/twitter/scalding/typed/HashEqualsArrayWrapper.scala create mode 100644 scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipeDiff.scala create mode 100644 scalding-core/src/test/scala/com/twitter/scalding/typed/HashEqualsArrayWrapperTest.scala create mode 100644 scalding-core/src/test/scala/com/twitter/scalding/typed/TypedPipeDiffTest.scala diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/HashEqualsArrayWrapper.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/HashEqualsArrayWrapper.scala new file mode 100644 index 0000000000..c5eabbb05d --- /dev/null +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/HashEqualsArrayWrapper.scala @@ -0,0 +1,319 @@ +package com.twitter.scalding.typed + +import java.util + +import reflect.ClassTag + +sealed trait HashEqualsArrayWrapper[T] { + def wrapped: Array[T] +} + +object HashEqualsArrayWrapper { + + // gross way to make specialized wrappers for primitives + // relies on the fact that Array generic types are not erased + def wrap[T](arr: Array[T]): HashEqualsArrayWrapper[T] = + wrapFn[T, Array[T]](arr.getClass.asInstanceOf[Class[Array[T]]])(arr) + + def wrapFn[T, A <: Array[T]](clazz: Class[A]): A => HashEqualsArrayWrapper[T] = { + val fn = clazz match { + case c if classOf[Array[Long]].equals(c) => a: Array[Long] => new HashEqualsLongArrayWrapper(a) + case c if classOf[Array[Int]].equals(c) => a: Array[Int] => new HashEqualsIntArrayWrapper(a) + case c if classOf[Array[Short]].equals(c) => a: Array[Short] => new HashEqualsShortArrayWrapper(a) + case c if classOf[Array[Char]].equals(c) => a: Array[Char] => new HashEqualsCharArrayWrapper(a) + case c if classOf[Array[Byte]].equals(c) => a: Array[Byte] => new HashEqualsByteArrayWrapper(a) + case c if classOf[Array[Boolean]].equals(c) => a: Array[Boolean] => new HashEqualsBooleanArrayWrapper(a) + case c if classOf[Array[Float]].equals(c) => a: Array[Float] => new HashEqualsFloatArrayWrapper(a) + case c if classOf[Array[Double]].equals(c) => a: Array[Double] => new HashEqualsDoubleArrayWrapper(a) + case c => a: Array[T] => new HashEqualsObjectArrayWrapper(a) + } + + fn.asInstanceOf[(Array[T] => HashEqualsArrayWrapper[T])] + } + + def wrapFn[T: ClassTag]: Array[T] => HashEqualsArrayWrapper[T] = + wrapFn(scala.reflect.classTag[T].runtimeClass.asInstanceOf[Class[Array[T]]]) + + implicit val longArrayOrd: Ordering[Array[Long]] = new Ordering[Array[Long]] { + override def compare(x: Array[Long], y: Array[Long]): Int = { + val lenCmp = java.lang.Integer.compare(x.length, y.length) + + if (lenCmp != 0) { + lenCmp + } else if (x.length == 0) { + 0 + } else { + val len = x.length + var i = 1 + var cmp = java.lang.Long.compare(x(0), y(0)) + while (i < len && cmp == 0) { + cmp = java.lang.Long.compare(x(i), y(i)) + i = i + 1 + } + cmp + } + } + } + + implicit val intArrayOrd: Ordering[Array[Int]] = new Ordering[Array[Int]] { + override def compare(x: Array[Int], y: Array[Int]): Int = { + val lenCmp = java.lang.Integer.compare(x.length, y.length) + + if (lenCmp != 0) { + lenCmp + } else if (x.length == 0) { + 0 + } else { + val len = x.length + var i = 1 + var cmp = java.lang.Integer.compare(x(0), y(0)) + while (i < len && cmp == 0) { + cmp = java.lang.Integer.compare(x(i), y(i)) + i = i + 1 + } + cmp + } + } + } + + implicit val shortArrayOrd: Ordering[Array[Short]] = new Ordering[Array[Short]] { + override def compare(x: Array[Short], y: Array[Short]): Int = { + val lenCmp = java.lang.Integer.compare(x.length, y.length) + + if (lenCmp != 0) { + lenCmp + } else if (x.length == 0) { + 0 + } else { + val len = x.length + var i = 1 + var cmp = java.lang.Short.compare(x(0), y(0)) + while (i < len && cmp == 0) { + cmp = java.lang.Short.compare(x(i), y(i)) + i = i + 1 + } + cmp + } + } + } + + implicit val charArrayOrd: Ordering[Array[Char]] = new Ordering[Array[Char]] { + override def compare(x: Array[Char], y: Array[Char]): Int = { + val lenCmp = java.lang.Integer.compare(x.length, y.length) + + if (lenCmp != 0) { + lenCmp + } else if (x.length == 0) { + 0 + } else { + val len = x.length + var i = 1 + var cmp = java.lang.Character.compare(x(0), y(0)) + while (i < len && cmp == 0) { + cmp = java.lang.Character.compare(x(i), y(i)) + i = i + 1 + } + cmp + } + } + } + + implicit val byteArrayOrd: Ordering[Array[Byte]] = new Ordering[Array[Byte]] { + override def compare(x: Array[Byte], y: Array[Byte]): Int = { + val lenCmp = java.lang.Integer.compare(x.length, y.length) + + if (lenCmp != 0) { + lenCmp + } else if (x.length == 0) { + 0 + } else { + val len = x.length + var i = 1 + var cmp = java.lang.Byte.compare(x(0), y(0)) + while (i < len && cmp == 0) { + cmp = java.lang.Byte.compare(x(i), y(i)) + i = i + 1 + } + cmp + } + } + } + + implicit val booleanArrayOrd: Ordering[Array[Boolean]] = new Ordering[Array[Boolean]] { + override def compare(x: Array[Boolean], y: Array[Boolean]): Int = { + val lenCmp = java.lang.Integer.compare(x.length, y.length) + + if (lenCmp != 0) { + lenCmp + } else if (x.length == 0) { + 0 + } else { + val len = x.length + var i = 1 + var cmp = java.lang.Boolean.compare(x(0), y(0)) + while (i < len && cmp == 0) { + cmp = java.lang.Boolean.compare(x(i), y(i)) + i = i + 1 + } + cmp + } + } + } + + implicit val floatArrayOrd: Ordering[Array[Float]] = new Ordering[Array[Float]] { + override def compare(x: Array[Float], y: Array[Float]): Int = { + val lenCmp = java.lang.Integer.compare(x.length, y.length) + + if (lenCmp != 0) { + lenCmp + } else if (x.length == 0) { + 0 + } else { + val len = x.length + var i = 1 + var cmp = java.lang.Float.compare(x(0), y(0)) + while (i < len && cmp == 0) { + cmp = java.lang.Float.compare(x(i), y(i)) + i = i + 1 + } + cmp + } + } + } + + implicit val doubleArrayOrd: Ordering[Array[Double]] = new Ordering[Array[Double]] { + override def compare(x: Array[Double], y: Array[Double]): Int = { + val lenCmp = java.lang.Integer.compare(x.length, y.length) + + if (lenCmp != 0) { + lenCmp + } else if (x.length == 0) { + 0 + } else { + val len = x.length + var i = 1 + var cmp = java.lang.Double.compare(x(0), y(0)) + while (i < len && cmp == 0) { + cmp = java.lang.Double.compare(x(i), y(i)) + i = i + 1 + } + cmp + } + } + } + + implicit val hashEqualsLongOrdering: Ordering[HashEqualsArrayWrapper[Long]] = new Ordering[HashEqualsArrayWrapper[Long]] { + override def compare(x: HashEqualsArrayWrapper[Long], y: HashEqualsArrayWrapper[Long]): Int = + longArrayOrd.compare(x.wrapped, y.wrapped) + } + + implicit val hashEqualsIntOrdering: Ordering[HashEqualsArrayWrapper[Int]] = new Ordering[HashEqualsArrayWrapper[Int]] { + override def compare(x: HashEqualsArrayWrapper[Int], y: HashEqualsArrayWrapper[Int]): Int = + intArrayOrd.compare(x.wrapped, y.wrapped) + } + + implicit val hashEqualsShortOrdering: Ordering[HashEqualsArrayWrapper[Short]] = new Ordering[HashEqualsArrayWrapper[Short]] { + override def compare(x: HashEqualsArrayWrapper[Short], y: HashEqualsArrayWrapper[Short]): Int = + shortArrayOrd.compare(x.wrapped, y.wrapped) + } + + implicit val hashEqualsCharOrdering: Ordering[HashEqualsArrayWrapper[Char]] = new Ordering[HashEqualsArrayWrapper[Char]] { + override def compare(x: HashEqualsArrayWrapper[Char], y: HashEqualsArrayWrapper[Char]): Int = + charArrayOrd.compare(x.wrapped, y.wrapped) + } + + implicit val hashEqualsByteOrdering: Ordering[HashEqualsArrayWrapper[Byte]] = new Ordering[HashEqualsArrayWrapper[Byte]] { + override def compare(x: HashEqualsArrayWrapper[Byte], y: HashEqualsArrayWrapper[Byte]): Int = + byteArrayOrd.compare(x.wrapped, y.wrapped) + } + + implicit val hashEqualsBooleanOrdering: Ordering[HashEqualsArrayWrapper[Boolean]] = new Ordering[HashEqualsArrayWrapper[Boolean]] { + override def compare(x: HashEqualsArrayWrapper[Boolean], y: HashEqualsArrayWrapper[Boolean]): Int = + booleanArrayOrd.compare(x.wrapped, y.wrapped) + } + + implicit val hashEqualsFloatOrdering: Ordering[HashEqualsArrayWrapper[Float]] = new Ordering[HashEqualsArrayWrapper[Float]] { + override def compare(x: HashEqualsArrayWrapper[Float], y: HashEqualsArrayWrapper[Float]): Int = + floatArrayOrd.compare(x.wrapped, y.wrapped) + } + + implicit val hashEqualsDoubleOrdering: Ordering[HashEqualsArrayWrapper[Double]] = new Ordering[HashEqualsArrayWrapper[Double]] { + override def compare(x: HashEqualsArrayWrapper[Double], y: HashEqualsArrayWrapper[Double]): Int = + doubleArrayOrd.compare(x.wrapped, y.wrapped) + } + +} + +final class HashEqualsLongArrayWrapper(override val wrapped: Array[Long]) extends HashEqualsArrayWrapper[Long] { + override def hashCode(): Int = util.Arrays.hashCode(wrapped) + override def equals(obj: scala.Any): Boolean = obj match { + case other: HashEqualsLongArrayWrapper => util.Arrays.equals(wrapped, other.wrapped) + case _ => false + } +} + +final class HashEqualsIntArrayWrapper(override val wrapped: Array[Int]) extends HashEqualsArrayWrapper[Int] { + override def hashCode(): Int = util.Arrays.hashCode(wrapped) + override def equals(obj: scala.Any): Boolean = obj match { + case other: HashEqualsIntArrayWrapper => util.Arrays.equals(wrapped, other.wrapped) + case _ => false + } +} + +final class HashEqualsShortArrayWrapper(override val wrapped: Array[Short]) extends HashEqualsArrayWrapper[Short] { + override def hashCode(): Int = util.Arrays.hashCode(wrapped) + override def equals(obj: scala.Any): Boolean = obj match { + case other: HashEqualsShortArrayWrapper => util.Arrays.equals(wrapped, other.wrapped) + case _ => false + } +} + +final class HashEqualsCharArrayWrapper(override val wrapped: Array[Char]) extends HashEqualsArrayWrapper[Char] { + override def hashCode(): Int = util.Arrays.hashCode(wrapped) + override def equals(obj: scala.Any): Boolean = obj match { + case other: HashEqualsCharArrayWrapper => util.Arrays.equals(wrapped, other.wrapped) + case _ => false + } +} + +final class HashEqualsByteArrayWrapper(override val wrapped: Array[Byte]) extends HashEqualsArrayWrapper[Byte] { + override def hashCode(): Int = util.Arrays.hashCode(wrapped) + override def equals(obj: scala.Any): Boolean = obj match { + case other: HashEqualsByteArrayWrapper => util.Arrays.equals(wrapped, other.wrapped) + case _ => false + } +} + +final class HashEqualsBooleanArrayWrapper(override val wrapped: Array[Boolean]) extends HashEqualsArrayWrapper[Boolean] { + override def hashCode(): Int = util.Arrays.hashCode(wrapped) + override def equals(obj: scala.Any): Boolean = obj match { + case other: HashEqualsBooleanArrayWrapper => util.Arrays.equals(wrapped, other.wrapped) + case _ => false + } +} + +final class HashEqualsFloatArrayWrapper(override val wrapped: Array[Float]) extends HashEqualsArrayWrapper[Float] { + override def hashCode(): Int = util.Arrays.hashCode(wrapped) + override def equals(obj: scala.Any): Boolean = obj match { + case other: HashEqualsFloatArrayWrapper => util.Arrays.equals(wrapped, other.wrapped) + case _ => false + } +} + +final class HashEqualsDoubleArrayWrapper(override val wrapped: Array[Double]) extends HashEqualsArrayWrapper[Double] { + override def hashCode(): Int = util.Arrays.hashCode(wrapped) + + override def equals(obj: scala.Any): Boolean = obj match { + case other: HashEqualsDoubleArrayWrapper => util.Arrays.equals(wrapped, other.wrapped) + case _ => false + } +} + +final class HashEqualsObjectArrayWrapper[T](override val wrapped: Array[T]) extends HashEqualsArrayWrapper[T] { + private val wrappedInternal = wrapped.toSeq + override def hashCode(): Int = wrappedInternal.hashCode() + override def equals(obj: scala.Any): Boolean = obj match { + case other: HashEqualsObjectArrayWrapper[T] => wrappedInternal.equals(other.wrappedInternal) + case _ => false + } +} diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipeDiff.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipeDiff.scala new file mode 100644 index 0000000000..bb81b91277 --- /dev/null +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipeDiff.scala @@ -0,0 +1,158 @@ +package com.twitter.scalding.typed + +import java.io.{ BufferedWriter, File, FileWriter } + +import com.twitter.scalding.Execution + +import scala.reflect.ClassTag + +/** + * Some methods for comparing two typed pipes and finding out the difference between them. + * + * Has support for the normal case where the typed pipes are pipes of objects usable as keys + * in scalding (have an ordering, proper equals and hashCode), as well as some special cases + * for dealing with Arrays and thrift objects. + * + * See diffByHashCode for comparing typed pipes of objects that have no ordering but a stable hash code + * (such as Scrooge thrift). + * + * See diffByGroup for comparing typed pipes of objects that have no ordering *and* an unstable hash code. + */ +object TypedPipeDiff { + + /** + * Returns a mapping from T to a count of the occurrences of T in the left and right pipes, + * only for cases where the counts are not equal. + * + * Requires that T have an ordering and a hashCode and equals that is stable across JVMs (not reference based). + * See diffArrayPipes for diffing pipes of arrays, since arrays do not meet these requirements by default. + */ + def diff[T: Ordering](left: TypedPipe[T], right: TypedPipe[T], reducers: Option[Int] = None): UnsortedGrouped[T, (Long, Long)] = { + val lefts = left.map { x => (x, (1L, 0L)) } + val rights = right.map { x => (x, (0L, 1L)) } + val counts = (lefts ++ rights).sumByKey + val diff = counts.filter { case (key, (lCount, rCount)) => lCount != rCount } + reducers.map(diff.withReducers).getOrElse(diff) + } + + /** + * Same as diffByHashCode, but takes care to wrap the Array[T] in a wrapper, + * which has the correct hashCode and equals needed. This does not involve + * copying the arrays, just wrapping them, and is specialized for primitive arrays. + */ + def diffArrayPipes[T: ClassTag](left: TypedPipe[Array[T]], + right: TypedPipe[Array[T]], + reducers: Option[Int] = None): TypedPipe[(Array[T], (Long, Long))] = { + + // cache this instead of reflecting on every single array + val wrapFn = HashEqualsArrayWrapper.wrapFn[T] + + diffByHashCode(left.map(wrapFn), right.map(wrapFn), reducers) + .map { case (k, counts) => (k.wrapped, counts) } + } + + /** + * NOTE: Prefer diff over this method if you can find or construct an Ordering[T]. + * + * Returns a mapping from T to a count of the occurrences of T in the left and right pipes, + * only for cases where the counts are not equal. + * + * This implementation does not require an ordering on T, but does require a function (groupByFn) + * that extracts a value of type K (which has an ordering) from a record of type T. + * + * The groupByFn should be something that partitions records as evenly as possible, + * because all unique records that result in the same groupByFn value will be materialized into an in memory map. + * + * groupByFn must be a pure function, such that: + * x == y implies that groupByFn(x) == groupByFn(y) + * + * T must have a hash code suitable for use in a hash map on a single JVM (doesn't have to be stable cross JVM) + * K must have a hash code this *is* stable across JVMs. + * K must have an ordering. + * + * Example groupByFns would be x => x.hashCode, assuming x's hashCode is stable across jvms, + * or maybe x => x.timestamp, if x's hashCode is not stable, assuming there's shouldn't be too + * many records with the same timestamp. + */ + def diffByGroup[T, K: Ordering]( + left: TypedPipe[T], + right: TypedPipe[T], + reducers: Option[Int] = None)(groupByFn: T => K): TypedPipe[(T, (Long, Long))] = { + + val lefts = left.map { t => (groupByFn(t), Map(t -> (1L, 0L))) } + val rights = right.map { t => (groupByFn(t), Map(t -> (0L, 1L))) } + + val diff = (lefts ++ rights) + .sumByKey + .flattenValues + .filter { case (k, (t, (lCount, rCount))) => lCount != rCount } + + reducers.map(diff.withReducers).getOrElse(diff).values + } + + /** + * NOTE: Prefer diff over this method if you can find or construct an Ordering[T]. + * + * Same as diffByGroup but uses T.hashCode as the groupByFn + * + * This method does an exact diff, it does not use the hashCode as a proxy for equality. + */ + def diffByHashCode[T]( + left: TypedPipe[T], + right: TypedPipe[T], + reducers: Option[Int] = None): TypedPipe[(T, (Long, Long))] = diffByGroup(left, right, reducers)(_.hashCode) + + def diffSummary[T](diff: TypedPipe[(T, (Long, Long))]): TypedPipe[String] = { + diff.map { + case (key, (lCount, rCount)) => + s"For key $key there were $lCount records in the left pipe and $rCount records in the right pipe" + } + } + + /** + * Returns an Execution that writes the difference into a local file. + * Truncates to maxDiff lines of output. + */ + def writeDiffSummaryToFile[T](diff: TypedPipe[(T, (Long, Long))], f: File, maxDiff: Int): Execution[Unit] = + + diffSummary(diff.groupAll.bufferedTake(maxDiff).values) + .toIterableExecution + .map { iter => + var writer: BufferedWriter = null + + try { + writer = new BufferedWriter(new FileWriter(f)) + iter.foreach { line => + writer.write(line) + writer.write("\n") + } + } finally { + if (writer != null) { + writer.close() + } + } + + () + } + + object Enrichments { + + implicit class Diff[T](val left: TypedPipe[T]) extends AnyVal { + + def diff(right: TypedPipe[T], reducers: Option[Int] = None)(implicit ev: Ordering[T]): UnsortedGrouped[T, (Long, Long)] = + TypedPipeDiff.diff(left, right, reducers) + + def diffByGroup[K: Ordering](right: TypedPipe[T], reducers: Option[Int] = None)(groupByFn: T => K): TypedPipe[(T, (Long, Long))] = + TypedPipeDiff.diffByGroup(left, right, reducers)(groupByFn) + + def diffByHashCode(right: TypedPipe[T], reducers: Option[Int] = None): TypedPipe[(T, (Long, Long))] = TypedPipeDiff.diffByHashCode(left, right, reducers) + } + + implicit class DiffArray[T](val left: TypedPipe[Array[T]]) extends AnyVal { + + def diffArrayPipes(right: TypedPipe[Array[T]], reducers: Option[Int] = None)(implicit ev: ClassTag[T]): TypedPipe[(Array[T], (Long, Long))] = + TypedPipeDiff.diffArrayPipes(left, right, reducers) + } + + } +} diff --git a/scalding-core/src/test/scala/com/twitter/scalding/typed/HashEqualsArrayWrapperTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/typed/HashEqualsArrayWrapperTest.scala new file mode 100644 index 0000000000..f504e647b0 --- /dev/null +++ b/scalding-core/src/test/scala/com/twitter/scalding/typed/HashEqualsArrayWrapperTest.scala @@ -0,0 +1,72 @@ +package com.twitter.scalding.typed + +import org.scalacheck.{ Arbitrary, Prop } +import org.scalatest.PropSpec +import org.scalatest.prop.{ Checkers, PropertyChecks } + +object HashArrayEqualsWrapperLaws { + + def check2[T](ordToTest: Ordering[HashEqualsArrayWrapper[T]])(implicit ord: Ordering[T], arb: Arbitrary[Array[T]]): Prop = + + Prop.forAll { (left: Array[T], right: Array[T]) => + + val leftWrapped = HashEqualsArrayWrapper.wrap(left) + val rightWrapped = HashEqualsArrayWrapper.wrap(right) + + import scala.Ordering.Implicits.seqDerivedOrdering + + val slowOrd: Ordering[Seq[T]] = seqDerivedOrdering[Seq, T](ord) + + val cmp = ordToTest.compare(leftWrapped, rightWrapped) + + val lenCmp = java.lang.Integer.compare(leftWrapped.wrapped.length, rightWrapped.wrapped.length) + if (lenCmp != 0) { + cmp.signum == lenCmp.signum + } else { + cmp.signum == slowOrd.compare(leftWrapped.wrapped.toSeq, rightWrapped.wrapped.toSeq).signum + } + } + + def check[T](ordToTest: Ordering[Array[T]])(implicit ord: Ordering[T], arb: Arbitrary[Array[T]]): Prop = + + Prop.forAll { (left: Array[T], right: Array[T]) => + import scala.Ordering.Implicits.seqDerivedOrdering + + val slowOrd: Ordering[Seq[T]] = seqDerivedOrdering[Seq, T](ord) + + val cmp = ordToTest.compare(left, right) + + val lenCmp = java.lang.Integer.compare(left.length, right.length) + if (lenCmp != 0) { + cmp.signum == lenCmp.signum + } else { + cmp.signum == slowOrd.compare(left.toSeq, right.toSeq).signum + } + } +} + +class HashArrayEqualsWrapperTest extends PropSpec with PropertyChecks with Checkers { + + property("Specialized orderings obey all laws for Arrays") { + check(HashArrayEqualsWrapperLaws.check(HashEqualsArrayWrapper.longArrayOrd)) + check(HashArrayEqualsWrapperLaws.check(HashEqualsArrayWrapper.intArrayOrd)) + check(HashArrayEqualsWrapperLaws.check(HashEqualsArrayWrapper.shortArrayOrd)) + check(HashArrayEqualsWrapperLaws.check(HashEqualsArrayWrapper.charArrayOrd)) + check(HashArrayEqualsWrapperLaws.check(HashEqualsArrayWrapper.byteArrayOrd)) + check(HashArrayEqualsWrapperLaws.check(HashEqualsArrayWrapper.booleanArrayOrd)) + check(HashArrayEqualsWrapperLaws.check(HashEqualsArrayWrapper.floatArrayOrd)) + check(HashArrayEqualsWrapperLaws.check(HashEqualsArrayWrapper.doubleArrayOrd)) + } + + property("Specialized orderings obey all laws for wrapped Arrays") { + check(HashArrayEqualsWrapperLaws.check2(HashEqualsArrayWrapper.hashEqualsLongOrdering)) + check(HashArrayEqualsWrapperLaws.check2(HashEqualsArrayWrapper.hashEqualsIntOrdering)) + check(HashArrayEqualsWrapperLaws.check2(HashEqualsArrayWrapper.hashEqualsShortOrdering)) + check(HashArrayEqualsWrapperLaws.check2(HashEqualsArrayWrapper.hashEqualsCharOrdering)) + check(HashArrayEqualsWrapperLaws.check2(HashEqualsArrayWrapper.hashEqualsByteOrdering)) + check(HashArrayEqualsWrapperLaws.check2(HashEqualsArrayWrapper.hashEqualsBooleanOrdering)) + check(HashArrayEqualsWrapperLaws.check2(HashEqualsArrayWrapper.hashEqualsFloatOrdering)) + check(HashArrayEqualsWrapperLaws.check2(HashEqualsArrayWrapper.hashEqualsDoubleOrdering)) + } + +} \ No newline at end of file diff --git a/scalding-core/src/test/scala/com/twitter/scalding/typed/TypedPipeDiffTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/typed/TypedPipeDiffTest.scala new file mode 100644 index 0000000000..d75856f1db --- /dev/null +++ b/scalding-core/src/test/scala/com/twitter/scalding/typed/TypedPipeDiffTest.scala @@ -0,0 +1,240 @@ +package com.twitter.scalding.typed + +import java.io.File +import java.nio.file.Files + +import com.twitter.algebird.MapAlgebra +import com.twitter.scalding.{ Config, Local } +import org.scalacheck.{ Arbitrary, Prop } +import org.scalatest.prop.{ Checkers, PropertyChecks } +import org.scalatest.{ FunSuite, PropSpec } + +import scala.reflect.ClassTag + +class NoOrdering(val x: String) { + + override def equals(other: Any): Boolean = other match { + case that: NoOrdering => x.equals(that.x) + case _ => false + } + + override def hashCode(): Int = x.hashCode +} + +class NoOrderingHashCollisions(val x: String) { + + override def equals(other: Any): Boolean = other match { + case that: NoOrderingHashCollisions => x.equals(that.x) + case _ => false + } + + override def hashCode(): Int = 0 +} + +object TypedPipeRunner { + def runToList[T](output: TypedPipe[T]): List[T] = + output + .toIterableExecution + .waitFor(Config.default, Local(strictSources = true)) + .get + .toList +} + +class TypedPipeDiffTest extends FunSuite { + import com.twitter.scalding.typed.TypedPipeRunner._ + + val left = List("hi", "hi", "bye", "foo", "bar") + val right = List("hi", "bye", "foo", "baz") + val expectedSortedDiff = List(("bar", (1, 0)), ("baz", (0, 1)), ("hi", (2, 1))).sorted + + val leftArr = List( + Array[Byte](3, 3, 5, 3, 2), + Array[Byte](2, 2, 2), + Array[Byte](0, 1, 0)) + + val rightArr = List( + Array[Byte](2, 2, 2), + Array[Byte](2, 2, 2), + Array[Byte](3, 3, 5, 3, 2), + Array[Byte](0, 1, 1)) + + val expectedSortedArrDiff = List( + (Array[Byte](0, 1, 0).toSeq, (1, 0)), + (Array[Byte](0, 1, 1).toSeq, (0, 1)), + (Array[Byte](2, 2, 2).toSeq, (1, 2))) + + test("diff works for objects with ordering and good hashcodes") { + val pipe1 = TypedPipe.from(left) + val pipe2 = TypedPipe.from(right) + val diff = TypedPipeDiff.diff(pipe1, pipe2) + + assert(expectedSortedDiff === runToList(diff.toTypedPipe).sorted) + } + + // this lets us sort the results, + // without bringing an ordering into scope + private def sort(x: List[(Seq[Byte], (Long, Long))]): List[(Seq[Byte], (Long, Long))] = { + import scala.Ordering.Implicits.seqDerivedOrdering + x.sorted + } + + test("diffArrayPipes works for arrays") { + val pipe1 = TypedPipe.from(leftArr) + val pipe2 = TypedPipe.from(rightArr) + + val diff = TypedPipeDiff.diffArrayPipes(pipe1, pipe2).map { case (arr, counts) => (arr.toSeq, counts) } + + assert(expectedSortedArrDiff === sort(runToList(diff))) + } + + test("diffWithoutOrdering works for objects with ordering and good hashcodes") { + val pipe1 = TypedPipe.from(left) + val pipe2 = TypedPipe.from(right) + val diff = TypedPipeDiff.diffByHashCode(pipe1, pipe2) + + assert(expectedSortedDiff === runToList(diff).sorted) + } + + test("diffWithoutOrdering does not require ordering") { + val pipe1 = TypedPipe.from(left.map(new NoOrdering(_))) + val pipe2 = TypedPipe.from(right.map(new NoOrdering(_))) + val diff = TypedPipeDiff.diffByHashCode(pipe1, pipe2) + + assert(expectedSortedDiff === runToList(diff).map { case (nord, counts) => (nord.x, counts) }.sorted) + } + + test("diffWithoutOrdering works even with hash collisions") { + val pipe1 = TypedPipe.from(left.map(new NoOrderingHashCollisions(_))) + val pipe2 = TypedPipe.from(right.map(new NoOrderingHashCollisions(_))) + val diff = TypedPipeDiff.diffByHashCode(pipe1, pipe2) + assert(expectedSortedDiff === runToList(diff).map { case (nord, counts) => (nord.x, counts) }.sorted) + } + + test("diffArrayPipesWithoutOrdering works for arrays of objects with no ordering") { + val pipe1 = TypedPipe.from(leftArr.map { arr => arr.map { b => new NoOrdering(b.toString) } }) + val pipe2 = TypedPipe.from(rightArr.map { arr => arr.map { b => new NoOrdering(b.toString) } }) + val diff = TypedPipeDiff.diffArrayPipes(pipe1, pipe2) + + assert(expectedSortedArrDiff === sort(runToList(diff).map{ case (arr, counts) => (arr.map(_.x.toByte).toSeq, counts) })) + } + + test("writeDiffSummaryToFile") { + val pipe1 = TypedPipe.from(left) + val pipe2 = TypedPipe.from(right) + val diff = TypedPipeDiff.diff(pipe1, pipe2) + + val root = Files.createTempDirectory("writeDiffSummaryToFileOut").toFile + val f = new File(root, "summary") + + val e = TypedPipeDiff.writeDiffSummaryToFile(diff.toTypedPipe, f, 1000) + + e.waitFor(Config.default, Local(strictSources = true)).get + + val src = scala.io.Source.fromFile(f) + val fileContents = src.getLines().toSet + src.close() + + assert(Set("For key bar there were 1 records in the left pipe and 0 records in the right pipe", + "For key baz there were 0 records in the left pipe and 1 records in the right pipe", + "For key hi there were 2 records in the left pipe and 1 records in the right pipe") === fileContents) + } + +} + +object TypedPipeDiffLaws { + import com.twitter.scalding.typed.TypedPipeDiff.Enrichments._ + import com.twitter.scalding.typed.TypedPipeRunner._ + + def checkDiff[T](left: List[T], right: List[T], diff: List[(T, (Long, Long))]): Boolean = { + val noDuplicates = diff.size == diff.map(_._1).toSet.size + val expected = MapAlgebra.sumByKey(left.map((_, (1L, 0L))).iterator ++ right.map((_, (0L, 1L))).iterator) + .filter { case (t, (rCount, lCount)) => rCount != lCount } + + noDuplicates && expected == diff.toMap + } + + def checkArrayDiff[T](left: List[Array[T]], right: List[Array[T]], diff: List[(Seq[T], (Long, Long))]): Boolean = { + checkDiff(left.map(_.toSeq), right.map(_.toSeq), diff) + } + + def diffLaw[T: Ordering: Arbitrary]: Prop = Prop.forAll { (left: List[T], right: List[T]) => + val diff = runToList(TypedPipe.from(left).diff(TypedPipe.from(right)).toTypedPipe) + checkDiff(left, right, diff) + } + + def diffArrayLaw[T](implicit arb: Arbitrary[List[Array[T]]], ct: ClassTag[T]): Prop = Prop.forAll { (left: List[Array[T]], right: List[Array[T]]) => + val diff = runToList(TypedPipe.from(left).diffArrayPipes(TypedPipe.from(right))) + .map { case (arr, counts) => (arr.toSeq, counts) } + checkArrayDiff(left, right, diff) + } + + def diffByGroupLaw[T: Arbitrary]: Prop = Prop.forAll { (left: List[T], right: List[T]) => + val diff = runToList(TypedPipe.from(left).diffByHashCode(TypedPipe.from(right))) + checkDiff(left, right, diff) + } + +} + +class TypedPipeDiffLaws extends PropSpec with PropertyChecks with Checkers { + + property("diffLaws") { + check(TypedPipeDiffLaws.diffLaw[Int]) + check(TypedPipeDiffLaws.diffLaw[String]) + } + + property("diffArrayLaws") { + + implicit val arbNoOrdering = Arbitrary { + for { + strs <- Arbitrary.arbitrary[Array[String]] + } yield { + strs.map { new NoOrdering(_) } + } + } + + implicit val arbNoOrderingHashCollision = Arbitrary { + for { + strs <- Arbitrary.arbitrary[Array[String]] + } yield { + strs.map { new NoOrderingHashCollisions(_) } + } + } + + check(TypedPipeDiffLaws.diffArrayLaw[Long]) + check(TypedPipeDiffLaws.diffArrayLaw[Int]) + check(TypedPipeDiffLaws.diffArrayLaw[Short]) + check(TypedPipeDiffLaws.diffArrayLaw[Char]) + check(TypedPipeDiffLaws.diffArrayLaw[Byte]) + check(TypedPipeDiffLaws.diffArrayLaw[Boolean]) + check(TypedPipeDiffLaws.diffArrayLaw[Float]) + check(TypedPipeDiffLaws.diffArrayLaw[Double]) + check(TypedPipeDiffLaws.diffArrayLaw[String]) + check(TypedPipeDiffLaws.diffArrayLaw[NoOrdering]) + check(TypedPipeDiffLaws.diffArrayLaw[NoOrderingHashCollisions]) + } + + property("diffByGroupLaws") { + + implicit val arbNoOrdering = Arbitrary { + for { + name <- Arbitrary.arbitrary[String] + } yield { + new NoOrdering(name) + } + } + + implicit val arbNoOrderingHashCollision = Arbitrary { + for { + name <- Arbitrary.arbitrary[String] + } yield { + new NoOrderingHashCollisions(name) + } + } + + check(TypedPipeDiffLaws.diffByGroupLaw[Int]) + check(TypedPipeDiffLaws.diffByGroupLaw[String]) + check(TypedPipeDiffLaws.diffByGroupLaw[NoOrdering]) + check(TypedPipeDiffLaws.diffByGroupLaw[NoOrderingHashCollisions]) + } + +} \ No newline at end of file From a17b51e6f986ab78cb187cf2efc5f6948e2d87f1 Mon Sep 17 00:00:00 2001 From: Ruban Monu Date: Thu, 30 Apr 2015 12:51:32 -0700 Subject: [PATCH 128/177] add LzoGenericScheme, LzoGenericSource --- project/Build.scala | 2 +- .../commons/source/BinaryConverters.scala | 46 +++++++ .../commons/source/LzoCodecSource.scala | 2 +- .../commons/source/LzoGenericScheme.scala | 118 ++++++++++++++++++ .../commons/source/LzoGenericSource.scala | 42 +++++++ 5 files changed, 208 insertions(+), 2 deletions(-) create mode 100644 scalding-commons/src/main/scala/com/twitter/scalding/commons/source/BinaryConverters.scala create mode 100644 scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericScheme.scala create mode 100644 scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericSource.scala diff --git a/project/Build.scala b/project/Build.scala index ffa4197dd9..79aebf833a 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -26,7 +26,7 @@ object ScaldingBuild extends Build { val cascadingAvroVersion = "2.1.2" val chillVersion = "0.5.2" val dfsDatastoresVersion = "1.3.4" - val elephantbirdVersion = "4.6" + val elephantbirdVersion = "4.7-SNAPSHOT" val hadoopLzoVersion = "0.4.16" val hadoopVersion = "1.2.1" val hbaseVersion = "0.94.10" diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/BinaryConverters.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/BinaryConverters.scala new file mode 100644 index 0000000000..1cd317cd83 --- /dev/null +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/BinaryConverters.scala @@ -0,0 +1,46 @@ +/* +Copyright 2015 Twitter, Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + +http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package com.twitter.scalding.commons.source + +import com.twitter.elephantbird.mapreduce.io.BinaryConverter +import com.twitter.scrooge.{ BinaryThriftStructSerializer, ThriftStructCodec, ThriftStruct } +import scala.reflect.ClassTag + +/* + * Common BinaryConverters to be used with GenericSource / GenericScheme. + */ + +case object IdentityBinaryConverter extends BinaryConverter[Array[Byte]] { + override def fromBytes(messageBuffer: Array[Byte]) = messageBuffer + override def toBytes(message: Array[Byte]) = message +} + +object ScroogeBinaryConverter { + def apply[T <: ThriftStruct: ClassTag]: BinaryConverter[T] = { + val ct = implicitly[ClassTag[T]] + new BinaryConverter[T] { + val serializer = BinaryThriftStructSerializer[T] { + val companionClass = Class.forName(ct.runtimeClass.getName + "$") + val companionObject = companionClass.getField("MODULE$").get(null) + companionObject.asInstanceOf[ThriftStructCodec[T]] + } + override def toBytes(struct: T) = serializer.toBytes(struct) + override def fromBytes(bytes: Array[Byte]): T = serializer.fromBytes(bytes) + } + } +} + diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoCodecSource.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoCodecSource.scala index 74d2ff6924..78b579e077 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoCodecSource.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoCodecSource.scala @@ -30,6 +30,6 @@ object LzoCodecSource { val hdfsPaths = paths val localPath = { assert(paths.size == 1, "Cannot use multiple input files on local mode"); paths(0) } val boxed = Externalizer(passedInjection) - override def injection = boxed.get + override lazy val injection = boxed.get } } diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericScheme.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericScheme.scala new file mode 100644 index 0000000000..c2d434d1bb --- /dev/null +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericScheme.scala @@ -0,0 +1,118 @@ +/* +Copyright 2015 Twitter, Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + +http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package com.twitter.scalding.commons.source + +import scala.reflect.ClassTag + +import com.twitter.bijection._ +import com.twitter.chill.Externalizer +import com.twitter.elephantbird.cascading2.scheme.LzoBinaryScheme +import com.twitter.elephantbird.mapreduce.input.combine.DelegateCombineFileInputFormat +import com.twitter.elephantbird.mapreduce.io.{ BinaryConverter, GenericWritable } +import com.twitter.elephantbird.mapreduce.input.{ BinaryConverterProvider, MultiInputFormat } +import com.twitter.elephantbird.mapreduce.output.LzoGenericBlockOutputFormat +import com.twitter.elephantbird.mapred.output.DeprecatedOutputFormatWrapper + +import org.apache.hadoop.mapred.{ JobConf, OutputCollector, RecordReader } +import org.apache.hadoop.conf.Configuration + +import cascading.tap.Tap +import cascading.flow.FlowProcess + +/** + * Serializes BinaryConverters to JobConf. + */ +private[source] object ExternalizerSerializer { + val inj: Injection[Externalizer[_], String] = { + import com.twitter.bijection.Inversion.attemptWhen + import com.twitter.bijection.codec.Base64 + + implicit val baseInj = JavaSerializationInjection[Externalizer[_]] + + implicit val unwrap: Injection[GZippedBase64String, String] = + new AbstractInjection[GZippedBase64String, String] { + override def apply(gzbs: GZippedBase64String) = gzbs.str + override def invert(str: String) = attemptWhen(str)(Base64.isBase64)(GZippedBase64String(_)) + } + + Injection.connect[Externalizer[_], Array[Byte], GZippedBase64String, String] + } +} + +private[source] object ConfigBinaryConverterProvider { + val ProviderConfKey = "com.twitter.scalding.lzo.converter.provider" +} + +/** + * Provides BinaryConverter serialized in JobConf. + */ +private[source] class ConfigBinaryConverterProvider[M] extends BinaryConverterProvider[M] { + import ConfigBinaryConverterProvider._ + override def getConverter(conf: Configuration): BinaryConverter[M] = { + val data = conf.get(ProviderConfKey) + require(data != null, s"No data in field $ProviderConfKey") + + val extern: Externalizer[_] = ExternalizerSerializer.inj.invert(data).get + extern.get.asInstanceOf[BinaryConverter[M]] + } +} + +/** + * Generic scheme for data stored as lzo-compressed protobuf messages. + * Serialization is performed using the supplied BinaryConverter. + */ +class LzoGenericScheme[M: ClassTag](@transient conv: BinaryConverter[M]) extends LzoBinaryScheme[M, GenericWritable[M]] { + + override protected def prepareBinaryWritable(): GenericWritable[M] = + new GenericWritable(conv) + + override def sourceConfInit(fp: FlowProcess[JobConf], + tap: Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]], + conf: JobConf): Unit = { + + val extern = Externalizer(conv) + try { + ExternalizerSerializer.inj.invert(ExternalizerSerializer.inj(extern)).get + } catch { + case e: Exception => throw new RuntimeException("Unable to roundtrip the BinaryConverter in the Externalizer.", e) + } + + conf.set(ConfigBinaryConverterProvider.ProviderConfKey, ExternalizerSerializer.inj(extern)) + + MultiInputFormat.setClassConf(conv.getClass, conf) + MultiInputFormat.setGenericConverterClassConf(classOf[ConfigBinaryConverterProvider[_]], conf) + + DelegateCombineFileInputFormat.setDelegateInputFormat(conf, classOf[MultiInputFormat[_]]) + } + + override def sinkConfInit(fp: FlowProcess[JobConf], + tap: Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]], + conf: JobConf): Unit = { + val extern = Externalizer(conv) + try { + ExternalizerSerializer.inj.invert(ExternalizerSerializer.inj(extern)).get + } catch { + case e: Exception => throw new RuntimeException("Unable to roundtrip the BinaryConverter in the Externalizer.", e) + } + + LzoGenericBlockOutputFormat.setClassConf(implicitly[ClassTag[M]].runtimeClass, conf) + conf.set(ConfigBinaryConverterProvider.ProviderConfKey, ExternalizerSerializer.inj(extern)) + LzoGenericBlockOutputFormat.setGenericConverterClassConf(classOf[ConfigBinaryConverterProvider[_]], conf) + DeprecatedOutputFormatWrapper.setOutputFormat(classOf[LzoGenericBlockOutputFormat[_]], conf) + } +} + diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericSource.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericSource.scala new file mode 100644 index 0000000000..39701e5f3a --- /dev/null +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericSource.scala @@ -0,0 +1,42 @@ +/* +Copyright 2015 Twitter, Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + +http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package com.twitter.scalding.commons.source + +import scala.reflect.ClassTag + +import com.twitter.elephantbird.mapreduce.io.BinaryConverter +import com.twitter.scalding._ + +import cascading.scheme.Scheme + +/** + * Generic source with an underlying GenericScheme that uses the supplied BinaryConverter. + */ +abstract class LzoGenericSource[T: ClassTag] extends FileSource with SingleMappable[T] with TypedSink[T] with LocalTapSource { + def conv: BinaryConverter[T] + override def setter[U <: T] = TupleSetter.asSubSetter[T, U](TupleSetter.singleSetter[T]) + override def hdfsScheme = HadoopSchemeInstance((new LzoGenericScheme(conv)).asInstanceOf[Scheme[_, _, _, _, _]]) +} + +object LzoGenericSource { + def apply[T: ClassTag](passedConv: BinaryConverter[T], paths: String*) = + new LzoGenericSource[T] { + override val conv: BinaryConverter[T] = passedConv + val hdfsPaths = paths + val localPath = { assert(paths.size == 1, "Cannot use multiple input files on local mode"); paths(0) } + } +} From aa094f79103bb0b422ed982ca83bbd516ebed072 Mon Sep 17 00:00:00 2001 From: Alex Levenson Date: Thu, 30 Apr 2015 13:04:31 -0700 Subject: [PATCH 129/177] Remove not so useful methods in TypedPipeDiff --- .../scalding/typed/TypedPipeDiff.scala | 33 ------------------- 1 file changed, 33 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipeDiff.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipeDiff.scala index bb81b91277..c49582b3e1 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipeDiff.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipeDiff.scala @@ -102,39 +102,6 @@ object TypedPipeDiff { right: TypedPipe[T], reducers: Option[Int] = None): TypedPipe[(T, (Long, Long))] = diffByGroup(left, right, reducers)(_.hashCode) - def diffSummary[T](diff: TypedPipe[(T, (Long, Long))]): TypedPipe[String] = { - diff.map { - case (key, (lCount, rCount)) => - s"For key $key there were $lCount records in the left pipe and $rCount records in the right pipe" - } - } - - /** - * Returns an Execution that writes the difference into a local file. - * Truncates to maxDiff lines of output. - */ - def writeDiffSummaryToFile[T](diff: TypedPipe[(T, (Long, Long))], f: File, maxDiff: Int): Execution[Unit] = - - diffSummary(diff.groupAll.bufferedTake(maxDiff).values) - .toIterableExecution - .map { iter => - var writer: BufferedWriter = null - - try { - writer = new BufferedWriter(new FileWriter(f)) - iter.foreach { line => - writer.write(line) - writer.write("\n") - } - } finally { - if (writer != null) { - writer.close() - } - } - - () - } - object Enrichments { implicit class Diff[T](val left: TypedPipe[T]) extends AnyVal { From d663d5110d1b5341008c48639c791ced595ed67d Mon Sep 17 00:00:00 2001 From: Mansur Ashraf Date: Fri, 1 May 2015 13:29:20 -0700 Subject: [PATCH 130/177] upgrade sbt to 0.13.8 --- project/build.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/build.properties b/project/build.properties index be6c454fba..a6e117b610 100644 --- a/project/build.properties +++ b/project/build.properties @@ -1 +1 @@ -sbt.version=0.13.5 +sbt.version=0.13.8 From ba747f37d03fa207816208ae861a6585ebee1901 Mon Sep 17 00:00:00 2001 From: Mansur Ashraf Date: Fri, 1 May 2015 14:05:23 -0700 Subject: [PATCH 131/177] upgrade to latest sbt launcher script --- sbt | 268 +++++++++++++++++++++++++++++++++++++++--------------------- 1 file changed, 177 insertions(+), 91 deletions(-) diff --git a/sbt b/sbt index 25cd36d65c..422327fc31 100755 --- a/sbt +++ b/sbt @@ -1,17 +1,17 @@ #!/usr/bin/env bash # # A more capable sbt runner, coincidentally also called sbt. -# Author: Paul Phillips +# Author: Paul Phillips # todo - make this dynamic -declare -r sbt_release_version="0.13.5" -declare -r sbt_unreleased_version="0.13.6-MSERVER-1" +declare -r sbt_release_version="0.13.8" +declare -r sbt_unreleased_version="0.13.8" declare -r buildProps="project/build.properties" declare sbt_jar sbt_dir sbt_create sbt_version -declare scala_version java_home sbt_explicit_version +declare scala_version sbt_explicit_version declare verbose noshare batch trace_level log_level -declare sbt_saved_stty +declare sbt_saved_stty debugUs echoerr () { echo >&2 "$@"; } vlog () { [[ -n "$verbose" ]] && echoerr "$@"; } @@ -19,7 +19,7 @@ vlog () { [[ -n "$verbose" ]] && echoerr "$@"; } # spaces are possible, e.g. sbt.version = 0.13.0 build_props_sbt () { [[ -r "$buildProps" ]] && \ - grep '^sbt\.version' "$buildProps" | tr '=' ' ' | awk '{ print $2; }' + grep '^sbt\.version' "$buildProps" | tr '=\r' ' ' | awk '{ print $2; }' } update_build_props_sbt () { @@ -101,12 +101,12 @@ init_default_option_file () { declare -r cms_opts="-XX:+CMSClassUnloadingEnabled -XX:+UseConcMarkSweepGC" declare -r jit_opts="-XX:ReservedCodeCacheSize=256m -XX:+TieredCompilation" -declare -r default_jvm_opts="-XX:MaxPermSize=384m -Xms512m -Xmx1536m -Xss2m $jit_opts $cms_opts" +declare -r default_jvm_opts_common="-Xms512m -Xmx1536m -Xss2m $jit_opts $cms_opts" declare -r noshare_opts="-Dsbt.global.base=project/.sbtboot -Dsbt.boot.directory=project/.boot -Dsbt.ivy.home=project/.ivy" declare -r latest_28="2.8.2" declare -r latest_29="2.9.3" -declare -r latest_210="2.10.4" -declare -r latest_211="2.11.1" +declare -r latest_210="2.10.5" +declare -r latest_211="2.11.6" declare -r script_path="$(get_script_path "$BASH_SOURCE")" declare -r script_name="${script_path##*/}" @@ -115,7 +115,7 @@ declare -r script_name="${script_path##*/}" declare java_cmd="java" declare sbt_opts_file="$(init_default_option_file SBT_OPTS .sbtopts)" declare jvm_opts_file="$(init_default_option_file JVM_OPTS .jvmopts)" -declare sbt_launch_repo="https://private-repo.typesafe.com/typesafe/ivy-releases" +declare sbt_launch_repo="http://typesafe.artifactoryonline.com/typesafe/ivy-releases" # pull -J and -D options to give to java. declare -a residual_args @@ -126,14 +126,79 @@ declare -a sbt_commands # args to jvm/sbt via files or environment variables declare -a extra_jvm_opts extra_sbt_opts -# if set, use JAVA_HOME over java found in path -[[ -e "$JAVA_HOME/bin/java" ]] && java_cmd="$JAVA_HOME/bin/java" +addJava () { + vlog "[addJava] arg = '$1'" + java_args+=("$1") +} +addSbt () { + vlog "[addSbt] arg = '$1'" + sbt_commands+=("$1") +} +setThisBuild () { + vlog "[addBuild] args = '$@'" + local key="$1" && shift + addSbt "set $key in ThisBuild := $@" +} +addScalac () { + vlog "[addScalac] arg = '$1'" + scalac_args+=("$1") +} +addResidual () { + vlog "[residual] arg = '$1'" + residual_args+=("$1") +} +addResolver () { + addSbt "set resolvers += $1" +} +addDebugger () { + addJava "-Xdebug" + addJava "-Xrunjdwp:transport=dt_socket,server=y,suspend=n,address=$1" +} +setScalaVersion () { + [[ "$1" == *"-SNAPSHOT" ]] && addResolver 'Resolver.sonatypeRepo("snapshots")' + addSbt "++ $1" +} +setJavaHome () { + java_cmd="$1/bin/java" + setThisBuild javaHome "Some(file(\"$1\"))" + export JAVA_HOME="$1" + export JDK_HOME="$1" + export PATH="$JAVA_HOME/bin:$PATH" +} +setJavaHomeQuietly () { + addSbt warn + setJavaHome "$1" + addSbt info +} + +# if set, use JDK_HOME/JAVA_HOME over java found in path +if [[ -e "$JDK_HOME/lib/tools.jar" ]]; then + setJavaHomeQuietly "$JDK_HOME" +elif [[ -e "$JAVA_HOME/bin/java" ]]; then + setJavaHomeQuietly "$JAVA_HOME" +fi # directory to store sbt launchers declare sbt_launch_dir="$HOME/.sbt/launchers" [[ -d "$sbt_launch_dir" ]] || mkdir -p "$sbt_launch_dir" [[ -w "$sbt_launch_dir" ]] || sbt_launch_dir="$(mktemp -d -t sbt_extras_launchers.XXXXXX)" +java_version () { + local version=$("$java_cmd" -version 2>&1 | grep -E -e '(java|openjdk) version' | awk '{ print $3 }' | tr -d \") + vlog "Detected Java version: $version" + echo "${version:2:1}" +} + +# MaxPermSize critical on pre-8 jvms but incurs noisy warning on 8+ +default_jvm_opts () { + local v="$(java_version)" + if [[ $v -ge 8 ]]; then + echo "$default_jvm_opts_common" + else + echo "-XX:MaxPermSize=384m $default_jvm_opts_common" + fi +} + build_props_scala () { if [[ -r "$buildProps" ]]; then versionLine="$(grep '^build.scala.versions' "$buildProps")" @@ -157,9 +222,7 @@ execRunner () { vlog "" } - if [[ -n "$batch" ]]; then - exec display stack traces with a max of frames (default: -1, traces suppressed) + -debug-inc enable debugging log for the incremental compiler -no-colors disable ANSI color codes -sbt-create start sbt even if current directory contains no sbt project -sbt-dir path to global settings/plugins directory (default: ~/.sbt/) @@ -220,7 +291,9 @@ are not special. -prompt Set the sbt prompt; in expr, 's' is the State and 'e' is Extracted # sbt version (default: sbt.version from $buildProps if present, otherwise $sbt_release_version) + -sbt-force-latest force the use of the latest release of sbt: $sbt_release_version -sbt-version use the specified version of sbt (default: $sbt_release_version) + -sbt-dev use the latest pre-release version of sbt: $sbt_unreleased_version -sbt-jar use the specified jar as the sbt launcher -sbt-launch-dir directory to hold sbt launchers (default: ~/.sbt/launchers) -sbt-launch-repo repo url for downloading sbt launcher jar (default: $sbt_launch_repo) @@ -239,7 +312,7 @@ are not special. # passing options to the jvm - note it does NOT use JAVA_OPTS due to pollution # The default set is used if JVM_OPTS is unset and no -jvm-opts file is found - $default_jvm_opts + $(default_jvm_opts) JVM_OPTS environment variable holding either the jvm args directly, or the reference to a file containing jvm args if given path is prepended by '@' (e.g. '@/etc/jvmopts') Note: "@"-file is overridden by local '.jvmopts' or '-jvm-opts' argument. @@ -256,34 +329,6 @@ are not special. EOM } -addJava () { - vlog "[addJava] arg = '$1'" - java_args=( "${java_args[@]}" "$1" ) -} -addSbt () { - vlog "[addSbt] arg = '$1'" - sbt_commands=( "${sbt_commands[@]}" "$1" ) -} -addScalac () { - vlog "[addScalac] arg = '$1'" - scalac_args=( "${scalac_args[@]}" "$1" ) -} -addResidual () { - vlog "[residual] arg = '$1'" - residual_args=( "${residual_args[@]}" "$1" ) -} -addResolver () { - addSbt "set resolvers += $1" -} -addDebugger () { - addJava "-Xdebug" - addJava "-Xrunjdwp:transport=dt_socket,server=y,suspend=n,address=$1" -} -setScalaVersion () { - [[ "$1" == *"-SNAPSHOT" ]] && addResolver 'Resolver.sonatypeRepo("snapshots")' - addSbt "++ $1" -} - process_args () { require_arg () { @@ -297,45 +342,50 @@ process_args () } while [[ $# -gt 0 ]]; do case "$1" in - -h|-help) usage; exit 1 ;; - -v) verbose=true && shift ;; - -d) addSbt "--debug" && shift ;; - -w) addSbt "--warn" && shift ;; - -q) addSbt "--error" && shift ;; - -trace) require_arg integer "$1" "$2" && trace_level="$2" && shift 2 ;; - -ivy) require_arg path "$1" "$2" && addJava "-Dsbt.ivy.home=$2" && shift 2 ;; - -no-colors) addJava "-Dsbt.log.noformat=true" && shift ;; - -no-share) noshare=true && shift ;; - -sbt-boot) require_arg path "$1" "$2" && addJava "-Dsbt.boot.directory=$2" && shift 2 ;; - -sbt-dir) require_arg path "$1" "$2" && sbt_dir="$2" && shift 2 ;; - -debug-inc) addJava "-Dxsbt.inc.debug=true" && shift ;; - -offline) addSbt "set offline := true" && shift ;; - -jvm-debug) require_arg port "$1" "$2" && addDebugger "$2" && shift 2 ;; - -batch) batch=true && shift ;; - -prompt) require_arg "expr" "$1" "$2" && addSbt "set shellPrompt in ThisBuild := (s => { val e = Project.extract(s) ; $2 })" && shift 2 ;; - - -sbt-create) sbt_create=true && shift ;; - -sbt-jar) require_arg path "$1" "$2" && sbt_jar="$2" && shift 2 ;; - -sbt-version) require_arg version "$1" "$2" && sbt_explicit_version="$2" && shift 2 ;; - -sbt-dev) sbt_explicit_version="$sbt_unreleased_version" && shift ;; --sbt-launch-dir) require_arg path "$1" "$2" && sbt_launch_dir="$2" && shift 2 ;; --sbt-launch-repo) require_arg path "$1" "$2" && sbt_launch_repo="$2" && shift 2 ;; - -scala-version) require_arg version "$1" "$2" && setScalaVersion "$2" && shift 2 ;; --binary-version) require_arg version "$1" "$2" && addSbt "set scalaBinaryVersion in ThisBuild := \"$2\"" && shift 2 ;; - -scala-home) require_arg path "$1" "$2" && addSbt "set every scalaHome := Some(file(\"$2\"))" && shift 2 ;; - -java-home) require_arg path "$1" "$2" && java_cmd="$2/bin/java" && shift 2 ;; - -sbt-opts) require_arg path "$1" "$2" && sbt_opts_file="$2" && shift 2 ;; - -jvm-opts) require_arg path "$1" "$2" && jvm_opts_file="$2" && shift 2 ;; - - -D*) addJava "$1" && shift ;; - -J*) addJava "${1:2}" && shift ;; - -S*) addScalac "${1:2}" && shift ;; - -28) setScalaVersion "$latest_28" && shift ;; - -29) setScalaVersion "$latest_29" && shift ;; - -210) setScalaVersion "$latest_210" && shift ;; - -211) setScalaVersion "$latest_211" && shift ;; - - *) addResidual "$1" && shift ;; + -h|-help) usage; exit 1 ;; + -v) verbose=true && shift ;; + -d) addSbt "--debug" && addSbt debug && shift ;; + -w) addSbt "--warn" && addSbt warn && shift ;; + -q) addSbt "--error" && addSbt error && shift ;; + -x) debugUs=true && shift ;; + -trace) require_arg integer "$1" "$2" && trace_level="$2" && shift 2 ;; + -ivy) require_arg path "$1" "$2" && addJava "-Dsbt.ivy.home=$2" && shift 2 ;; + -no-colors) addJava "-Dsbt.log.noformat=true" && shift ;; + -no-share) noshare=true && shift ;; + -sbt-boot) require_arg path "$1" "$2" && addJava "-Dsbt.boot.directory=$2" && shift 2 ;; + -sbt-dir) require_arg path "$1" "$2" && sbt_dir="$2" && shift 2 ;; + -debug-inc) addJava "-Dxsbt.inc.debug=true" && shift ;; + -offline) addSbt "set offline := true" && shift ;; + -jvm-debug) require_arg port "$1" "$2" && addDebugger "$2" && shift 2 ;; + -batch) batch=true && shift ;; + -prompt) require_arg "expr" "$1" "$2" && setThisBuild shellPrompt "(s => { val e = Project.extract(s) ; $2 })" && shift 2 ;; + + -sbt-create) sbt_create=true && shift ;; + -sbt-jar) require_arg path "$1" "$2" && sbt_jar="$2" && shift 2 ;; + -sbt-version) require_arg version "$1" "$2" && sbt_explicit_version="$2" && shift 2 ;; + -sbt-force-latest) sbt_explicit_version="$sbt_release_version" && shift ;; + -sbt-dev) sbt_explicit_version="$sbt_unreleased_version" && shift ;; + -sbt-launch-dir) require_arg path "$1" "$2" && sbt_launch_dir="$2" && shift 2 ;; + -sbt-launch-repo) require_arg path "$1" "$2" && sbt_launch_repo="$2" && shift 2 ;; + -scala-version) require_arg version "$1" "$2" && setScalaVersion "$2" && shift 2 ;; + -binary-version) require_arg version "$1" "$2" && setThisBuild scalaBinaryVersion "\"$2\"" && shift 2 ;; + -scala-home) require_arg path "$1" "$2" && setThisBuild scalaHome "Some(file(\"$2\"))" && shift 2 ;; + -java-home) require_arg path "$1" "$2" && setJavaHome "$2" && shift 2 ;; + -sbt-opts) require_arg path "$1" "$2" && sbt_opts_file="$2" && shift 2 ;; + -jvm-opts) require_arg path "$1" "$2" && jvm_opts_file="$2" && shift 2 ;; + + -D*) addJava "$1" && shift ;; + -J*) addJava "${1:2}" && shift ;; + -S*) addScalac "${1:2}" && shift ;; + -28) setScalaVersion "$latest_28" && shift ;; + -29) setScalaVersion "$latest_29" && shift ;; + -210) setScalaVersion "$latest_210" && shift ;; + -211) setScalaVersion "$latest_211" && shift ;; + + --debug) addSbt debug && addResidual "$1" && shift ;; + --warn) addSbt warn && addResidual "$1" && shift ;; + --error) addSbt error && addResidual "$1" && shift ;; + *) addResidual "$1" && shift ;; esac done } @@ -375,7 +425,7 @@ set_sbt_version setTraceLevel() { case "$sbt_version" in "0.7."* | "0.10."* | "0.11."* ) echoerr "Cannot set trace level in sbt version $sbt_version" ;; - *) addSbt "set every traceLevel := $trace_level" ;; + *) setThisBuild traceLevel $trace_level ;; esac } @@ -442,16 +492,52 @@ elif [[ -n "$JVM_OPTS" && ! ("$JVM_OPTS" =~ ^@.*) ]]; then extra_jvm_opts=( $JVM_OPTS ) else vlog "Using default jvm options" - extra_jvm_opts=( $default_jvm_opts ) + extra_jvm_opts=( $(default_jvm_opts) ) fi # traceLevel is 0.12+ [[ -n "$trace_level" ]] && setTraceLevel +main () { + execRunner "$java_cmd" \ + "${extra_jvm_opts[@]}" \ + "${java_args[@]}" \ + -jar "$sbt_jar" \ + "${sbt_commands[@]}" \ + "${residual_args[@]}" +} + +# sbt inserts this string on certain lines when formatting is enabled: +# val OverwriteLine = "\r\u001BM\u001B[2K" +# ...in order not to spam the console with a million "Resolving" lines. +# Unfortunately that makes it that much harder to work with when +# we're not going to print those lines anyway. We strip that bit of +# line noise, but leave the other codes to preserve color. +mainFiltered () { + local ansiOverwrite='\r\x1BM\x1B[2K' + local excludeRegex=$(egrep -v '^#|^$' ~/.sbtignore | paste -sd'|' -) + + echoLine () { + local line="$1" + local line1="$(echo "$line" | sed -r 's/\r\x1BM\x1B\[2K//g')" # This strips the OverwriteLine code. + local line2="$(echo "$line1" | sed -r 's/\x1B\[[0-9;]*[JKmsu]//g')" # This strips all codes - we test regexes against this. + + if [[ $line2 =~ $excludeRegex ]]; then + [[ -n $debugUs ]] && echo "[X] $line1" + else + [[ -n $debugUs ]] && echo " $line1" || echo "$line1" + fi + } + + echoLine "Starting sbt with output filtering enabled." + main | while read -r line; do echoLine "$line"; done +} + +# Only filter if there's a filter file and we don't see a known interactive command. +# Obviously this is super ad hoc but I don't know how to improve on it. Testing whether +# stdin is a terminal is useless because most of my use cases for this filtering are +# exactly when I'm at a terminal, running sbt non-interactively. +shouldFilter () { [[ -f ~/.sbtignore ]] && ! egrep -q '\b(shell|console|consoleProject)\b' <<<"${residual_args[@]}"; } + # run sbt -execRunner "$java_cmd" \ - "${extra_jvm_opts[@]}" \ - "${java_args[@]}" \ - -jar "$sbt_jar" \ - "${sbt_commands[@]}" \ - "${residual_args[@]}" \ No newline at end of file +if shouldFilter; then mainFiltered; else main; fi \ No newline at end of file From d5601c0cc791fb39c9cefedf7ec6bca5ffca5190 Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Fri, 1 May 2015 12:23:04 -0700 Subject: [PATCH 132/177] Change separator from \1 to \u0001 --- .../src/main/scala/com/twitter/scalding/FileSource.scala | 2 +- .../src/main/scala/com/twitter/scalding/TypedDelimited.scala | 2 +- .../com/twitter/scalding/typed/PartitionedDelimitedSource.scala | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala b/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala index e783525be8..ddd2acbfc5 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala @@ -355,7 +355,7 @@ case class Osv(p: String, f: Fields = Fields.ALL, override val sinkMode: SinkMode = SinkMode.REPLACE) extends FixedPathSource(p) with DelimitedScheme { override val fields = f - override val separator = "\1" + override val separator = "\u0001" } object TextLine { diff --git a/scalding-core/src/main/scala/com/twitter/scalding/TypedDelimited.scala b/scalding-core/src/main/scala/com/twitter/scalding/TypedDelimited.scala index e9ea62b217..b24573e19c 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/TypedDelimited.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/TypedDelimited.scala @@ -69,7 +69,7 @@ object TypedPsv extends TypedSeperatedFile { * Typed one separated values file (commonly used by Pig) */ object TypedOsv extends TypedSeperatedFile { - val separator = "\1" + val separator = "\u0001" } object FixedPathTypedDelimited { diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/PartitionedDelimitedSource.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/PartitionedDelimitedSource.scala index 554e8a0bb5..0008779ff1 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/PartitionedDelimitedSource.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/PartitionedDelimitedSource.scala @@ -118,5 +118,5 @@ object PartitionedPsv extends PartitionedDelimited { /** Partitioned typed `\1` separated source (commonly used by Pig).*/ object PartitionedOsv extends PartitionedDelimited { - val separator = "\1" + val separator = "\u0001" } From 2bde85c5c0513d6d12f83e176034f803a7cd3226 Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Sat, 2 May 2015 10:09:43 -0700 Subject: [PATCH 133/177] Get rid of more warnings caused by type erasure --- .../scala/com/twitter/scalding/Config.scala | 4 +- .../com/twitter/scalding/CumulativeSum.scala | 8 +- .../scalding/serialization/KryoHadoop.scala | 4 +- .../com/twitter/scalding/LookupJoinTest.scala | 266 +++++++++--------- 4 files changed, 138 insertions(+), 144 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala index 02ac5063b4..cfd616d897 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala @@ -267,8 +267,8 @@ trait Config { */ def addReducerEstimator(clsName: String): Config = update(Config.ReducerEstimators) { - case None => Some(clsName) -> () - case Some(lst) => Some(clsName + "," + lst) -> () + case None => Some(clsName) -> Unit + case Some(lst) => Some(clsName + "," + lst) -> Unit }._2 /** Set the entire list of reducer estimators (overriding the existing list) */ diff --git a/scalding-core/src/main/scala/com/twitter/scalding/CumulativeSum.scala b/scalding-core/src/main/scala/com/twitter/scalding/CumulativeSum.scala index f0555b650e..13b1acaff3 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/CumulativeSum.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/CumulativeSum.scala @@ -39,9 +39,9 @@ object CumulativeSum { def cumulativeSum( implicit sg: Semigroup[V], ordU: Ordering[U], ordK: Ordering[K]): SortedGrouped[K, (U, V)] = { pipe.group - .sortBy { case (u: U, _) => u } + .sortBy { case (u, _) => u } .scanLeft(Nil: List[(U, V)]) { - case (acc, (u: U, v: V)) => + case (acc, (u, v)) => acc match { case List((previousU, previousSum)) => List((u, sg.plus(previousSum, v))) case _ => List((u, v)) @@ -63,7 +63,7 @@ object CumulativeSum { ordK: Ordering[K]): TypedPipe[(K, (U, V))] = { val sumPerS = pipe - .map { case (k, (u: U, v: V)) => (k, partition(u)) -> v } + .map { case (k, (u, v)) => (k, partition(u)) -> v } .sumByKey .map { case ((k, s), v) => (k, (s, v)) } .group @@ -87,7 +87,7 @@ object CumulativeSum { val summands = pipe .map { - case (k, (u: U, v: V)) => + case (k, (u, v)) => (k, partition(u)) -> (Some(u), v) } ++ sumPerS diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala index 5dbe4241ab..05a04ab725 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala @@ -40,7 +40,7 @@ import com.twitter.scalding.Args import com.twitter.chill.algebird._ import com.twitter.chill.config.Config -import com.twitter.chill.{SingletonSerializer, ScalaKryoInstantiator, KryoInstantiator} +import com.twitter.chill.{ SingletonSerializer, ScalaKryoInstantiator, KryoInstantiator } class KryoHadoop(config: Config) extends KryoInstantiator { /** @@ -61,7 +61,7 @@ class KryoHadoop(config: Config) extends KryoInstantiator { newK.register(classOf[DateRange], new DateRangeSerializer()) newK.register(classOf[Args], new ArgsSerializer) // Some of the monoids from Algebird that we use: - newK.register(classOf[com.twitter.algebird.AveragedValue], new AveragedValueSerializer) + newK.register(classOf[com.twitter.algebird.AveragedValue], new AveragedValueSerializer) newK.register(classOf[com.twitter.algebird.DecayedValue], new DecayedValueSerializer) newK.register(classOf[com.twitter.algebird.HyperLogLogMonoid], new HLLMonoidSerializer) newK.register(classOf[com.twitter.algebird.Moments], new MomentsSerializer) diff --git a/scalding-core/src/test/scala/com/twitter/scalding/LookupJoinTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/LookupJoinTest.scala index abfe81dca4..754c7f3633 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/LookupJoinTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/LookupJoinTest.scala @@ -28,7 +28,7 @@ object LookupJoinedTest { (0 until sz).view.map { _ => (rng.nextInt(maxTime), rng.nextInt(maxKey), rng.nextInt) } - .groupBy { case (t, k, v) => (t, k)} + .groupBy { case (t, k, v) => (t, k) } .mapValues(_.headOption.toList) .values .flatten @@ -36,188 +36,182 @@ object LookupJoinedTest { } } +class LookupJoinerJob(args: Args) extends Job(args) { - class LookupJoinerJob(args: Args) extends Job(args) { + import TDsl._ - import TDsl._ + val in0 = TypedTsv[(Int, Int, Int)]("input0") + val in1 = TypedTsv[(Int, Int, Int)]("input1") - val in0 = TypedTsv[(Int, Int, Int)]("input0") - val in1 = TypedTsv[(Int, Int, Int)]("input1") - - LookupJoin(TypedPipe.from(in0).map { case (t, k, v) => (t, (k, v))}, - TypedPipe.from(in1).map { case (t, k, v) => (t, (k, v))}) - .map { + LookupJoin(TypedPipe.from(in0).map { case (t, k, v) => (t, (k, v)) }, + TypedPipe.from(in1).map { case (t, k, v) => (t, (k, v)) }) + .map { case (t, (k, (v, opt))) => (t.toString, k.toString, v.toString, opt.toString) } - .write(TypedTsv[(String, String, String, String)]("output")) + .write(TypedTsv[(String, String, String, String)]("output")) - LookupJoin.rightSumming(TypedPipe.from(in0).map { case (t, k, v) => (t, (k, v))}, - TypedPipe.from(in1).map { case (t, k, v) => (t, (k, v))}) - .map { + LookupJoin.rightSumming(TypedPipe.from(in0).map { case (t, k, v) => (t, (k, v)) }, + TypedPipe.from(in1).map { case (t, k, v) => (t, (k, v)) }) + .map { case (t, (k, (v, opt))) => (t.toString, k.toString, v.toString, opt.toString) } - .write(TypedTsv[(String, String, String, String)]("output2")) - } + .write(TypedTsv[(String, String, String, String)]("output2")) +} - class LookupJoinedTest extends WordSpec with Matchers { +class LookupJoinedTest extends WordSpec with Matchers { - import Dsl._ - import LookupJoinedTest.genList + import Dsl._ + import LookupJoinedTest.genList - def lookupJoin[T: Ordering, K, V, W](in0: Iterable[(T, K, V)], in1: Iterable[(T, K, W)]) = { - val serv = in1.groupBy(_._2) - def lookup(t: T, k: K): Option[W] = { - val ord = Ordering.by { tkw: (T, K, W) => tkw._1} - serv.get(k).flatMap { in1s => - in1s.filter { case (t1, _, _) => Ordering[T].lt(t1, t)} - .reduceOption(ord.max(_, _)) - .map { + def lookupJoin[T: Ordering, K, V, W](in0: Iterable[(T, K, V)], in1: Iterable[(T, K, W)]) = { + val serv = in1.groupBy(_._2) + def lookup(t: T, k: K): Option[W] = { + val ord = Ordering.by { tkw: (T, K, W) => tkw._1 } + serv.get(k).flatMap { in1s => + in1s.filter { case (t1, _, _) => Ordering[T].lt(t1, t) } + .reduceOption(ord.max(_, _)) + .map { _._3 } - } } - in0.map { case (t, k, v) => (t.toString, k.toString, v.toString, lookup(t, k).toString)} } + in0.map { case (t, k, v) => (t.toString, k.toString, v.toString, lookup(t, k).toString) } + } - def lookupSumJoin[T: Ordering, K, V, W: Semigroup](in0: Iterable[(T, K, V)], in1: Iterable[(T, K, W)]) = { - implicit val ord: Ordering[(T, K, W)] = Ordering.by { - _._1 - } - val serv = in1.groupBy(_._2).mapValues { - _.toList - .sorted - .scanLeft(None: Option[(T, K, W)]) { (old, newer) => - old.map { case (_, _, w) => (newer._1, newer._2, Semigroup.plus(w, newer._3))} + def lookupSumJoin[T: Ordering, K, V, W: Semigroup](in0: Iterable[(T, K, V)], in1: Iterable[(T, K, W)]) = { + implicit val ord: Ordering[(T, K, W)] = Ordering.by { + _._1 + } + val serv = in1.groupBy(_._2).mapValues { + _.toList + .sorted + .scanLeft(None: Option[(T, K, W)]) { (old, newer) => + old.map { case (_, _, w) => (newer._1, newer._2, Semigroup.plus(w, newer._3)) } .orElse(Some(newer)) } - .filter { + .filter { _.isDefined } - .map { + .map { _.get } - }.toMap // Force the map - - def lookup(t: T, k: K): Option[W] = { - val ord = Ordering.by { tkw: (T, K, W) => tkw._1} - serv.get(k).flatMap { in1s => - in1s.filter { case (t1, _, _) => Ordering[T].lt(t1, t)} - .reduceOption(ord.max(_, _)) - .map { + }.toMap // Force the map + + def lookup(t: T, k: K): Option[W] = { + val ord = Ordering.by { tkw: (T, K, W) => tkw._1 } + serv.get(k).flatMap { in1s => + in1s.filter { case (t1, _, _) => Ordering[T].lt(t1, t) } + .reduceOption(ord.max(_, _)) + .map { _._3 } - } } - in0.map { case (t, k, v) => (t.toString, k.toString, v.toString, lookup(t, k).toString)} } + in0.map { case (t, k, v) => (t.toString, k.toString, v.toString, lookup(t, k).toString) } + } - "A LookupJoinerJob" should { - "correctly lookup" in { - val MAX_KEY = 100 - val VAL_COUNT = 10000 - val in0 = genList(Int.MaxValue, MAX_KEY, VAL_COUNT) - val in1 = genList(Int.MaxValue, MAX_KEY, VAL_COUNT) - JobTest(new LookupJoinerJob(_)) - .source(TypedTsv[(Int, Int, Int)]("input0"), in0) - .source(TypedTsv[(Int, Int, Int)]("input1"), in1) - .sink[(String, String, String, String)]( - TypedTsv[(String, String, String, String)]("output")) { outBuf => - outBuf.toSet should equal (lookupJoin(in0, in1).toSet) - in0.size should equal (outBuf.size) - } - .sink[(String, String, String, String)]( - TypedTsv[(String, String, String, String)]("output2")) { outBuf => - outBuf.toSet should equal(lookupSumJoin(in0, in1).toSet) - in0.size should equal(outBuf.size) - } - .run - //.runHadoop - .finish - } + "A LookupJoinerJob" should { + "correctly lookup" in { + val MAX_KEY = 100 + val VAL_COUNT = 10000 + val in0 = genList(Int.MaxValue, MAX_KEY, VAL_COUNT) + val in1 = genList(Int.MaxValue, MAX_KEY, VAL_COUNT) + JobTest(new LookupJoinerJob(_)) + .source(TypedTsv[(Int, Int, Int)]("input0"), in0) + .source(TypedTsv[(Int, Int, Int)]("input1"), in1) + .sink[(String, String, String, String)]( + TypedTsv[(String, String, String, String)]("output")) { outBuf => + outBuf.toSet should equal (lookupJoin(in0, in1).toSet) + in0.size should equal (outBuf.size) + } + .sink[(String, String, String, String)]( + TypedTsv[(String, String, String, String)]("output2")) { outBuf => + outBuf.toSet should equal(lookupSumJoin(in0, in1).toSet) + in0.size should equal(outBuf.size) + } + .run + //.runHadoop + .finish } } +} - class WindowLookupJoinerJob(args: Args) extends Job(args) { +class WindowLookupJoinerJob(args: Args) extends Job(args) { - import TDsl._ + import TDsl._ - val in0 = TypedTsv[(Int, Int, Int)]("input0") - val in1 = TypedTsv[(Int, Int, Int)]("input1") - val window = args("window").toInt + val in0 = TypedTsv[(Int, Int, Int)]("input0") + val in1 = TypedTsv[(Int, Int, Int)]("input1") + val window = args("window").toInt - def gate(left: Int, right: Int) = - (left.toLong - right.toLong) < window + def gate(left: Int, right: Int) = + (left.toLong - right.toLong) < window - LookupJoin.withWindow(TypedPipe.from(in0).map { case (t, k, v) => (t, (k, v))}, - TypedPipe.from(in1).map { case (t, k, v) => (t, (k, v))})(gate _) - .map { + LookupJoin.withWindow(TypedPipe.from(in0).map { case (t, k, v) => (t, (k, v)) }, + TypedPipe.from(in1).map { case (t, k, v) => (t, (k, v)) })(gate _) + .map { case (t, (k, (v, opt))) => (t.toString, k.toString, v.toString, opt.toString) } - .write(TypedTsv[(String, String, String, String)]("output")) - } + .write(TypedTsv[(String, String, String, String)]("output")) +} - class WindowLookupJoinedTest extends WordSpec with Matchers { +class WindowLookupJoinedTest extends WordSpec with Matchers { - import Dsl._ - import LookupJoinedTest.genList + import Dsl._ + import LookupJoinedTest.genList - def windowLookupJoin[K, V, W](in0: Iterable[(Int, K, V)], in1: Iterable[(Int, K, W)], win: Int) = { - val serv = in1.groupBy(_._2) - // super inefficient, but easy to verify: - def lookup(t: Int, k: K): Option[W] = { - val ord = Ordering.by { tkw: (Int, K, W) => tkw._1} - serv.get(k).flatMap { in1s => - in1s.filter { - case (t1, _, _) => - (t1 < t) && ((t.toLong - t1.toLong) < win) - } - .reduceOption(ord.max(_, _)) - .map { + def windowLookupJoin[K, V, W](in0: Iterable[(Int, K, V)], in1: Iterable[(Int, K, W)], win: Int) = { + val serv = in1.groupBy(_._2) + // super inefficient, but easy to verify: + def lookup(t: Int, k: K): Option[W] = { + val ord = Ordering.by { tkw: (Int, K, W) => tkw._1 } + serv.get(k).flatMap { in1s => + in1s.filter { + case (t1, _, _) => + (t1 < t) && ((t.toLong - t1.toLong) < win) + } + .reduceOption(ord.max(_, _)) + .map { _._3 } - } } - in0.map { case (t, k, v) => (t.toString, k.toString, v.toString, lookup(t, k).toString)} } + in0.map { case (t, k, v) => (t.toString, k.toString, v.toString, lookup(t, k).toString) } + } - "A WindowLookupJoinerJob" should { - //Set up the job: - "correctly lookup" in { - val MAX_KEY = 10 - val MAX_TIME = 10000 - val sz:Int = 10000; - val in0 = genList(MAX_TIME, MAX_KEY, 10000) - val in1 = genList(MAX_TIME, MAX_KEY, 10000) - JobTest(new WindowLookupJoinerJob(_)) - .arg("window", "100") - .source(TypedTsv[(Int, Int, Int)]("input0"), in0) - .source(TypedTsv[(Int, Int, Int)]("input1"), in1) - .sink[(String, String, String, String)]( - TypedTsv[(String, String, String, String)]("output")) { outBuf => - val results = outBuf.toList.sorted - val correct = windowLookupJoin(in0, in1, 100).toList.sorted - def some(it: List[(String, String, String, String)]) = - it.filter(_._4.startsWith("Some")) - - def none(it: List[(String, String, String, String)]) = - it.filter(_._4.startsWith("None")) - - some(results) shouldBe (some(correct)) - none(results) shouldBe (none(correct)) - in0.size should equal (outBuf.size) - } - .run - //.runHadoop - .finish - } + "A WindowLookupJoinerJob" should { + //Set up the job: + "correctly lookup" in { + val MAX_KEY = 10 + val MAX_TIME = 10000 + val sz: Int = 10000; + val in0 = genList(MAX_TIME, MAX_KEY, 10000) + val in1 = genList(MAX_TIME, MAX_KEY, 10000) + JobTest(new WindowLookupJoinerJob(_)) + .arg("window", "100") + .source(TypedTsv[(Int, Int, Int)]("input0"), in0) + .source(TypedTsv[(Int, Int, Int)]("input1"), in1) + .sink[(String, String, String, String)]( + TypedTsv[(String, String, String, String)]("output")) { outBuf => + val results = outBuf.toList.sorted + val correct = windowLookupJoin(in0, in1, 100).toList.sorted + def some(it: List[(String, String, String, String)]) = + it.filter(_._4.startsWith("Some")) + + def none(it: List[(String, String, String, String)]) = + it.filter(_._4.startsWith("None")) + + some(results) shouldBe (some(correct)) + none(results) shouldBe (none(correct)) + in0.size should equal (outBuf.size) + } + .run + //.runHadoop + .finish } } - - - - - +} From 1a425ed807ef205d1dd37ffa54e9773faf41b73c Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Sun, 3 May 2015 15:26:57 -0700 Subject: [PATCH 134/177] Change localPath to localPaths to support multiple local paths and make use of MultiSourceTap for symmetry --- .../com/twitter/scalding/FileSource.scala | 34 ++++++++++++++++--- 1 file changed, 29 insertions(+), 5 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala b/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala index e783525be8..5bba5b894e 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala @@ -64,7 +64,16 @@ abstract class SchemedSource extends Source { */ trait LocalSourceOverride extends SchemedSource { /** A path to use for the local tap. */ - def localPath: String + def localPaths: Iterable[String] + + // By default, we write to the last path for local paths + def localWritePath = localPaths.last + + object CastFileTap { + // The scala compiler has problems with the generics in Cascading + def apply(tap: FileTap): Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]] = + tap.asInstanceOf[Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]]] + } /** * Creates a local tap. @@ -72,7 +81,13 @@ trait LocalSourceOverride extends SchemedSource { * @param sinkMode The mode for handling output conflicts. * @returns A tap. */ - def createLocalTap(sinkMode: SinkMode): Tap[_, _, _] = new FileTap(localScheme, localPath, sinkMode) + def createLocalTap(sinkMode: SinkMode): Tap[JobConf, _, _] = { + val taps = localPaths.map { + p: String => + CastFileTap(new FileTap(localScheme, p, sinkMode)) + }.toList + new ScaldingMultiSourceTap(taps) + } } object HiddenFileFilter extends PathFilter { @@ -145,7 +160,10 @@ abstract class FileSource extends SchemedSource with LocalSourceOverride { mode match { // TODO support strict in Local case Local(_) => { - createLocalTap(sinkMode) + readOrWrite match { + case Read => createLocalTap(sinkMode) + case Write => new FileTap(localScheme, localWritePath, sinkMode) + } } case hdfsMode @ Hdfs(_, _) => readOrWrite match { case Read => createHdfsReadTap(hdfsMode) @@ -306,11 +324,17 @@ trait SuccessFileSource extends FileSource { * Put another way, this runs a Hadoop tap outside of Hadoop in the Cascading local mode */ trait LocalTapSource extends LocalSourceOverride { - override def createLocalTap(sinkMode: SinkMode) = new LocalTap(localPath, hdfsScheme, sinkMode).asInstanceOf[Tap[_, _, _]] + override def createLocalTap(sinkMode: SinkMode) = { + val taps = localPaths.map { + p => + new LocalTap(p, hdfsScheme, sinkMode).asInstanceOf[Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]]] + }.toSeq + new ScaldingMultiSourceTap(taps) + } } abstract class FixedPathSource(path: String*) extends FileSource { - def localPath = { assert(path.size == 1, "Cannot use multiple input files on local mode"); path(0) } + def localPaths = path.toList def hdfsPaths = path.toList override def toString = getClass.getName + path override def hashCode = toString.hashCode From 28c3b6b77d8b14dd15a70b5eb9686d29a6ea81cb Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Sun, 3 May 2015 15:27:49 -0700 Subject: [PATCH 135/177] Change TimePathedSource to suit localPaths change --- .../main/scala/com/twitter/scalding/TimePathedSource.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/TimePathedSource.scala b/scalding-core/src/main/scala/com/twitter/scalding/TimePathedSource.scala index e2db997c0c..b694120601 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/TimePathedSource.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/TimePathedSource.scala @@ -115,7 +115,11 @@ abstract class TimePathedSource(val pattern: String, val stripped = pattern.slice(0, lastSlashPos) TimePathedSource.toPath(stripped, dateRange.end, tz) } - override def localPath = pattern + + override def localPaths = patterns + .flatMap{ pattern: String => + Globifier(pattern)(tz).globify(dateRange) + } } /* From 09ee051dccb7fe9922bb3779a762a639109e22e0 Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Sun, 3 May 2015 15:28:19 -0700 Subject: [PATCH 136/177] Remove paths.size = 1 restriction for localPath --- .../com/twitter/scalding/commons/source/LzoCodecSource.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoCodecSource.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoCodecSource.scala index c97a612644..676558885c 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoCodecSource.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoCodecSource.scala @@ -29,7 +29,7 @@ object LzoCodecSource { def apply[T](paths: String*)(implicit passedInjection: Injection[T, Array[Byte]]) = new LzoCodec[T] { val hdfsPaths = paths - val localPath = { assert(paths.size == 1, "Cannot use multiple input files on local mode"); paths(0) } + val localPaths = paths val boxed = Externalizer(passedInjection) override def injection = boxed.get } From 7ebac47a2bfdec04d57b854c36faaae7c6edb659 Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Sun, 3 May 2015 15:28:50 -0700 Subject: [PATCH 137/177] change localPath: String to localPaths: Iterable[String] --- .../src/test/scala/com/twitter/scalding/FileSourceTest.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/scalding-core/src/test/scala/com/twitter/scalding/FileSourceTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/FileSourceTest.scala index f6872aea7e..da84e777c9 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/FileSourceTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/FileSourceTest.scala @@ -170,7 +170,7 @@ object TestFileSource extends FileSource { import TestPath.testfsPathRoot override def hdfsPaths: Iterable[String] = Iterable.empty - override def localPath: String = "" + override def localPaths: Iterable[String] = Iterable.empty val conf = new Configuration() @@ -180,7 +180,7 @@ object TestFileSource extends FileSource { object TestSuccessFileSource extends FileSource with SuccessFileSource { import TestPath.testfsPathRoot override def hdfsPaths: Iterable[String] = Iterable.empty - override def localPath: String = "" + override def localPaths: Iterable[String] = Iterable.empty val conf = new Configuration() From 70865cca9c74dc067c685da4d3347912e3b00789 Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Sun, 3 May 2015 15:30:11 -0700 Subject: [PATCH 138/177] Remove hdfsPaths = 1 in CodecSource.scala --- .../main/scala/com/twitter/scalding/source/CodecSource.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/source/CodecSource.scala b/scalding-core/src/main/scala/com/twitter/scalding/source/CodecSource.scala index 72e963f427..e8ea3ff4c8 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/source/CodecSource.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/source/CodecSource.scala @@ -56,9 +56,8 @@ class CodecSource[T] private (val hdfsPaths: Seq[String], val maxFailures: Int = lazy val field = new Fields(fieldSym.name) val injectionBox = Externalizer(injection andThen BytesWritableCodec.get) - def localPath = { - require(hdfsPaths.size == 1, "Local mode only supports a single path"); - hdfsPaths(0) + def localPaths = { + hdfsPaths } override def converter[U >: T] = TupleConverter.asSuperConverter[T, U](TupleConverter.singleConverter[T]) override def hdfsScheme = From 94113241d11dd0eaa14a534b9f9d4ae2ef805513 Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Sun, 3 May 2015 16:03:15 -0700 Subject: [PATCH 139/177] Format --- .../scalding/serialization/KryoHadoop.scala | 4 +- .../com/twitter/scalding/LookupJoinTest.scala | 266 +++++++++--------- 2 files changed, 132 insertions(+), 138 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala index 5dbe4241ab..05a04ab725 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala @@ -40,7 +40,7 @@ import com.twitter.scalding.Args import com.twitter.chill.algebird._ import com.twitter.chill.config.Config -import com.twitter.chill.{SingletonSerializer, ScalaKryoInstantiator, KryoInstantiator} +import com.twitter.chill.{ SingletonSerializer, ScalaKryoInstantiator, KryoInstantiator } class KryoHadoop(config: Config) extends KryoInstantiator { /** @@ -61,7 +61,7 @@ class KryoHadoop(config: Config) extends KryoInstantiator { newK.register(classOf[DateRange], new DateRangeSerializer()) newK.register(classOf[Args], new ArgsSerializer) // Some of the monoids from Algebird that we use: - newK.register(classOf[com.twitter.algebird.AveragedValue], new AveragedValueSerializer) + newK.register(classOf[com.twitter.algebird.AveragedValue], new AveragedValueSerializer) newK.register(classOf[com.twitter.algebird.DecayedValue], new DecayedValueSerializer) newK.register(classOf[com.twitter.algebird.HyperLogLogMonoid], new HLLMonoidSerializer) newK.register(classOf[com.twitter.algebird.Moments], new MomentsSerializer) diff --git a/scalding-core/src/test/scala/com/twitter/scalding/LookupJoinTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/LookupJoinTest.scala index abfe81dca4..754c7f3633 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/LookupJoinTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/LookupJoinTest.scala @@ -28,7 +28,7 @@ object LookupJoinedTest { (0 until sz).view.map { _ => (rng.nextInt(maxTime), rng.nextInt(maxKey), rng.nextInt) } - .groupBy { case (t, k, v) => (t, k)} + .groupBy { case (t, k, v) => (t, k) } .mapValues(_.headOption.toList) .values .flatten @@ -36,188 +36,182 @@ object LookupJoinedTest { } } +class LookupJoinerJob(args: Args) extends Job(args) { - class LookupJoinerJob(args: Args) extends Job(args) { + import TDsl._ - import TDsl._ + val in0 = TypedTsv[(Int, Int, Int)]("input0") + val in1 = TypedTsv[(Int, Int, Int)]("input1") - val in0 = TypedTsv[(Int, Int, Int)]("input0") - val in1 = TypedTsv[(Int, Int, Int)]("input1") - - LookupJoin(TypedPipe.from(in0).map { case (t, k, v) => (t, (k, v))}, - TypedPipe.from(in1).map { case (t, k, v) => (t, (k, v))}) - .map { + LookupJoin(TypedPipe.from(in0).map { case (t, k, v) => (t, (k, v)) }, + TypedPipe.from(in1).map { case (t, k, v) => (t, (k, v)) }) + .map { case (t, (k, (v, opt))) => (t.toString, k.toString, v.toString, opt.toString) } - .write(TypedTsv[(String, String, String, String)]("output")) + .write(TypedTsv[(String, String, String, String)]("output")) - LookupJoin.rightSumming(TypedPipe.from(in0).map { case (t, k, v) => (t, (k, v))}, - TypedPipe.from(in1).map { case (t, k, v) => (t, (k, v))}) - .map { + LookupJoin.rightSumming(TypedPipe.from(in0).map { case (t, k, v) => (t, (k, v)) }, + TypedPipe.from(in1).map { case (t, k, v) => (t, (k, v)) }) + .map { case (t, (k, (v, opt))) => (t.toString, k.toString, v.toString, opt.toString) } - .write(TypedTsv[(String, String, String, String)]("output2")) - } + .write(TypedTsv[(String, String, String, String)]("output2")) +} - class LookupJoinedTest extends WordSpec with Matchers { +class LookupJoinedTest extends WordSpec with Matchers { - import Dsl._ - import LookupJoinedTest.genList + import Dsl._ + import LookupJoinedTest.genList - def lookupJoin[T: Ordering, K, V, W](in0: Iterable[(T, K, V)], in1: Iterable[(T, K, W)]) = { - val serv = in1.groupBy(_._2) - def lookup(t: T, k: K): Option[W] = { - val ord = Ordering.by { tkw: (T, K, W) => tkw._1} - serv.get(k).flatMap { in1s => - in1s.filter { case (t1, _, _) => Ordering[T].lt(t1, t)} - .reduceOption(ord.max(_, _)) - .map { + def lookupJoin[T: Ordering, K, V, W](in0: Iterable[(T, K, V)], in1: Iterable[(T, K, W)]) = { + val serv = in1.groupBy(_._2) + def lookup(t: T, k: K): Option[W] = { + val ord = Ordering.by { tkw: (T, K, W) => tkw._1 } + serv.get(k).flatMap { in1s => + in1s.filter { case (t1, _, _) => Ordering[T].lt(t1, t) } + .reduceOption(ord.max(_, _)) + .map { _._3 } - } } - in0.map { case (t, k, v) => (t.toString, k.toString, v.toString, lookup(t, k).toString)} } + in0.map { case (t, k, v) => (t.toString, k.toString, v.toString, lookup(t, k).toString) } + } - def lookupSumJoin[T: Ordering, K, V, W: Semigroup](in0: Iterable[(T, K, V)], in1: Iterable[(T, K, W)]) = { - implicit val ord: Ordering[(T, K, W)] = Ordering.by { - _._1 - } - val serv = in1.groupBy(_._2).mapValues { - _.toList - .sorted - .scanLeft(None: Option[(T, K, W)]) { (old, newer) => - old.map { case (_, _, w) => (newer._1, newer._2, Semigroup.plus(w, newer._3))} + def lookupSumJoin[T: Ordering, K, V, W: Semigroup](in0: Iterable[(T, K, V)], in1: Iterable[(T, K, W)]) = { + implicit val ord: Ordering[(T, K, W)] = Ordering.by { + _._1 + } + val serv = in1.groupBy(_._2).mapValues { + _.toList + .sorted + .scanLeft(None: Option[(T, K, W)]) { (old, newer) => + old.map { case (_, _, w) => (newer._1, newer._2, Semigroup.plus(w, newer._3)) } .orElse(Some(newer)) } - .filter { + .filter { _.isDefined } - .map { + .map { _.get } - }.toMap // Force the map - - def lookup(t: T, k: K): Option[W] = { - val ord = Ordering.by { tkw: (T, K, W) => tkw._1} - serv.get(k).flatMap { in1s => - in1s.filter { case (t1, _, _) => Ordering[T].lt(t1, t)} - .reduceOption(ord.max(_, _)) - .map { + }.toMap // Force the map + + def lookup(t: T, k: K): Option[W] = { + val ord = Ordering.by { tkw: (T, K, W) => tkw._1 } + serv.get(k).flatMap { in1s => + in1s.filter { case (t1, _, _) => Ordering[T].lt(t1, t) } + .reduceOption(ord.max(_, _)) + .map { _._3 } - } } - in0.map { case (t, k, v) => (t.toString, k.toString, v.toString, lookup(t, k).toString)} } + in0.map { case (t, k, v) => (t.toString, k.toString, v.toString, lookup(t, k).toString) } + } - "A LookupJoinerJob" should { - "correctly lookup" in { - val MAX_KEY = 100 - val VAL_COUNT = 10000 - val in0 = genList(Int.MaxValue, MAX_KEY, VAL_COUNT) - val in1 = genList(Int.MaxValue, MAX_KEY, VAL_COUNT) - JobTest(new LookupJoinerJob(_)) - .source(TypedTsv[(Int, Int, Int)]("input0"), in0) - .source(TypedTsv[(Int, Int, Int)]("input1"), in1) - .sink[(String, String, String, String)]( - TypedTsv[(String, String, String, String)]("output")) { outBuf => - outBuf.toSet should equal (lookupJoin(in0, in1).toSet) - in0.size should equal (outBuf.size) - } - .sink[(String, String, String, String)]( - TypedTsv[(String, String, String, String)]("output2")) { outBuf => - outBuf.toSet should equal(lookupSumJoin(in0, in1).toSet) - in0.size should equal(outBuf.size) - } - .run - //.runHadoop - .finish - } + "A LookupJoinerJob" should { + "correctly lookup" in { + val MAX_KEY = 100 + val VAL_COUNT = 10000 + val in0 = genList(Int.MaxValue, MAX_KEY, VAL_COUNT) + val in1 = genList(Int.MaxValue, MAX_KEY, VAL_COUNT) + JobTest(new LookupJoinerJob(_)) + .source(TypedTsv[(Int, Int, Int)]("input0"), in0) + .source(TypedTsv[(Int, Int, Int)]("input1"), in1) + .sink[(String, String, String, String)]( + TypedTsv[(String, String, String, String)]("output")) { outBuf => + outBuf.toSet should equal (lookupJoin(in0, in1).toSet) + in0.size should equal (outBuf.size) + } + .sink[(String, String, String, String)]( + TypedTsv[(String, String, String, String)]("output2")) { outBuf => + outBuf.toSet should equal(lookupSumJoin(in0, in1).toSet) + in0.size should equal(outBuf.size) + } + .run + //.runHadoop + .finish } } +} - class WindowLookupJoinerJob(args: Args) extends Job(args) { +class WindowLookupJoinerJob(args: Args) extends Job(args) { - import TDsl._ + import TDsl._ - val in0 = TypedTsv[(Int, Int, Int)]("input0") - val in1 = TypedTsv[(Int, Int, Int)]("input1") - val window = args("window").toInt + val in0 = TypedTsv[(Int, Int, Int)]("input0") + val in1 = TypedTsv[(Int, Int, Int)]("input1") + val window = args("window").toInt - def gate(left: Int, right: Int) = - (left.toLong - right.toLong) < window + def gate(left: Int, right: Int) = + (left.toLong - right.toLong) < window - LookupJoin.withWindow(TypedPipe.from(in0).map { case (t, k, v) => (t, (k, v))}, - TypedPipe.from(in1).map { case (t, k, v) => (t, (k, v))})(gate _) - .map { + LookupJoin.withWindow(TypedPipe.from(in0).map { case (t, k, v) => (t, (k, v)) }, + TypedPipe.from(in1).map { case (t, k, v) => (t, (k, v)) })(gate _) + .map { case (t, (k, (v, opt))) => (t.toString, k.toString, v.toString, opt.toString) } - .write(TypedTsv[(String, String, String, String)]("output")) - } + .write(TypedTsv[(String, String, String, String)]("output")) +} - class WindowLookupJoinedTest extends WordSpec with Matchers { +class WindowLookupJoinedTest extends WordSpec with Matchers { - import Dsl._ - import LookupJoinedTest.genList + import Dsl._ + import LookupJoinedTest.genList - def windowLookupJoin[K, V, W](in0: Iterable[(Int, K, V)], in1: Iterable[(Int, K, W)], win: Int) = { - val serv = in1.groupBy(_._2) - // super inefficient, but easy to verify: - def lookup(t: Int, k: K): Option[W] = { - val ord = Ordering.by { tkw: (Int, K, W) => tkw._1} - serv.get(k).flatMap { in1s => - in1s.filter { - case (t1, _, _) => - (t1 < t) && ((t.toLong - t1.toLong) < win) - } - .reduceOption(ord.max(_, _)) - .map { + def windowLookupJoin[K, V, W](in0: Iterable[(Int, K, V)], in1: Iterable[(Int, K, W)], win: Int) = { + val serv = in1.groupBy(_._2) + // super inefficient, but easy to verify: + def lookup(t: Int, k: K): Option[W] = { + val ord = Ordering.by { tkw: (Int, K, W) => tkw._1 } + serv.get(k).flatMap { in1s => + in1s.filter { + case (t1, _, _) => + (t1 < t) && ((t.toLong - t1.toLong) < win) + } + .reduceOption(ord.max(_, _)) + .map { _._3 } - } } - in0.map { case (t, k, v) => (t.toString, k.toString, v.toString, lookup(t, k).toString)} } + in0.map { case (t, k, v) => (t.toString, k.toString, v.toString, lookup(t, k).toString) } + } - "A WindowLookupJoinerJob" should { - //Set up the job: - "correctly lookup" in { - val MAX_KEY = 10 - val MAX_TIME = 10000 - val sz:Int = 10000; - val in0 = genList(MAX_TIME, MAX_KEY, 10000) - val in1 = genList(MAX_TIME, MAX_KEY, 10000) - JobTest(new WindowLookupJoinerJob(_)) - .arg("window", "100") - .source(TypedTsv[(Int, Int, Int)]("input0"), in0) - .source(TypedTsv[(Int, Int, Int)]("input1"), in1) - .sink[(String, String, String, String)]( - TypedTsv[(String, String, String, String)]("output")) { outBuf => - val results = outBuf.toList.sorted - val correct = windowLookupJoin(in0, in1, 100).toList.sorted - def some(it: List[(String, String, String, String)]) = - it.filter(_._4.startsWith("Some")) - - def none(it: List[(String, String, String, String)]) = - it.filter(_._4.startsWith("None")) - - some(results) shouldBe (some(correct)) - none(results) shouldBe (none(correct)) - in0.size should equal (outBuf.size) - } - .run - //.runHadoop - .finish - } + "A WindowLookupJoinerJob" should { + //Set up the job: + "correctly lookup" in { + val MAX_KEY = 10 + val MAX_TIME = 10000 + val sz: Int = 10000; + val in0 = genList(MAX_TIME, MAX_KEY, 10000) + val in1 = genList(MAX_TIME, MAX_KEY, 10000) + JobTest(new WindowLookupJoinerJob(_)) + .arg("window", "100") + .source(TypedTsv[(Int, Int, Int)]("input0"), in0) + .source(TypedTsv[(Int, Int, Int)]("input1"), in1) + .sink[(String, String, String, String)]( + TypedTsv[(String, String, String, String)]("output")) { outBuf => + val results = outBuf.toList.sorted + val correct = windowLookupJoin(in0, in1, 100).toList.sorted + def some(it: List[(String, String, String, String)]) = + it.filter(_._4.startsWith("Some")) + + def none(it: List[(String, String, String, String)]) = + it.filter(_._4.startsWith("None")) + + some(results) shouldBe (some(correct)) + none(results) shouldBe (none(correct)) + in0.size should equal (outBuf.size) + } + .run + //.runHadoop + .finish } } - - - - - +} From 4e508aecc69837915eea258e9c96bb43cc482f3f Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Sun, 3 May 2015 16:11:49 -0700 Subject: [PATCH 140/177] format' --- .../src/main/scala/com/twitter/scalding/FileSource.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala b/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala index 5bba5b894e..c85f513295 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala @@ -336,6 +336,7 @@ trait LocalTapSource extends LocalSourceOverride { abstract class FixedPathSource(path: String*) extends FileSource { def localPaths = path.toList def hdfsPaths = path.toList + override def toString = getClass.getName + path override def hashCode = toString.hashCode override def equals(that: Any): Boolean = (that != null) && (that.toString == toString) From 5981d1099beeb9a39d04d31806716102ead9ff18 Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Sun, 3 May 2015 21:34:56 -0700 Subject: [PATCH 141/177] Fix test --- .../com/twitter/scalding/FileSource.scala | 19 ++++++++++++++----- .../twitter/scalding/TimePathedSource.scala | 4 ++-- 2 files changed, 16 insertions(+), 7 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala b/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala index e554ee4ae4..a5c4c45e30 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala @@ -214,11 +214,20 @@ abstract class FileSource extends SchemedSource with LocalSourceOverride { } } - case Local(_) => { - val file = new java.io.File(localPath) - if (!file.exists) + case Local(strict) => { + val files = localPaths.map{ + p => + new java.io.File(p) + } + + if (strict && !files.forall(_.exists)) { + throw new InvalidSourceException( + "[" + this.toString + "] Data is missing from one or more paths in: " + + localPaths.toString) + } else if (!files.exists(_.exists)) { throw new InvalidSourceException( - "[" + this.toString + "] Nothing at path: " + localPath) + "[" + this.toString + "] No good paths in: " + hdfsPaths.toString) + } } case _ => () } @@ -343,7 +352,7 @@ trait LocalTapSource extends LocalSourceOverride { abstract class FixedPathSource(path: String*) extends FileSource { def localPaths = path.toList def hdfsPaths = path.toList - + override def toString = getClass.getName + path override def hashCode = toString.hashCode override def equals(that: Any): Boolean = (that != null) && (that.toString == toString) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/TimePathedSource.scala b/scalding-core/src/main/scala/com/twitter/scalding/TimePathedSource.scala index 347af48580..a764854653 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/TimePathedSource.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/TimePathedSource.scala @@ -133,8 +133,8 @@ abstract class TimePathedSource(val pattern: String, override def localPaths = patterns .flatMap{ pattern: String => - Globifier(pattern)(tz).globify(dateRange) - } + Globifier(pattern)(tz).globify(dateRange) + } } /* From e0c64b1b03cf7c5998f6ec1694b5c68572abc13d Mon Sep 17 00:00:00 2001 From: Alex Levenson Date: Mon, 4 May 2015 13:23:33 -0700 Subject: [PATCH 142/177] Remove test for removed method --- .../scalding/typed/TypedPipeDiffTest.scala | 21 ------------------- 1 file changed, 21 deletions(-) diff --git a/scalding-core/src/test/scala/com/twitter/scalding/typed/TypedPipeDiffTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/typed/TypedPipeDiffTest.scala index d75856f1db..dc203d7fc1 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/typed/TypedPipeDiffTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/typed/TypedPipeDiffTest.scala @@ -118,27 +118,6 @@ class TypedPipeDiffTest extends FunSuite { assert(expectedSortedArrDiff === sort(runToList(diff).map{ case (arr, counts) => (arr.map(_.x.toByte).toSeq, counts) })) } - test("writeDiffSummaryToFile") { - val pipe1 = TypedPipe.from(left) - val pipe2 = TypedPipe.from(right) - val diff = TypedPipeDiff.diff(pipe1, pipe2) - - val root = Files.createTempDirectory("writeDiffSummaryToFileOut").toFile - val f = new File(root, "summary") - - val e = TypedPipeDiff.writeDiffSummaryToFile(diff.toTypedPipe, f, 1000) - - e.waitFor(Config.default, Local(strictSources = true)).get - - val src = scala.io.Source.fromFile(f) - val fileContents = src.getLines().toSet - src.close() - - assert(Set("For key bar there were 1 records in the left pipe and 0 records in the right pipe", - "For key baz there were 0 records in the left pipe and 1 records in the right pipe", - "For key hi there were 2 records in the left pipe and 1 records in the right pipe") === fileContents) - } - } object TypedPipeDiffLaws { From 5a6be974380b152fb0ebfe02dbf78e94175e18ed Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Mon, 4 May 2015 14:00:09 -0700 Subject: [PATCH 143/177] Use () instead of Unit --- .../src/main/scala/com/twitter/scalding/Config.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala index cfd616d897..8a5fc94b39 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala @@ -267,8 +267,8 @@ trait Config { */ def addReducerEstimator(clsName: String): Config = update(Config.ReducerEstimators) { - case None => Some(clsName) -> Unit - case Some(lst) => Some(clsName + "," + lst) -> Unit + case None => (Some(clsName), ()) + case Some(lst) => (Some(s"$clsName,$lst"), ()) }._2 /** Set the entire list of reducer estimators (overriding the existing list) */ From 302ffa055803cf19ed09c57e2c3a870418a4f3d8 Mon Sep 17 00:00:00 2001 From: Ruban Monu Date: Mon, 4 May 2015 14:26:53 -0700 Subject: [PATCH 144/177] initial fix with test, hopefully passes --- .../scalding/reducer_estimation/Common.scala | 3 +- .../ReducerEstimatorTest.scala | 31 +++++++++++++++++++ 2 files changed, 33 insertions(+), 1 deletion(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala index 6c17c701a4..5e1b86d9d1 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala @@ -97,7 +97,8 @@ object ReducerEstimatorStepStrategy extends FlowStepStrategy[JobConf] { conf.setInt(EstimatorConfig.estimatedNumReducers, numReducers.getOrElse(-1)) // set number of reducers - if (!setExplicitly || overrideExplicit) { + // stepNumReducers == 0 implies no reduce phase in this step + if (stepNumReducers != 0 && (!setExplicitly || overrideExplicit)) { numReducers.foreach(conf.setNumReduceTasks) } } diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala index d141587077..f76e92cf00 100644 --- a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala +++ b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala @@ -64,6 +64,14 @@ class GroupAllJob(args: Args) extends Job(args) { .write(size) } +class SimpleNoReducePhaseJob(args: Args) extends Job(args) { + import HipJob._ + // simple job with no reduce phase + TypedPipe.from(inSrc) + .flatMap(_.split("[^\\w]+")) + .write(TypedTsv[String]("mapped_output")) +} + class ReducerEstimatorTestSingle extends WordSpec with Matchers with HadoopPlatformTest { import HipJob._ @@ -151,3 +159,26 @@ class ReducerEstimatorTestMulti extends WordSpec with Matchers with HadoopPlatfo } } } + +class ReducerEstimatorTestNoReducePhase extends WordSpec with Matchers with HadoopPlatformTest { + import HipJob._ + + override def initialize() = cluster.initialize(Config.empty + .addReducerEstimator(classOf[InputSizeReducerEstimator]) + + (InputSizeReducerEstimator.BytesPerReducer -> (1L << 10).toString)) + + "No reduce phase job with reducer estimator" should { + "does not set num reducers" in { + HadoopPlatformJobTest(new SimpleNoReducePhaseJob(_), cluster) + .inspectCompletedFlow { flow => + val steps = flow.getFlowSteps.asScala + steps should have size 1 + + val conf = Config.fromHadoop(steps.head.getConfig) + conf.getNumReducers should contain (0) + } + .run + } + } +} + From 6d98cd93a588e89842d90b72e80720612eb9227f Mon Sep 17 00:00:00 2001 From: Ruban Monu Date: Mon, 4 May 2015 14:39:08 -0700 Subject: [PATCH 145/177] minor comment fiz --- .../scalding/reducer_estimation/ReducerEstimatorTest.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala index f76e92cf00..eed0b72e44 100644 --- a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala +++ b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala @@ -168,7 +168,7 @@ class ReducerEstimatorTestNoReducePhase extends WordSpec with Matchers with Hado (InputSizeReducerEstimator.BytesPerReducer -> (1L << 10).toString)) "No reduce phase job with reducer estimator" should { - "does not set num reducers" in { + "not set num reducers" in { HadoopPlatformJobTest(new SimpleNoReducePhaseJob(_), cluster) .inspectCompletedFlow { flow => val steps = flow.getFlowSteps.asScala From 7128229a7bd4793547da8770738426c26c4b8737 Mon Sep 17 00:00:00 2001 From: Ruban Monu Date: Mon, 4 May 2015 17:51:22 -0700 Subject: [PATCH 146/177] working fix, test cleanup --- .../scalding/reducer_estimation/Common.scala | 19 +++++++++++++++++-- .../ReducerEstimatorTest.scala | 8 ++++---- 2 files changed, 21 insertions(+), 6 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala index 5e1b86d9d1..116afe1313 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala @@ -4,6 +4,7 @@ import cascading.flow.{ FlowStep, Flow, FlowStepStrategy } import com.twitter.algebird.Monoid import com.twitter.scalding.{ StringUtility, Config } import org.apache.hadoop.mapred.JobConf +import org.slf4j.LoggerFactory import java.util.{ List => JList } import scala.collection.JavaConverters._ @@ -51,6 +52,8 @@ case class FallbackEstimator(first: ReducerEstimator, fallback: ReducerEstimator object ReducerEstimatorStepStrategy extends FlowStepStrategy[JobConf] { + private val LOG = LoggerFactory.getLogger(this.getClass) + implicit val estimatorMonoid: Monoid[ReducerEstimator] = new Monoid[ReducerEstimator] { override def zero: ReducerEstimator = new ReducerEstimator override def plus(l: ReducerEstimator, r: ReducerEstimator): ReducerEstimator = @@ -67,6 +70,19 @@ object ReducerEstimatorStepStrategy extends FlowStepStrategy[JobConf] { final override def apply(flow: Flow[JobConf], preds: JList[FlowStep[JobConf]], step: FlowStep[JobConf]): Unit = { + + val conf = step.getConfig + // for steps with reduce phase, mapred.reduce.tasks is set in the jobconf at this point + // so we check that to determine if this is a map-only step. + conf.getNumReduceTasks match { + case 0 => LOG.info(s"${flow.getName} is a map-only step. Skipping reducer estimation.") + case _ => estimate(flow, preds, step) + } + } + + private def estimate(flow: Flow[JobConf], + preds: JList[FlowStep[JobConf]], + step: FlowStep[JobConf]): Unit = { val conf = step.getConfig val stepNumReducers = conf.get(Config.HadoopNumReducers) @@ -97,8 +113,7 @@ object ReducerEstimatorStepStrategy extends FlowStepStrategy[JobConf] { conf.setInt(EstimatorConfig.estimatedNumReducers, numReducers.getOrElse(-1)) // set number of reducers - // stepNumReducers == 0 implies no reduce phase in this step - if (stepNumReducers != 0 && (!setExplicitly || overrideExplicit)) { + if (!setExplicitly || overrideExplicit) { numReducers.foreach(conf.setNumReduceTasks) } } diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala index eed0b72e44..3d8833433b 100644 --- a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala +++ b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala @@ -64,7 +64,7 @@ class GroupAllJob(args: Args) extends Job(args) { .write(size) } -class SimpleNoReducePhaseJob(args: Args) extends Job(args) { +class SimpleMapOnlyJob(args: Args) extends Job(args) { import HipJob._ // simple job with no reduce phase TypedPipe.from(inSrc) @@ -160,16 +160,16 @@ class ReducerEstimatorTestMulti extends WordSpec with Matchers with HadoopPlatfo } } -class ReducerEstimatorTestNoReducePhase extends WordSpec with Matchers with HadoopPlatformTest { +class ReducerEstimatorTestMapOnly extends WordSpec with Matchers with HadoopPlatformTest { import HipJob._ override def initialize() = cluster.initialize(Config.empty .addReducerEstimator(classOf[InputSizeReducerEstimator]) + (InputSizeReducerEstimator.BytesPerReducer -> (1L << 10).toString)) - "No reduce phase job with reducer estimator" should { + "Map-only job with reducer estimator" should { "not set num reducers" in { - HadoopPlatformJobTest(new SimpleNoReducePhaseJob(_), cluster) + HadoopPlatformJobTest(new SimpleMapOnlyJob(_), cluster) .inspectCompletedFlow { flow => val steps = flow.getFlowSteps.asScala steps should have size 1 From 69272c8bb43a52fcc4c6fb98fd819bbb18b34696 Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Tue, 5 May 2015 11:14:03 -0700 Subject: [PATCH 147/177] Address Oscar's comment --- .../src/main/scala/com/twitter/scalding/FileSource.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala b/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala index a5c4c45e30..64142ac743 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala @@ -86,7 +86,12 @@ trait LocalSourceOverride extends SchemedSource { p: String => CastFileTap(new FileTap(localScheme, p, sinkMode)) }.toList - new ScaldingMultiSourceTap(taps) + + taps match { + case Nil => throw new InvalidSourceException("LocalPaths is empty") + case oneTap :: Nil => oneTap + case many => new ScaldingMultiSourceTap(many) + } } } From 556a040b6fb5e5878e0a0c93b294cecff8dac023 Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Tue, 5 May 2015 11:20:31 -0700 Subject: [PATCH 148/177] Use multisource when there are multiple only --- .../scala/com/twitter/scalding/FileSource.scala | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala b/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala index 64142ac743..15f24f8c06 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala @@ -220,11 +220,7 @@ abstract class FileSource extends SchemedSource with LocalSourceOverride { } case Local(strict) => { - val files = localPaths.map{ - p => - new java.io.File(p) - } - + val files = localPaths.map{ p => new java.io.File(p) } if (strict && !files.forall(_.exists)) { throw new InvalidSourceException( "[" + this.toString + "] Data is missing from one or more paths in: " + @@ -346,11 +342,15 @@ trait SuccessFileSource extends FileSource { */ trait LocalTapSource extends LocalSourceOverride { override def createLocalTap(sinkMode: SinkMode) = { - val taps = localPaths.map { - p => + val taps = localPaths.map { p => new LocalTap(p, hdfsScheme, sinkMode).asInstanceOf[Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]]] }.toSeq - new ScaldingMultiSourceTap(taps) + + taps match { + case Nil => throw new InvalidSourceException("LocalPaths is empty") + case oneTap :: Nil => oneTap + case many => new ScaldingMultiSourceTap(many) + } } } From 6106d8120ecf72a90e0e53686b21a678eca7ffd9 Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Tue, 5 May 2015 11:21:53 -0700 Subject: [PATCH 149/177] Fix formatting --- .../main/scala/com/twitter/scalding/TimePathedSource.scala | 2 +- .../main/scala/com/twitter/scalding/source/CodecSource.scala | 5 ++--- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/TimePathedSource.scala b/scalding-core/src/main/scala/com/twitter/scalding/TimePathedSource.scala index a764854653..126a56c4b9 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/TimePathedSource.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/TimePathedSource.scala @@ -132,7 +132,7 @@ abstract class TimePathedSource(val pattern: String, override def hdfsWritePath = TimePathedSource.writePathFor(pattern, dateRange, tz) override def localPaths = patterns - .flatMap{ pattern: String => + .flatMap { pattern: String => Globifier(pattern)(tz).globify(dateRange) } } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/source/CodecSource.scala b/scalding-core/src/main/scala/com/twitter/scalding/source/CodecSource.scala index e8ea3ff4c8..394d7b7d10 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/source/CodecSource.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/source/CodecSource.scala @@ -56,9 +56,8 @@ class CodecSource[T] private (val hdfsPaths: Seq[String], val maxFailures: Int = lazy val field = new Fields(fieldSym.name) val injectionBox = Externalizer(injection andThen BytesWritableCodec.get) - def localPaths = { - hdfsPaths - } + def localPaths = hdfsPaths + override def converter[U >: T] = TupleConverter.asSuperConverter[T, U](TupleConverter.singleConverter[T]) override def hdfsScheme = HadoopSchemeInstance(new WritableSequenceFile(field, classOf[BytesWritable]).asInstanceOf[Scheme[_, _, _, _, _]]) From 5d905c53e9f1a22e2ef1e557ecf28a9684f49801 Mon Sep 17 00:00:00 2001 From: Joe Nievelt Date: Tue, 5 May 2015 13:32:48 -0700 Subject: [PATCH 150/177] Hydrate both sides of sampledCounts in skewJoinWithSmaller Augment sampledCounts in JoinAlgorithms#skewJoinWithSmaller to avoid dropping items when keys get filtered by sampling --- .../com/twitter/scalding/JoinAlgorithms.scala | 25 ++++++++++++++++++- 1 file changed, 24 insertions(+), 1 deletion(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/JoinAlgorithms.scala b/scalding-core/src/main/scala/com/twitter/scalding/JoinAlgorithms.scala index f234a93d23..bcc6e29a70 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/JoinAlgorithms.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/JoinAlgorithms.scala @@ -374,6 +374,7 @@ trait JoinAlgorithms { (otherPipe, fs._2, Fields.NONE) else // For now, we are assuming an inner join. renameCollidingFields(otherPipe, fs._2, intersection) + val mergedJoinKeys = Fields.join(fs._1, rightResolvedJoinFields) // 1. First, get an approximate count of the left join keys and the right join keys, so that we // know how much to replicate. @@ -392,7 +393,29 @@ trait JoinAlgorithms { val sampledRight = rightPipe.sample(sampleRate, Seed) .groupBy(rightResolvedJoinFields) { _.size(rightSampledCountField) } val sampledCounts = sampledLeft.joinWithSmaller(fs._1 -> rightResolvedJoinFields, sampledRight, joiner = new OuterJoin) - .project(Fields.join(fs._1, rightResolvedJoinFields, sampledCountFields)) + .project(Fields.join(mergedJoinKeys, sampledCountFields)) + .map(mergedJoinKeys -> mergedJoinKeys) { t: cascading.tuple.Tuple => + // Make the outer join look like an inner join so that we can join + // either the left or right fields for every entry. + // Accomplished by replacing any null field with the corresponding + // field from the other half. E.g., + // (1, 2, "foo", null, null, null) -> (1, 2, "foo", 1, 2, "foo") + val keysSize = t.size / 2 + val result = new cascading.tuple.Tuple(t) + + for (index <- 0 until keysSize) { + val leftValue = result.getObject(index) + val rightValue = result.getObject(index + keysSize) + + if (leftValue == null) { + result.set(index, rightValue) + } else if (rightValue == null) { + result.set(index + keysSize, leftValue) + } + } + + result + } // 2. Now replicate each group of join keys in the left and right pipes, according to the sampled counts // from the previous step. From a5a71c0665f808d236c89f6aa42f356f84cfdeb1 Mon Sep 17 00:00:00 2001 From: Joe Nievelt Date: Tue, 5 May 2015 13:35:44 -0700 Subject: [PATCH 151/177] Fix SkewJoinTest Accumulate the results instead of just computing the append. Also make input1 smaller to more easily expose sampling issues. --- .../src/test/scala/com/twitter/scalding/SkewJoinTest.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/scalding-core/src/test/scala/com/twitter/scalding/SkewJoinTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/SkewJoinTest.scala index b018867de4..af3140b684 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/SkewJoinTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/SkewJoinTest.scala @@ -64,9 +64,9 @@ object JoinTestHelper { .arg("replicationFactor", replicationFactor.toString) .arg("replicator", replicator.toString) .source(Tsv("input0"), generateInput(1000, 100)) - .source(Tsv("input1"), generateInput(1000, 100)) - .sink[(Int, Int, Int, Int, Int, Int)](Tsv("output")) { outBuf => skewResult ++ outBuf } - .sink[(Int, Int, Int, Int, Int, Int)](Tsv("jws-output")) { outBuf => innerResult ++ outBuf } + .source(Tsv("input1"), generateInput(100, 100)) + .sink[(Int, Int, Int, Int, Int, Int)](Tsv("output")) { outBuf => skewResult ++= outBuf } + .sink[(Int, Int, Int, Int, Int, Int)](Tsv("jws-output")) { outBuf => innerResult ++= outBuf } .run //.runHadoop //this takes MUCH longer to run. Commented out by default, but tests pass on my machine .finish From 48fa3883482578c120904c46f5f712348801abda Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Tue, 5 May 2015 14:32:33 -0700 Subject: [PATCH 152/177] Only use multisource when necessary --- .../src/main/scala/com/twitter/scalding/FileSource.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala b/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala index 15f24f8c06..aa90731aad 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala @@ -341,9 +341,9 @@ trait SuccessFileSource extends FileSource { * Put another way, this runs a Hadoop tap outside of Hadoop in the Cascading local mode */ trait LocalTapSource extends LocalSourceOverride { - override def createLocalTap(sinkMode: SinkMode) = { + override def createLocalTap(sinkMode: SinkMode): Tap[JobConf, _, _] = { val taps = localPaths.map { p => - new LocalTap(p, hdfsScheme, sinkMode).asInstanceOf[Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]]] + new LocalTap(p, hdfsScheme, sinkMode).asInstanceOf[Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]]] }.toSeq taps match { From 616292eff47eaf934a2d5b4499afb4b0180f78ef Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Tue, 5 May 2015 20:13:38 -0700 Subject: [PATCH 153/177] Move CastFileTap out; Add more explicit exception info if there're files missing in strict mode for LocalSOurceOverride --- .../scala/com/twitter/scalding/FileSource.scala | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala b/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala index aa90731aad..80c2a06b1f 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala @@ -15,7 +15,7 @@ limitations under the License. */ package com.twitter.scalding -import java.io.{ InputStream, OutputStream } +import java.io.{ File, InputStream, OutputStream } import java.util.{ UUID, Properties } import cascading.scheme.Scheme @@ -59,6 +59,12 @@ abstract class SchemedSource extends Source { val sinkMode: SinkMode = SinkMode.REPLACE } +private[scalding] object CastFileTap { + // The scala compiler has problems with the generics in Cascading + def apply(tap: FileTap): Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]] = + tap.asInstanceOf[Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]]] +} + /** * A trait which provides a method to create a local tap. */ @@ -69,12 +75,6 @@ trait LocalSourceOverride extends SchemedSource { // By default, we write to the last path for local paths def localWritePath = localPaths.last - object CastFileTap { - // The scala compiler has problems with the generics in Cascading - def apply(tap: FileTap): Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]] = - tap.asInstanceOf[Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]]] - } - /** * Creates a local tap. * @@ -223,8 +223,7 @@ abstract class FileSource extends SchemedSource with LocalSourceOverride { val files = localPaths.map{ p => new java.io.File(p) } if (strict && !files.forall(_.exists)) { throw new InvalidSourceException( - "[" + this.toString + "] Data is missing from one or more paths in: " + - localPaths.toString) + "[" + this.toString + s"] Data is missing from: $localPaths.filterNot{ p => new java.io.File(p).exists }") } else if (!files.exists(_.exists)) { throw new InvalidSourceException( "[" + this.toString + "] No good paths in: " + hdfsPaths.toString) From aeb041322752ebbd6d31699a5a3f1ed4b4275271 Mon Sep 17 00:00:00 2001 From: Ruban Monu Date: Wed, 6 May 2015 10:53:27 -0700 Subject: [PATCH 154/177] review comments --- .../twitter/scalding/commons/source/LzoGenericScheme.scala | 4 ++-- .../twitter/scalding/commons/source/LzoGenericSource.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericScheme.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericScheme.scala index c2d434d1bb..577bdf1e97 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericScheme.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericScheme.scala @@ -64,7 +64,7 @@ private[source] class ConfigBinaryConverterProvider[M] extends BinaryConverterPr import ConfigBinaryConverterProvider._ override def getConverter(conf: Configuration): BinaryConverter[M] = { val data = conf.get(ProviderConfKey) - require(data != null, s"No data in field $ProviderConfKey") + require(data != null, s"$ProviderConfKey is not set in configuration") val extern: Externalizer[_] = ExternalizerSerializer.inj.invert(data).get extern.get.asInstanceOf[BinaryConverter[M]] @@ -93,7 +93,7 @@ class LzoGenericScheme[M: ClassTag](@transient conv: BinaryConverter[M]) extends conf.set(ConfigBinaryConverterProvider.ProviderConfKey, ExternalizerSerializer.inj(extern)) - MultiInputFormat.setClassConf(conv.getClass, conf) + MultiInputFormat.setClassConf(implicitly[ClassTag[M]].runtimeClass, conf) MultiInputFormat.setGenericConverterClassConf(classOf[ConfigBinaryConverterProvider[_]], conf) DelegateCombineFileInputFormat.setDelegateInputFormat(conf, classOf[MultiInputFormat[_]]) diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericSource.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericSource.scala index 39701e5f3a..e7432980c9 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericSource.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericSource.scala @@ -29,7 +29,7 @@ import cascading.scheme.Scheme abstract class LzoGenericSource[T: ClassTag] extends FileSource with SingleMappable[T] with TypedSink[T] with LocalTapSource { def conv: BinaryConverter[T] override def setter[U <: T] = TupleSetter.asSubSetter[T, U](TupleSetter.singleSetter[T]) - override def hdfsScheme = HadoopSchemeInstance((new LzoGenericScheme(conv)).asInstanceOf[Scheme[_, _, _, _, _]]) + override def hdfsScheme = HadoopSchemeInstance((new LzoGenericScheme[T](conv)).asInstanceOf[Scheme[_, _, _, _, _]]) } object LzoGenericSource { From 478c811ed7a29dbb07d751b674e36d205982ac4a Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Wed, 6 May 2015 11:13:09 -0700 Subject: [PATCH 155/177] Format --- .../src/main/scala/com/twitter/scalding/FileSource.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala b/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala index 80c2a06b1f..f2289d2691 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala @@ -223,7 +223,7 @@ abstract class FileSource extends SchemedSource with LocalSourceOverride { val files = localPaths.map{ p => new java.io.File(p) } if (strict && !files.forall(_.exists)) { throw new InvalidSourceException( - "[" + this.toString + s"] Data is missing from: $localPaths.filterNot{ p => new java.io.File(p).exists }") + "[" + this.toString + s"] Data is missing from: ${localPaths.filterNot { p => new java.io.File(p).exists }}") } else if (!files.exists(_.exists)) { throw new InvalidSourceException( "[" + this.toString + "] No good paths in: " + hdfsPaths.toString) From 6ffa191b866fc283354d2e9f688dd8ad5b2a6485 Mon Sep 17 00:00:00 2001 From: Ruban Monu Date: Wed, 6 May 2015 13:05:03 -0700 Subject: [PATCH 156/177] support scrooge Unions --- .../commons/source/BinaryConverters.scala | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/BinaryConverters.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/BinaryConverters.scala index 1cd317cd83..24beaa7e9f 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/BinaryConverters.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/BinaryConverters.scala @@ -19,6 +19,7 @@ package com.twitter.scalding.commons.source import com.twitter.elephantbird.mapreduce.io.BinaryConverter import com.twitter.scrooge.{ BinaryThriftStructSerializer, ThriftStructCodec, ThriftStruct } import scala.reflect.ClassTag +import scala.util.Try /* * Common BinaryConverters to be used with GenericSource / GenericScheme. @@ -30,13 +31,22 @@ case object IdentityBinaryConverter extends BinaryConverter[Array[Byte]] { } object ScroogeBinaryConverter { + + // codec code borrowed from chill's ScroogeThriftStructSerializer class + private[this] def codecForNormal[T <: ThriftStruct](thriftStructClass: Class[T]): Try[ThriftStructCodec[T]] = + Try(Class.forName(thriftStructClass.getName + "$").getField("MODULE$").get(null)) + .map(_.asInstanceOf[ThriftStructCodec[T]]) + + private[this] def codecForUnion[T <: ThriftStruct](maybeUnion: Class[T]): Try[ThriftStructCodec[T]] = + Try(Class.forName(maybeUnion.getName.reverse.dropWhile(_ != '$').reverse).getField("MODULE$").get(null)) + .map(_.asInstanceOf[ThriftStructCodec[T]]) + def apply[T <: ThriftStruct: ClassTag]: BinaryConverter[T] = { val ct = implicitly[ClassTag[T]] new BinaryConverter[T] { val serializer = BinaryThriftStructSerializer[T] { - val companionClass = Class.forName(ct.runtimeClass.getName + "$") - val companionObject = companionClass.getField("MODULE$").get(null) - companionObject.asInstanceOf[ThriftStructCodec[T]] + val clazz = ct.runtimeClass.asInstanceOf[Class[T]] + codecForNormal[T](clazz).orElse(codecForUnion[T](clazz)).get } override def toBytes(struct: T) = serializer.toBytes(struct) override def fromBytes(bytes: Array[Byte]): T = serializer.fromBytes(bytes) From 8dfa354acea0a99ca8fe9bd8fe3f5e660523b21c Mon Sep 17 00:00:00 2001 From: Ruban Monu Date: Wed, 6 May 2015 13:12:57 -0700 Subject: [PATCH 157/177] set type in Externalizer --- .../scalding/commons/source/LzoCodecSource.scala | 2 +- .../scalding/commons/source/LzoGenericScheme.scala | 10 ++++++---- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoCodecSource.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoCodecSource.scala index 78b579e077..74d2ff6924 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoCodecSource.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoCodecSource.scala @@ -30,6 +30,6 @@ object LzoCodecSource { val hdfsPaths = paths val localPath = { assert(paths.size == 1, "Cannot use multiple input files on local mode"); paths(0) } val boxed = Externalizer(passedInjection) - override lazy val injection = boxed.get + override def injection = boxed.get } } diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericScheme.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericScheme.scala index 577bdf1e97..2e3d882d6b 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericScheme.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericScheme.scala @@ -37,19 +37,21 @@ import cascading.flow.FlowProcess * Serializes BinaryConverters to JobConf. */ private[source] object ExternalizerSerializer { - val inj: Injection[Externalizer[_], String] = { + def inj[T]: Injection[Externalizer[T], String] = { import com.twitter.bijection.Inversion.attemptWhen import com.twitter.bijection.codec.Base64 - implicit val baseInj = JavaSerializationInjection[Externalizer[_]] + implicit val baseInj = JavaSerializationInjection[Externalizer[T]] implicit val unwrap: Injection[GZippedBase64String, String] = + // this does not catch cases where it's Base64 but not compressed + // but the decompression injection will, so it's safe to do this new AbstractInjection[GZippedBase64String, String] { override def apply(gzbs: GZippedBase64String) = gzbs.str override def invert(str: String) = attemptWhen(str)(Base64.isBase64)(GZippedBase64String(_)) } - Injection.connect[Externalizer[_], Array[Byte], GZippedBase64String, String] + Injection.connect[Externalizer[T], Array[Byte], GZippedBase64String, String] } } @@ -66,7 +68,7 @@ private[source] class ConfigBinaryConverterProvider[M] extends BinaryConverterPr val data = conf.get(ProviderConfKey) require(data != null, s"$ProviderConfKey is not set in configuration") - val extern: Externalizer[_] = ExternalizerSerializer.inj.invert(data).get + val extern = ExternalizerSerializer.inj.invert(data).get extern.get.asInstanceOf[BinaryConverter[M]] } } From 683f5aaca8fc45a3157041e23e77a64550ede1be Mon Sep 17 00:00:00 2001 From: Steve Mardenfeld Date: Thu, 7 May 2015 17:14:57 -1000 Subject: [PATCH 158/177] fix the spelling of the cumulativeSumTest file --- .../scalding/{CumulitiveSumTest.scala => CumulativeSumTest.scala} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename scalding-core/src/test/scala/com/twitter/scalding/{CumulitiveSumTest.scala => CumulativeSumTest.scala} (100%) diff --git a/scalding-core/src/test/scala/com/twitter/scalding/CumulitiveSumTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/CumulativeSumTest.scala similarity index 100% rename from scalding-core/src/test/scala/com/twitter/scalding/CumulitiveSumTest.scala rename to scalding-core/src/test/scala/com/twitter/scalding/CumulativeSumTest.scala From 6e63331ecb95e64d0828e4f04dcd56d9c4c68237 Mon Sep 17 00:00:00 2001 From: Ruban Monu Date: Mon, 11 May 2015 13:27:30 -0700 Subject: [PATCH 159/177] Upgrade to elephantbird 4.7 --- project/Build.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/Build.scala b/project/Build.scala index ffa4197dd9..43b2f3e71b 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -26,7 +26,7 @@ object ScaldingBuild extends Build { val cascadingAvroVersion = "2.1.2" val chillVersion = "0.5.2" val dfsDatastoresVersion = "1.3.4" - val elephantbirdVersion = "4.6" + val elephantbirdVersion = "4.7" val hadoopLzoVersion = "0.4.16" val hadoopVersion = "1.2.1" val hbaseVersion = "0.94.10" From a83b9ff6d64d58ec1d4a11d21a6adb50a3fd3e68 Mon Sep 17 00:00:00 2001 From: Ruban Monu Date: Tue, 12 May 2015 10:29:21 -0700 Subject: [PATCH 160/177] build fix for LzoGenericSource --- .../twitter/scalding/commons/source/LzoGenericSource.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericSource.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericSource.scala index e7432980c9..49eb70cea2 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericSource.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericSource.scala @@ -36,7 +36,7 @@ object LzoGenericSource { def apply[T: ClassTag](passedConv: BinaryConverter[T], paths: String*) = new LzoGenericSource[T] { override val conv: BinaryConverter[T] = passedConv - val hdfsPaths = paths - val localPath = { assert(paths.size == 1, "Cannot use multiple input files on local mode"); paths(0) } + override val hdfsPaths = paths + override val localPaths = { assert(paths.size == 1, "Cannot use multiple input files on local mode"); paths } } } From 512eca502069c928c30a29ace4739bb3812fd1c7 Mon Sep 17 00:00:00 2001 From: Joe Nievelt Date: Tue, 12 May 2015 11:24:02 -0700 Subject: [PATCH 161/177] Bijection 0.8.0, algebird 0.10.0, chill 0.6.0, scala 2.10.5 --- .travis.yml | 26 +++++++++---------- project/Build.scala | 10 +++---- .../scalding/ReduceOperationsTest.scala | 13 ++++++---- 3 files changed, 26 insertions(+), 23 deletions(-) diff --git a/.travis.yml b/.travis.yml index c394d03735..119a0863d7 100644 --- a/.travis.yml +++ b/.travis.yml @@ -7,7 +7,7 @@ script: matrix: include: #BASE TESTS - - scala: 2.10.4 + - scala: 2.10.5 env: BUILD="base" TEST_TARGET="scalding-args scalding-date" script: "scripts/run_test.sh" @@ -15,7 +15,7 @@ matrix: env: BUILD="base" TEST_TARGET="scalding-args scalding-date" script: "scripts/run_test.sh" - - scala: 2.10.4 + - scala: 2.10.5 env: BUILD="base" TEST_TARGET="scalding-avro scalding-hraven scalding-commons" script: "scripts/run_test.sh" @@ -23,7 +23,7 @@ matrix: env: BUILD="base" TEST_TARGET="scalding-avro scalding-hraven scalding-commons" script: "scripts/run_test.sh" - - scala: 2.10.4 + - scala: 2.10.5 env: BUILD="base" TEST_TARGET="scalding-core" script: "scripts/run_test.sh" @@ -31,7 +31,7 @@ matrix: env: BUILD="base" TEST_TARGET="scalding-core" script: "scripts/run_test.sh" - - scala: 2.10.4 + - scala: 2.10.5 env: BUILD="base" TEST_TARGET="scalding-hadoop-test" script: "scripts/run_test.sh" @@ -39,7 +39,7 @@ matrix: env: BUILD="base" TEST_TARGET="scalding-hadoop-test" script: "scripts/run_test.sh" - - scala: 2.10.4 + - scala: 2.10.5 env: BUILD="base" TEST_TARGET="scalding-jdbc scalding-json" script: "scripts/run_test.sh" @@ -47,7 +47,7 @@ matrix: env: BUILD="base" TEST_TARGET="scalding-jdbc scalding-json" script: "scripts/run_test.sh" - - scala: 2.10.4 + - scala: 2.10.5 env: BUILD="base" TEST_TARGET="scalding-macros" script: "scripts/run_test.sh" @@ -56,7 +56,7 @@ matrix: script: "scripts/run_test.sh" # not committed yet - # - scala: 2.10.4 + # - scala: 2.10.5 # env: BUILD="base" TEST_TARGET="scalding-commons-macros" # script: "scripts/run_test.sh" @@ -64,7 +64,7 @@ matrix: # env: BUILD="base" TEST_TARGET="scalding-commons-macros" # script: "scripts/run_test.sh" - - scala: 2.10.4 + - scala: 2.10.5 env: BUILD="base" TEST_TARGET="scalding-parquet scalding-parquet-scrooge" script: "scripts/run_test.sh" @@ -72,11 +72,11 @@ matrix: env: BUILD="base" TEST_TARGET="scalding-parquet scalding-parquet-scrooge" script: "scripts/run_test.sh" - - scala: 2.10.4 + - scala: 2.10.5 env: BUILD="base" TEST_TARGET="scalding-repl" script: "scripts/run_test.sh" - - scala: 2.10.4 + - scala: 2.10.5 env: BUILD="test tutorials" script: - "scripts/build_assembly_no_test.sh scalding" @@ -88,7 +88,7 @@ matrix: - "scripts/build_assembly_no_test.sh scalding" - "scripts/test_tutorials.sh" - - scala: 2.10.4 + - scala: 2.10.5 env: BUILD="test matrix tutorials" script: - "scripts/build_assembly_no_test.sh scalding" @@ -100,7 +100,7 @@ matrix: - "scripts/build_assembly_no_test.sh scalding" - "scripts/test_matrix_tutorials.sh" - - scala: 2.10.4 + - scala: 2.10.5 env: BUILD="test repl and typed tutorials" script: - "scripts/build_assembly_no_test.sh scalding-repl" @@ -114,7 +114,7 @@ matrix: - "scripts/build_assembly_no_test.sh scalding-core" - "scripts/test_typed_tutorials.sh" - - scala: 2.10.4 + - scala: 2.10.5 env: BUILD="test execution tutorials" script: - "scripts/build_assembly_no_test.sh execution-tutorial" diff --git a/project/Build.scala b/project/Build.scala index 43b2f3e71b..0faa717b1d 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -20,11 +20,11 @@ object ScaldingBuild extends Build { } def isScala210x(scalaVersion: String) = scalaBinaryVersion(scalaVersion) == "2.10" - val algebirdVersion = "0.9.0" + val algebirdVersion = "0.10.0" val avroVersion = "1.7.4" - val bijectionVersion = "0.7.2" + val bijectionVersion = "0.8.0" val cascadingAvroVersion = "2.1.2" - val chillVersion = "0.5.2" + val chillVersion = "0.6.0" val dfsDatastoresVersion = "1.3.4" val elephantbirdVersion = "4.7" val hadoopLzoVersion = "0.4.16" @@ -47,9 +47,9 @@ object ScaldingBuild extends Build { val sharedSettings = Project.defaultSettings ++ assemblySettings ++ scalariformSettings ++ Seq( organization := "com.twitter", - scalaVersion := "2.10.4", + scalaVersion := "2.10.5", - crossScalaVersions := Seq("2.10.4", "2.11.5"), + crossScalaVersions := Seq("2.10.5", "2.11.5"), ScalariformKeys.preferences := formattingPreferences, diff --git a/scalding-core/src/test/scala/com/twitter/scalding/ReduceOperationsTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/ReduceOperationsTest.scala index 6dc773df65..bdd29eebad 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/ReduceOperationsTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/ReduceOperationsTest.scala @@ -78,6 +78,9 @@ class ApproximateUniqueCountJob(args: Args) extends Job(args) { .groupBy('category) { _.approximateUniqueCount[String]('os -> 'os_count) } + .map('os_count -> 'os_count) { + osCount: Double => osCount.toLong + } .write(Tsv("output0")) } catch { case e: Exception => e.printStackTrace() @@ -146,12 +149,12 @@ class ReduceOperationsTest extends WordSpec with Matchers { JobTest(new ApproximateUniqueCountJob(_)) .source(Tsv("input0", ('category, 'model, 'os)), inputData) - .sink[(String, Double)](Tsv("output0")) { buf => + .sink[(String, Long)](Tsv("output0")) { buf => "grouped OS count" in { - val whatWeWant: Map[String, Double] = Map( - "laptop" -> 1.0, - "mobile" -> 2.0) - val whatWeGet: Map[String, Double] = buf.toMap + val whatWeWant: Map[String, Long] = Map( + "laptop" -> 1, + "mobile" -> 2) + val whatWeGet: Map[String, Long] = buf.toMap whatWeGet should have size 2 whatWeGet.get("laptop").getOrElse("apples") shouldBe (whatWeWant.get("laptop").getOrElse("oranges")) whatWeGet.get("mobile").getOrElse("apples") shouldBe (whatWeWant.get("mobile").getOrElse("oranges")) From fe54088227d70b95fcf534579b5d56c311ff8c42 Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Tue, 12 May 2015 22:29:06 -0700 Subject: [PATCH 162/177] Remove ExecutionContextJob --- .../main/scala/com/twitter/scalding/Job.scala | 34 ------------------- 1 file changed, 34 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Job.scala b/scalding-core/src/main/scala/com/twitter/scalding/Job.scala index f83ad7ab7a..14ac2ce0a1 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Job.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Job.scala @@ -470,40 +470,6 @@ abstract class ExecutionJob[+T](args: Args) extends Job(args) { } } -/* - * this allows you to use ExecutionContext style, but wrap it in a job - * val ecFn = { (implicit ec: ExecutionContext) => - * // do stuff here - * }; - * class MyClass(args: Args) extends ExecutionContextJob(args) { - * def job = ecFn - * } - * Now you can run it with Tool as a standard Job-framework style. - * Only use this if you have an existing ExecutionContext style function - * you want to run as a Job - */ -@deprecated("Use ExecutionJob", "2014-07-29") -abstract class ExecutionContextJob[+T](args: Args) extends Job(args) { - /** - * This can be assigned from a Function1: - * def job = (ectxJob: (ExecutionContext => T)) - */ - def job: Reader[ExecutionContext, T] - /** - * This is the result of calling the job on the context for this job - * you should NOT call this in the job Reader (or reference this class at all - * in reader - */ - @transient final lazy val result: Try[T] = ec.map(job(_)) // mutate the flowDef with the job - - private[this] final def ec: Try[ExecutionContext] = - Config.tryFrom(config).map { conf => ExecutionContext.newContext(conf)(flowDef, mode) } - - override def buildFlow: Flow[_] = { - val forcedResult = result.get // make sure we have applied job once - super.buildFlow - } -} /* * Run a list of shell commands through bash in the given order. Return success From b807f2e749f0e6ef542b147a31c04bb070405b5f Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Tue, 12 May 2015 22:46:35 -0700 Subject: [PATCH 163/177] Remove deprecated approxUniques --- .../scala/com/twitter/scalding/ReduceOperations.scala | 9 --------- .../src/test/scala/com/twitter/scalding/CoreTest.scala | 7 ------- 2 files changed, 16 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/ReduceOperations.scala b/scalding-core/src/main/scala/com/twitter/scalding/ReduceOperations.scala index 851dd08c80..9fb5e651a0 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/ReduceOperations.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/ReduceOperations.scala @@ -106,15 +106,6 @@ trait ReduceOperations[+Self <: ReduceOperations[Self]] extends java.io.Serializ hyperLogLogMap[T, HLL](f, errPercent) { hll => hll } } - @deprecated("use of approximateUniqueCount is preferred.", "0.8.3") - def approxUniques(f: (Fields, Fields), errPercent: Double = 1.0) = { - // Legacy (pre-bijection) approximate unique count that uses in.toString.getBytes to - // obtain a long hash code. We specify the kludgy CTuple => Array[Byte] bijection - // explicitly. - implicit def kludgeHasher(in: CTuple) = in.toString.getBytes("UTF-8") - hyperLogLogMap[CTuple, Double](f, errPercent) { _.estimatedSize } - } - private[this] def hyperLogLogMap[T <% Array[Byte]: TupleConverter, U: TupleSetter](f: (Fields, Fields), errPercent: Double = 1.0)(fn: HLL => U) = { //bits = log(m) == 2 *log(104/errPercent) = 2log(104) - 2*log(errPercent) def log2(x: Double) = scala.math.log(x) / scala.math.log(2.0) diff --git a/scalding-core/src/test/scala/com/twitter/scalding/CoreTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/CoreTest.scala index 1244e42771..195e1c644f 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/CoreTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/CoreTest.scala @@ -1320,13 +1320,6 @@ class NormalizeTest extends WordSpec with Matchers { } } -class ApproxUniqJob(args: Args) extends Job(args) { - Tsv("in", ('x, 'y)) - .read - .groupBy('x) { _.approxUniques('y -> 'ycnt) } - .write(Tsv("out")) -} - class ApproxUniqTest extends WordSpec with Matchers { import Dsl._ From ad4a6ad7e6e748fa1af09ac25e6982f0a29da42d Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Tue, 12 May 2015 22:53:01 -0700 Subject: [PATCH 164/177] Remove plus in ReduceOperations.scala. Notice this name is very common, thus I used 'find usage' in IntelliJ and found no other files are using them. Hopefully this proves at Twitter we are not using them at least --- .../scala/com/twitter/scalding/ReduceOperations.scala | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/ReduceOperations.scala b/scalding-core/src/main/scala/com/twitter/scalding/ReduceOperations.scala index 9fb5e651a0..d6f89c1f32 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/ReduceOperations.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/ReduceOperations.scala @@ -311,17 +311,6 @@ trait ReduceOperations[+Self <: ReduceOperations[Self]] extends java.io.Serializ def sum[T](fs: Symbol*)(implicit sg: Semigroup[T], tconv: TupleConverter[T], tset: TupleSetter[T]): Self = sum[T](fs -> fs)(sg, tconv, tset) - @deprecated("Use sum", "0.9.0") - def plus[T](fd: (Fields, Fields))(implicit sg: Semigroup[T], tconv: TupleConverter[T], tset: TupleSetter[T]): Self = - sum[T](fd)(sg, tconv, tset) - /** - * The same as `plus(fs -> fs)` - * Assumed to be a commutative operation. If you don't want that, use .forceToReducers - */ - @deprecated("Use sum", "0.9.0") - def plus[T](fs: Symbol*)(implicit sg: Semigroup[T], tconv: TupleConverter[T], tset: TupleSetter[T]): Self = - sum[T](fs -> fs)(sg, tconv, tset) - /** * Returns the product of all the items in this grouping */ From 76efd696efbcda2cb5ec2d4ebe451e640800ab7b Mon Sep 17 00:00:00 2001 From: Eddie Xie Date: Tue, 12 May 2015 23:15:47 -0700 Subject: [PATCH 165/177] Delete test for deprecated ApproxUniqJob --- .../scala/com/twitter/scalding/CoreTest.scala | 20 ------------------- 1 file changed, 20 deletions(-) diff --git a/scalding-core/src/test/scala/com/twitter/scalding/CoreTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/CoreTest.scala index 195e1c644f..666128a04e 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/CoreTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/CoreTest.scala @@ -1320,26 +1320,6 @@ class NormalizeTest extends WordSpec with Matchers { } } -class ApproxUniqTest extends WordSpec with Matchers { - import Dsl._ - - "A ApproxUniqJob" should { - val input = (1 to 1000).flatMap { i => List(("x0", i), ("x1", i)) }.toList - JobTest(new ApproxUniqJob(_)) - .source(Tsv("in", ('x, 'y)), input) - .sink[(String, Double)](Tsv("out")) { outBuf => - "must approximately count" in { - outBuf should have size 2 - val kvresult = outBuf.groupBy { _._1 }.mapValues { _.head._2 } - kvresult("x0") shouldBe 1000.0 +- 30.0 //We should be 1%, but this is on average, so - kvresult("x1") shouldBe 1000.0 +- 30.0 //We should be 1%, but this is on average, so - } - } - .run - .finish - } -} - class ForceToDiskJob(args: Args) extends Job(args) { val x = Tsv("in", ('x, 'y)) .read From d6e129cdc60c383d27c783b851e47d589cfacd68 Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Tue, 12 May 2015 23:34:37 -0700 Subject: [PATCH 166/177] Move OrderedSerialization into zero-dep scalding-serialization module --- project/Build.scala | 13 +- .../serialization/KryoSerializers.scala | 11 ++ .../com/twitter/scalding/macros/Macros.scala | 3 - .../impl/OrderedBufferableProviderImpl.scala | 6 +- .../CompileTimeLengthTypes.scala | 2 +- .../ordered_serialization/ProductLike.scala | 8 +- .../TreeOrderedBuf.scala | 22 ++-- .../providers/ByteBufferOrderedBuf.scala | 4 +- .../providers/CaseClassOrderedBuf.scala | 5 +- .../providers/EitherOrderedBuf.scala | 6 +- .../providers/ImplicitOrderedBuf.scala | 12 +- .../providers/OptionOrderedBuf.scala | 6 +- .../providers/PrimitiveOrderedBuf.scala | 4 +- .../providers/ProductOrderedBuf.scala | 5 +- .../providers/StringOrderedBuf.scala | 8 +- .../providers/TraversablesOrderedBuf.scala | 26 ++-- .../providers/UnitOrderedBuf.scala | 4 +- .../runtime_helpers/LengthCalculations.scala | 2 +- .../runtime_helpers/TraversableHelpers.scala | 2 +- .../macros/MacroOrderingProperties.scala | 8 +- .../macros/TraversableHelperLaws.scala | 4 +- .../scalding/serialization/Boxed.scala | 111 ------------------ .../scalding/serialization/Hasher.scala | 0 .../serialization/JavaStreamEnrichments.scala | 0 .../twitter/scalding/serialization/Laws.scala | 0 .../serialization/MurmurHashUtils.scala | 0 .../serialization/OrderedSerialization.scala | 0 .../serialization/PositionInputStream.scala | 0 .../scalding/serialization/Reader.scala | 0 .../serialization/Serialization.scala | 0 .../serialization/Serialization2.scala | 0 .../StringOrderedSerialization.scala | 0 .../serialization/UnsignedComparisons.scala | 0 .../scalding/serialization/Writer.scala | 0 .../JavaStreamEnrichmentsProperties.scala | 0 .../SerializationProperties.scala | 0 .../UnsignedComparisonLaws.scala | 0 .../WriterReaderProperties.scala | 0 38 files changed, 89 insertions(+), 183 deletions(-) rename {scalding-macros/src/main/scala/com/twitter/scalding => scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization}/macros/impl/OrderedBufferableProviderImpl.scala (93%) rename {scalding-macros/src/main/scala/com/twitter/scalding => scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization}/macros/impl/ordered_serialization/CompileTimeLengthTypes.scala (96%) rename {scalding-macros/src/main/scala/com/twitter/scalding => scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization}/macros/impl/ordered_serialization/ProductLike.scala (93%) rename {scalding-macros/src/main/scala/com/twitter/scalding => scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization}/macros/impl/ordered_serialization/TreeOrderedBuf.scala (88%) rename {scalding-macros/src/main/scala/com/twitter/scalding => scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization}/macros/impl/ordered_serialization/providers/ByteBufferOrderedBuf.scala (93%) rename {scalding-macros/src/main/scala/com/twitter/scalding => scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization}/macros/impl/ordered_serialization/providers/CaseClassOrderedBuf.scala (92%) rename {scalding-macros/src/main/scala/com/twitter/scalding => scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization}/macros/impl/ordered_serialization/providers/EitherOrderedBuf.scala (95%) rename {scalding-macros/src/main/scala/com/twitter/scalding => scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization}/macros/impl/ordered_serialization/providers/ImplicitOrderedBuf.scala (82%) rename {scalding-macros/src/main/scala/com/twitter/scalding => scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization}/macros/impl/ordered_serialization/providers/OptionOrderedBuf.scala (93%) rename {scalding-macros/src/main/scala/com/twitter/scalding => scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization}/macros/impl/ordered_serialization/providers/PrimitiveOrderedBuf.scala (95%) rename {scalding-macros/src/main/scala/com/twitter/scalding => scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization}/macros/impl/ordered_serialization/providers/ProductOrderedBuf.scala (95%) rename {scalding-macros/src/main/scala/com/twitter/scalding => scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization}/macros/impl/ordered_serialization/providers/StringOrderedBuf.scala (90%) rename {scalding-macros/src/main/scala/com/twitter/scalding => scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization}/macros/impl/ordered_serialization/providers/TraversablesOrderedBuf.scala (87%) rename {scalding-macros/src/main/scala/com/twitter/scalding => scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization}/macros/impl/ordered_serialization/providers/UnitOrderedBuf.scala (89%) rename {scalding-macros/src/main/scala/com/twitter/scalding => scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization}/macros/impl/ordered_serialization/runtime_helpers/LengthCalculations.scala (93%) rename {scalding-macros/src/main/scala/com/twitter/scalding => scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization}/macros/impl/ordered_serialization/runtime_helpers/TraversableHelpers.scala (98%) rename {scalding-macros/src/test/scala/com/twitter/scalding => scalding-serialization-macros/src/test/scala/com/twitter/scalding/serialization}/macros/MacroOrderingProperties.scala (98%) rename {scalding-macros/src/test/scala/com/twitter/scalding => scalding-serialization-macros/src/test/scala/com/twitter/scalding/serialization}/macros/TraversableHelperLaws.scala (93%) rename {scalding-core => scalding-serialization}/src/main/scala/com/twitter/scalding/serialization/Boxed.scala (69%) rename {scalding-core => scalding-serialization}/src/main/scala/com/twitter/scalding/serialization/Hasher.scala (100%) rename {scalding-core => scalding-serialization}/src/main/scala/com/twitter/scalding/serialization/JavaStreamEnrichments.scala (100%) rename {scalding-core => scalding-serialization}/src/main/scala/com/twitter/scalding/serialization/Laws.scala (100%) rename {scalding-core => scalding-serialization}/src/main/scala/com/twitter/scalding/serialization/MurmurHashUtils.scala (100%) rename {scalding-core => scalding-serialization}/src/main/scala/com/twitter/scalding/serialization/OrderedSerialization.scala (100%) rename {scalding-core => scalding-serialization}/src/main/scala/com/twitter/scalding/serialization/PositionInputStream.scala (100%) rename {scalding-core => scalding-serialization}/src/main/scala/com/twitter/scalding/serialization/Reader.scala (100%) rename {scalding-core => scalding-serialization}/src/main/scala/com/twitter/scalding/serialization/Serialization.scala (100%) rename {scalding-core => scalding-serialization}/src/main/scala/com/twitter/scalding/serialization/Serialization2.scala (100%) rename {scalding-core => scalding-serialization}/src/main/scala/com/twitter/scalding/serialization/StringOrderedSerialization.scala (100%) rename {scalding-core => scalding-serialization}/src/main/scala/com/twitter/scalding/serialization/UnsignedComparisons.scala (100%) rename {scalding-core => scalding-serialization}/src/main/scala/com/twitter/scalding/serialization/Writer.scala (100%) rename {scalding-core => scalding-serialization}/src/test/scala/com/twitter/scalding/serialization/JavaStreamEnrichmentsProperties.scala (100%) rename {scalding-core => scalding-serialization}/src/test/scala/com/twitter/scalding/serialization/SerializationProperties.scala (100%) rename {scalding-core => scalding-serialization}/src/test/scala/com/twitter/scalding/serialization/UnsignedComparisonLaws.scala (100%) rename {scalding-core => scalding-serialization}/src/test/scala/com/twitter/scalding/serialization/WriterReaderProperties.scala (100%) diff --git a/project/Build.scala b/project/Build.scala index 0faa717b1d..dc854780e9 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -270,7 +270,7 @@ object ScaldingBuild extends Build { "org.slf4j" % "slf4j-api" % slf4jVersion, "org.slf4j" % "slf4j-log4j12" % slf4jVersion % "provided" ) - ).dependsOn(scaldingArgs, scaldingDate, maple) + ).dependsOn(scaldingArgs, scaldingDate, scaldingSerialization, maple) lazy val scaldingCommons = module("commons").settings( libraryDependencies ++= Seq( @@ -387,6 +387,17 @@ object ScaldingBuild extends Build { run <<= (run in Unprovided) ) + // zero dependency serialization module + lazy val scaldingSerialization = module("serialization") + lazy val scaldingSerializationMacros = module("serialization-macros").settings( + libraryDependencies <++= (scalaVersion) { scalaVersion => Seq( + "org.scala-lang" % "scala-library" % scalaVersion, + "org.scala-lang" % "scala-reflect" % scalaVersion + ) ++ (if(isScala210x(scalaVersion)) Seq("org.scalamacros" %% "quasiquotes" % "2.0.1") else Seq()) + }, + addCompilerPlugin("org.scalamacros" % "paradise" % "2.0.1" cross CrossVersion.full) + ).dependsOn(scaldingSerialization) + lazy val scaldingJson = module("json").settings( libraryDependencies <++= (scalaVersion) { scalaVersion => Seq( "org.apache.hadoop" % "hadoop-core" % hadoopVersion % "provided", diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoSerializers.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoSerializers.scala index 96999f198c..859d223431 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoSerializers.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoSerializers.scala @@ -21,6 +21,17 @@ import com.esotericsoftware.kryo.io.{ Input, Output } import com.twitter.scalding._ +/** + * This is a runtime check for types we should never be serializing + */ +class ThrowingSerializer[T] extends KSerializer[T] { + override def write(kryo: Kryo, output: Output, t: T) { + sys.error(s"Kryo should never be used to serialize an instance: $t") + } + override def read(kryo: Kryo, input: Input, t: Class[T]): T = + sys.error("Kryo should never be used to serialize an instance, class: $t") +} + /** * * * Below are some serializers for objects in the scalding project. diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/Macros.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/Macros.scala index 72d6bb9883..cd22bc0816 100644 --- a/scalding-macros/src/main/scala/com/twitter/scalding/macros/Macros.scala +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/Macros.scala @@ -45,7 +45,4 @@ object Macros { def caseClassTypeDescriptor[T]: TypeDescriptor[T] = macro TypeDescriptorProviderImpl.caseClassTypeDescriptorImpl[T] def caseClassTypeDescriptorWithUnknown[T]: TypeDescriptor[T] = macro TypeDescriptorProviderImpl.caseClassTypeDescriptorWithUnknownImpl[T] - - def orderedBufferSupplier[T]: OrderedSerialization[T] = macro com.twitter.scalding.macros.impl.OrderedSerializationProviderImpl[T] - } diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/OrderedBufferableProviderImpl.scala b/scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/OrderedBufferableProviderImpl.scala similarity index 93% rename from scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/OrderedBufferableProviderImpl.scala rename to scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/OrderedBufferableProviderImpl.scala index 4085b5c6db..3bb815ebf8 100644 --- a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/OrderedBufferableProviderImpl.scala +++ b/scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/OrderedBufferableProviderImpl.scala @@ -13,15 +13,15 @@ See the License for the specific language governing permissions and limitations under the License. */ -package com.twitter.scalding.macros.impl +package com.twitter.scalding.serialization.macros.impl import scala.language.experimental.macros import scala.reflect.macros.Context import scala.util.Random import com.twitter.scalding.serialization.OrderedSerialization -import com.twitter.scalding.macros.impl.ordered_serialization._ -import com.twitter.scalding.macros.impl.ordered_serialization.providers._ +import com.twitter.scalding.serialization.macros.impl.ordered_serialization._ +import com.twitter.scalding.serialization.macros.impl.ordered_serialization.providers._ object OrderedSerializationProviderImpl { def normalizedDispatcher(c: Context)(buildDispatcher: => PartialFunction[c.Type, TreeOrderedBuf[c.type]]): PartialFunction[c.Type, TreeOrderedBuf[c.type]] = { diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/CompileTimeLengthTypes.scala b/scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/CompileTimeLengthTypes.scala similarity index 96% rename from scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/CompileTimeLengthTypes.scala rename to scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/CompileTimeLengthTypes.scala index 8b419491b1..7a2640c603 100644 --- a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/CompileTimeLengthTypes.scala +++ b/scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/CompileTimeLengthTypes.scala @@ -13,7 +13,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package com.twitter.scalding.macros.impl.ordered_serialization +package com.twitter.scalding.serialization.macros.impl.ordered_serialization import scala.language.experimental.macros import scala.reflect.macros.Context diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/ProductLike.scala b/scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/ProductLike.scala similarity index 93% rename from scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/ProductLike.scala rename to scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/ProductLike.scala index 3975ffa529..23f874d6f1 100644 --- a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/ProductLike.scala +++ b/scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/ProductLike.scala @@ -13,7 +13,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package com.twitter.scalding.macros.impl.ordered_serialization +package com.twitter.scalding.serialization.macros.impl.ordered_serialization import scala.language.experimental.macros import scala.reflect.macros.Context @@ -111,9 +111,9 @@ object ProductLike { FastLengthCalculation(c)(combinedDynamic) } else { - val const = q"_root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.ConstLen" - val dyn = q"_root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.DynamicLen" - val noLen = q"_root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.NoLengthCalculation" + val const = q"_root_.com.twitter.scalding.serialization.macros.impl.ordered_serialization.runtime_helpers.ConstLen" + val dyn = q"_root_.com.twitter.scalding.serialization.macros.impl.ordered_serialization.runtime_helpers.DynamicLen" + val noLen = q"_root_.com.twitter.scalding.serialization.macros.impl.ordered_serialization.runtime_helpers.NoLengthCalculation" // Contains an MaybeLength val combinedMaybe: Tree = maybeLength.reduce { (hOpt, nxtOpt) => q"""$hOpt + $nxtOpt""" } if (dynamicFunctions.nonEmpty || constSize != 0) { diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/TreeOrderedBuf.scala b/scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/TreeOrderedBuf.scala similarity index 88% rename from scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/TreeOrderedBuf.scala rename to scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/TreeOrderedBuf.scala index 885e0a6253..d336e8d70a 100644 --- a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/TreeOrderedBuf.scala +++ b/scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/TreeOrderedBuf.scala @@ -13,7 +13,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package com.twitter.scalding.macros.impl.ordered_serialization +package com.twitter.scalding.serialization.macros.impl.ordered_serialization import com.twitter.scalding._ import com.twitter.scalding.serialization.OrderedSerialization @@ -80,14 +80,14 @@ object TreeOrderedBuf { case _: NoLengthCalculationAvailable[_] => None case const: ConstantLengthCalculation[_] => None case f: FastLengthCalculation[_] => Some(q""" - _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.DynamicLen(${f.asInstanceOf[FastLengthCalculation[c.type]].t}) + _root_.com.twitter.scalding.serialization.macros.impl.ordered_serialization.runtime_helpers.DynamicLen(${f.asInstanceOf[FastLengthCalculation[c.type]].t}) """) case m: MaybeLengthCalculation[_] => Some(m.asInstanceOf[MaybeLengthCalculation[c.type]].t) } fnBodyOpt.map { fnBody => q""" - @inline private[this] def payloadLength($element: $T): _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.MaybeLength = { + @inline private[this] def payloadLength($element: $T): _root_.com.twitter.scalding.serialization.macros.impl.ordered_serialization.runtime_helpers.MaybeLength = { $fnBody } """ @@ -103,9 +103,9 @@ object TreeOrderedBuf { override def dynamicSize($element: $typeName): Option[Int] = { val $tempLen = payloadLength($element) match { - case _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.NoLengthCalculation => None - case _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.ConstLen(l) => Some(l) - case _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.DynamicLen(l) => Some(l) + case _root_.com.twitter.scalding.serialization.macros.impl.ordered_serialization.runtime_helpers.NoLengthCalculation => None + case _root_.com.twitter.scalding.serialization.macros.impl.ordered_serialization.runtime_helpers.ConstLen(l) => Some(l) + case _root_.com.twitter.scalding.serialization.macros.impl.ordered_serialization.runtime_helpers.DynamicLen(l) => Some(l) } (if ($tempLen.isDefined) { // Avoid a closure here while we are geeking out @@ -172,11 +172,11 @@ object TreeOrderedBuf { @inline def withLenCalc(cnt: Int) = { ${withLenCalc(q"cnt")} } - val $tmpLenRes: _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.MaybeLength = payloadLength($element) + val $tmpLenRes: _root_.com.twitter.scalding.serialization.macros.impl.ordered_serialization.runtime_helpers.MaybeLength = payloadLength($element) $tmpLenRes match { - case _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.NoLengthCalculation => noLenCalc - case _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.ConstLen(const) => withLenCalc(const) - case _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.DynamicLen(s) => withLenCalc(s) + case _root_.com.twitter.scalding.serialization.macros.impl.ordered_serialization.runtime_helpers.NoLengthCalculation => noLenCalc + case _root_.com.twitter.scalding.serialization.macros.impl.ordered_serialization.runtime_helpers.ConstLen(const) => withLenCalc(const) + case _root_.com.twitter.scalding.serialization.macros.impl.ordered_serialization.runtime_helpers.DynamicLen(s) => withLenCalc(s) } """ } @@ -213,7 +213,7 @@ object TreeOrderedBuf { val lenB = freshT("lenB") t.ctx.Expr[OrderedSerialization[T]](q""" - new _root_.com.twitter.scalding.serialization.OrderedSerialization[$T] with _root_.com.twitter.bijection.macros.MacroGenerated { + new _root_.com.twitter.scalding.serialization.OrderedSerialization[$T] { import _root_.com.twitter.scalding.serialization.JavaStreamEnrichments._ ..$lazyVariables diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/ByteBufferOrderedBuf.scala b/scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/ByteBufferOrderedBuf.scala similarity index 93% rename from scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/ByteBufferOrderedBuf.scala rename to scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/ByteBufferOrderedBuf.scala index 3bf0608b32..af26712f42 100644 --- a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/ByteBufferOrderedBuf.scala +++ b/scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/ByteBufferOrderedBuf.scala @@ -13,13 +13,13 @@ See the License for the specific language governing permissions and limitations under the License. */ -package com.twitter.scalding.macros.impl.ordered_serialization.providers +package com.twitter.scalding.serialization.macros.impl.ordered_serialization.providers import scala.language.experimental.macros import scala.reflect.macros.Context import com.twitter.scalding._ -import com.twitter.scalding.macros.impl.ordered_serialization.{ CompileTimeLengthTypes, ProductLike, TreeOrderedBuf } +import com.twitter.scalding.serialization.macros.impl.ordered_serialization.{ CompileTimeLengthTypes, ProductLike, TreeOrderedBuf } import CompileTimeLengthTypes._ import java.nio.ByteBuffer diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/CaseClassOrderedBuf.scala b/scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/CaseClassOrderedBuf.scala similarity index 92% rename from scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/CaseClassOrderedBuf.scala rename to scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/CaseClassOrderedBuf.scala index d286e854b2..4f2ab075f3 100644 --- a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/CaseClassOrderedBuf.scala +++ b/scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/CaseClassOrderedBuf.scala @@ -13,16 +13,15 @@ See the License for the specific language governing permissions and limitations under the License. */ -package com.twitter.scalding.macros.impl.ordered_serialization.providers +package com.twitter.scalding.serialization.macros.impl.ordered_serialization.providers import scala.language.experimental.macros import scala.reflect.macros.Context import com.twitter.scalding._ -import com.twitter.scalding.macros.impl.ordered_serialization.{ CompileTimeLengthTypes, ProductLike, TreeOrderedBuf } +import com.twitter.scalding.serialization.macros.impl.ordered_serialization.{ CompileTimeLengthTypes, ProductLike, TreeOrderedBuf } import CompileTimeLengthTypes._ import com.twitter.scalding.serialization.OrderedSerialization -import com.twitter.bijection.macros.impl.IsCaseClassImpl object CaseClassOrderedBuf { def dispatch(c: Context)(buildDispatcher: => PartialFunction[c.Type, TreeOrderedBuf[c.type]]): PartialFunction[c.Type, TreeOrderedBuf[c.type]] = { diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/EitherOrderedBuf.scala b/scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/EitherOrderedBuf.scala similarity index 95% rename from scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/EitherOrderedBuf.scala rename to scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/EitherOrderedBuf.scala index aff24ebbb1..541302c2c0 100644 --- a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/EitherOrderedBuf.scala +++ b/scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/EitherOrderedBuf.scala @@ -13,13 +13,13 @@ See the License for the specific language governing permissions and limitations under the License. */ -package com.twitter.scalding.macros.impl.ordered_serialization.providers +package com.twitter.scalding.serialization.macros.impl.ordered_serialization.providers import scala.language.experimental.macros import scala.reflect.macros.Context import com.twitter.scalding._ -import com.twitter.scalding.macros.impl.ordered_serialization.{ CompileTimeLengthTypes, ProductLike, TreeOrderedBuf } +import com.twitter.scalding.serialization.macros.impl.ordered_serialization.{ CompileTimeLengthTypes, ProductLike, TreeOrderedBuf } import CompileTimeLengthTypes._ import com.twitter.scalding.serialization.OrderedSerialization @@ -142,7 +142,7 @@ object EitherOrderedBuf { override def length(element: Tree): CompileTimeLengthTypes[c.type] = { def tree(ctl: CompileTimeLengthTypes[_]): c.Tree = ctl.asInstanceOf[CompileTimeLengthTypes[c.type]].t - val dyn = q"""_root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.DynamicLen""" + val dyn = q"""_root_.com.twitter.scalding.serialization.macros.impl.ordered_serialization.runtime_helpers.DynamicLen""" (leftBuf.length(q"$element.left.get"), rightBuf.length(q"$element.right.get")) match { case (lconst: ConstantLengthCalculation[_], rconst: ConstantLengthCalculation[_]) if lconst.toInt == rconst.toInt => diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/ImplicitOrderedBuf.scala b/scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/ImplicitOrderedBuf.scala similarity index 82% rename from scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/ImplicitOrderedBuf.scala rename to scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/ImplicitOrderedBuf.scala index 2f40fb3b98..056d7e79b3 100644 --- a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/ImplicitOrderedBuf.scala +++ b/scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/ImplicitOrderedBuf.scala @@ -13,14 +13,14 @@ See the License for the specific language governing permissions and limitations under the License. */ -package com.twitter.scalding.macros.impl.ordered_serialization.providers +package com.twitter.scalding.serialization.macros.impl.ordered_serialization.providers import scala.language.experimental.macros import scala.reflect.macros.Context import com.twitter.scalding._ import com.twitter.scalding.serialization.OrderedSerialization -import com.twitter.scalding.macros.impl.ordered_serialization._ +import com.twitter.scalding.serialization.macros.impl.ordered_serialization._ /* A fall back ordered bufferable to look for the user to have an implicit in scope to satisfy the missing @@ -63,15 +63,15 @@ object ImplicitOrderedBuf { override def length(element: Tree) = CompileTimeLengthTypes.MaybeLengthCalculation(c)(q""" ($variableName.staticSize match { - case Some(s) => _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.ConstLen(s) + case Some(s) => _root_.com.twitter.scalding.serialization.macros.impl.ordered_serialization.runtime_helpers.ConstLen(s) case None => $variableName.dynamicSize($element) match { case Some(s) => - _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.DynamicLen(s) + _root_.com.twitter.scalding.serialization.macros.impl.ordered_serialization.runtime_helpers.DynamicLen(s) case None => - _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.NoLengthCalculation + _root_.com.twitter.scalding.serialization.macros.impl.ordered_serialization.runtime_helpers.NoLengthCalculation } - }): _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.MaybeLength + }): _root_.com.twitter.scalding.serialization.macros.impl.ordered_serialization.runtime_helpers.MaybeLength """) override def get(inputStream: ctx.TermName): ctx.Tree = diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/OptionOrderedBuf.scala b/scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/OptionOrderedBuf.scala similarity index 93% rename from scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/OptionOrderedBuf.scala rename to scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/OptionOrderedBuf.scala index 29c2ed8a5d..7d2c1403b8 100644 --- a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/OptionOrderedBuf.scala +++ b/scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/OptionOrderedBuf.scala @@ -13,13 +13,13 @@ See the License for the specific language governing permissions and limitations under the License. */ -package com.twitter.scalding.macros.impl.ordered_serialization.providers +package com.twitter.scalding.serialization.macros.impl.ordered_serialization.providers import scala.language.experimental.macros import scala.reflect.macros.Context import com.twitter.scalding._ -import com.twitter.scalding.macros.impl.ordered_serialization.{ CompileTimeLengthTypes, ProductLike, TreeOrderedBuf } +import com.twitter.scalding.serialization.macros.impl.ordered_serialization.{ CompileTimeLengthTypes, ProductLike, TreeOrderedBuf } import CompileTimeLengthTypes._ import com.twitter.scalding.serialization.OrderedSerialization @@ -134,7 +134,7 @@ object OptionOrderedBuf { """) case m: MaybeLengthCalculation[_] => val t = m.asInstanceOf[MaybeLengthCalculation[c.type]].t - val dynlen = q"""_root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.DynamicLen""" + val dynlen = q"""_root_.com.twitter.scalding.serialization.macros.impl.ordered_serialization.runtime_helpers.DynamicLen""" MaybeLengthCalculation(c)(q""" if ($element.isDefined) { $t + $dynlen(1) } else { $dynlen(1) } diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/PrimitiveOrderedBuf.scala b/scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/PrimitiveOrderedBuf.scala similarity index 95% rename from scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/PrimitiveOrderedBuf.scala rename to scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/PrimitiveOrderedBuf.scala index 8be80e8ecc..41b7584763 100644 --- a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/PrimitiveOrderedBuf.scala +++ b/scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/PrimitiveOrderedBuf.scala @@ -13,13 +13,13 @@ See the License for the specific language governing permissions and limitations under the License. */ -package com.twitter.scalding.macros.impl.ordered_serialization.providers +package com.twitter.scalding.serialization.macros.impl.ordered_serialization.providers import scala.language.experimental.macros import scala.reflect.macros.Context import com.twitter.scalding._ -import com.twitter.scalding.macros.impl.ordered_serialization.{ CompileTimeLengthTypes, ProductLike, TreeOrderedBuf } +import com.twitter.scalding.serialization.macros.impl.ordered_serialization.{ CompileTimeLengthTypes, ProductLike, TreeOrderedBuf } import CompileTimeLengthTypes._ import java.nio.ByteBuffer import com.twitter.scalding.serialization.OrderedSerialization diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/ProductOrderedBuf.scala b/scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/ProductOrderedBuf.scala similarity index 95% rename from scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/ProductOrderedBuf.scala rename to scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/ProductOrderedBuf.scala index d58133cdd4..4c534bce11 100644 --- a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/ProductOrderedBuf.scala +++ b/scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/ProductOrderedBuf.scala @@ -13,17 +13,16 @@ See the License for the specific language governing permissions and limitations under the License. */ -package com.twitter.scalding.macros.impl.ordered_serialization.providers +package com.twitter.scalding.serialization.macros.impl.ordered_serialization.providers import scala.language.experimental.macros import scala.reflect.macros.Context import com.twitter.scalding._ -import com.twitter.scalding.macros.impl.ordered_serialization.{ CompileTimeLengthTypes, ProductLike, TreeOrderedBuf } +import com.twitter.scalding.serialization.macros.impl.ordered_serialization.{ CompileTimeLengthTypes, ProductLike, TreeOrderedBuf } import CompileTimeLengthTypes._ import java.nio.ByteBuffer import com.twitter.scalding.serialization.OrderedSerialization -import com.twitter.bijection.macros.impl.IsCaseClassImpl object ProductOrderedBuf { def dispatch(c: Context)(buildDispatcher: => PartialFunction[c.Type, TreeOrderedBuf[c.type]]): PartialFunction[c.Type, TreeOrderedBuf[c.type]] = { diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/StringOrderedBuf.scala b/scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/StringOrderedBuf.scala similarity index 90% rename from scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/StringOrderedBuf.scala rename to scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/StringOrderedBuf.scala index d29dde6e4a..ac00d69680 100644 --- a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/StringOrderedBuf.scala +++ b/scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/StringOrderedBuf.scala @@ -13,13 +13,13 @@ See the License for the specific language governing permissions and limitations under the License. */ -package com.twitter.scalding.macros.impl.ordered_serialization.providers +package com.twitter.scalding.serialization.macros.impl.ordered_serialization.providers import scala.language.experimental.macros import scala.reflect.macros.Context import com.twitter.scalding._ -import com.twitter.scalding.macros.impl.ordered_serialization.{ CompileTimeLengthTypes, ProductLike, TreeOrderedBuf } +import com.twitter.scalding.serialization.macros.impl.ordered_serialization.{ CompileTimeLengthTypes, ProductLike, TreeOrderedBuf } import CompileTimeLengthTypes._ import java.nio.ByteBuffer import com.twitter.scalding.serialization.OrderedSerialization @@ -115,9 +115,9 @@ object StringOrderedBuf { override val lazyOuterVariables: Map[String, ctx.Tree] = Map.empty override def length(element: Tree): CompileTimeLengthTypes[c.type] = MaybeLengthCalculation(c)(q""" if($element.isEmpty) { - _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.DynamicLen(1) + _root_.com.twitter.scalding.serialization.macros.impl.ordered_serialization.runtime_helpers.DynamicLen(1) } else { - _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.NoLengthCalculation + _root_.com.twitter.scalding.serialization.macros.impl.ordered_serialization.runtime_helpers.NoLengthCalculation } """) } diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/TraversablesOrderedBuf.scala b/scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/TraversablesOrderedBuf.scala similarity index 87% rename from scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/TraversablesOrderedBuf.scala rename to scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/TraversablesOrderedBuf.scala index e7d47c9cff..b552645cb5 100644 --- a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/TraversablesOrderedBuf.scala +++ b/scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/TraversablesOrderedBuf.scala @@ -13,14 +13,14 @@ See the License for the specific language governing permissions and limitations under the License. */ -package com.twitter.scalding.macros.impl.ordered_serialization.providers +package com.twitter.scalding.serialization.macros.impl.ordered_serialization.providers import scala.language.experimental.macros import scala.reflect.macros.Context import java.io.InputStream import com.twitter.scalding._ -import com.twitter.scalding.macros.impl.ordered_serialization.{ CompileTimeLengthTypes, ProductLike, TreeOrderedBuf } +import com.twitter.scalding.serialization.macros.impl.ordered_serialization.{ CompileTimeLengthTypes, ProductLike, TreeOrderedBuf } import CompileTimeLengthTypes._ import com.twitter.scalding.serialization.OrderedSerialization import scala.reflect.ClassTag @@ -119,7 +119,7 @@ object TraversablesOrderedBuf { val $b = b ${innerBuf.compareBinary(a, b)} }; - _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.TraversableHelpers.rawCompare($inputStreamA, $inputStreamB)($innerCompareFn) + _root_.com.twitter.scalding.serialization.macros.impl.ordered_serialization.runtime_helpers.TraversableHelpers.rawCompare($inputStreamA, $inputStreamB)($innerCompareFn) """ } @@ -244,12 +244,12 @@ object TraversablesOrderedBuf { maybeSort match { case DoSort => q""" - _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.TraversableHelpers.sortedCompare[${innerBuf.tpe}]($elementA, $elementB)($innerOrd) + _root_.com.twitter.scalding.serialization.macros.impl.ordered_serialization.runtime_helpers.TraversableHelpers.sortedCompare[${innerBuf.tpe}]($elementA, $elementB)($innerOrd) """ case NoSort => q""" - _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.TraversableHelpers.iteratorCompare[${innerBuf.tpe}]($elementA.iterator, $elementB.iterator)($innerOrd) + _root_.com.twitter.scalding.serialization.macros.impl.ordered_serialization.runtime_helpers.TraversableHelpers.iteratorCompare[${innerBuf.tpe}]($elementA.iterator, $elementB.iterator)($innerOrd) """ } @@ -269,20 +269,20 @@ object TraversablesOrderedBuf { MaybeLengthCalculation(c)(q""" if($element.isEmpty) { val sizeOfZero = 1 // writing the constant 0, for length, takes 1 byte - _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.DynamicLen(sizeOfZero) + _root_.com.twitter.scalding.serialization.macros.impl.ordered_serialization.runtime_helpers.DynamicLen(sizeOfZero) } else { val maybeRes = ${m.asInstanceOf[MaybeLengthCalculation[c.type]].t} maybeRes match { - case _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.ConstLen(constSize) => + case _root_.com.twitter.scalding.serialization.macros.impl.ordered_serialization.runtime_helpers.ConstLen(constSize) => val sizeOverhead = posVarIntSize($element.size) - _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.DynamicLen(constSize * $element.size + sizeOverhead) + _root_.com.twitter.scalding.serialization.macros.impl.ordered_serialization.runtime_helpers.DynamicLen(constSize * $element.size + sizeOverhead) // todo maybe we should support this case // where we can visit every member of the list relatively fast to ask // its length. Should we care about sizes instead maybe? - case _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.DynamicLen(_) => - _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.NoLengthCalculation - case _ => _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.NoLengthCalculation + case _root_.com.twitter.scalding.serialization.macros.impl.ordered_serialization.runtime_helpers.DynamicLen(_) => + _root_.com.twitter.scalding.serialization.macros.impl.ordered_serialization.runtime_helpers.NoLengthCalculation + case _ => _root_.com.twitter.scalding.serialization.macros.impl.ordered_serialization.runtime_helpers.NoLengthCalculation } } """) @@ -290,9 +290,9 @@ object TraversablesOrderedBuf { case _ => MaybeLengthCalculation(c)(q""" if($element.isEmpty) { val sizeOfZero = 1 // writing the constant 0, for length, takes 1 byte - _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.DynamicLen(sizeOfZero) + _root_.com.twitter.scalding.serialization.macros.impl.ordered_serialization.runtime_helpers.DynamicLen(sizeOfZero) } else { - _root_.com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.NoLengthCalculation + _root_.com.twitter.scalding.serialization.macros.impl.ordered_serialization.runtime_helpers.NoLengthCalculation } """) } diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/UnitOrderedBuf.scala b/scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/UnitOrderedBuf.scala similarity index 89% rename from scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/UnitOrderedBuf.scala rename to scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/UnitOrderedBuf.scala index bbce0c81a8..e0cedb05a9 100644 --- a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/providers/UnitOrderedBuf.scala +++ b/scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/UnitOrderedBuf.scala @@ -13,13 +13,13 @@ See the License for the specific language governing permissions and limitations under the License. */ -package com.twitter.scalding.macros.impl.ordered_serialization.providers +package com.twitter.scalding.serialization.macros.impl.ordered_serialization.providers import scala.language.experimental.macros import scala.reflect.macros.Context import com.twitter.scalding._ -import com.twitter.scalding.macros.impl.ordered_serialization.{ CompileTimeLengthTypes, ProductLike, TreeOrderedBuf } +import com.twitter.scalding.serialization.macros.impl.ordered_serialization.{ CompileTimeLengthTypes, ProductLike, TreeOrderedBuf } import CompileTimeLengthTypes._ import java.nio.ByteBuffer import com.twitter.scalding.serialization.OrderedSerialization diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/runtime_helpers/LengthCalculations.scala b/scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/runtime_helpers/LengthCalculations.scala similarity index 93% rename from scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/runtime_helpers/LengthCalculations.scala rename to scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/runtime_helpers/LengthCalculations.scala index 22e5738009..137c82060b 100644 --- a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/runtime_helpers/LengthCalculations.scala +++ b/scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/runtime_helpers/LengthCalculations.scala @@ -13,7 +13,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers +package com.twitter.scalding.serialization.macros.impl.ordered_serialization.runtime_helpers /** * There is a Monoid on MaybeLength, with diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/runtime_helpers/TraversableHelpers.scala b/scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/runtime_helpers/TraversableHelpers.scala similarity index 98% rename from scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/runtime_helpers/TraversableHelpers.scala rename to scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/runtime_helpers/TraversableHelpers.scala index 4cbb5ebba4..8e9961e9f4 100644 --- a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/ordered_serialization/runtime_helpers/TraversableHelpers.scala +++ b/scalding-serialization-macros/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/runtime_helpers/TraversableHelpers.scala @@ -13,7 +13,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers +package com.twitter.scalding.serialization.macros.impl.ordered_serialization.runtime_helpers import java.io.InputStream import scala.collection.mutable.Buffer diff --git a/scalding-macros/src/test/scala/com/twitter/scalding/macros/MacroOrderingProperties.scala b/scalding-serialization-macros/src/test/scala/com/twitter/scalding/serialization/macros/MacroOrderingProperties.scala similarity index 98% rename from scalding-macros/src/test/scala/com/twitter/scalding/macros/MacroOrderingProperties.scala rename to scalding-serialization-macros/src/test/scala/com/twitter/scalding/serialization/macros/MacroOrderingProperties.scala index 49c7263691..ee0bc5fc8f 100644 --- a/scalding-macros/src/test/scala/com/twitter/scalding/macros/MacroOrderingProperties.scala +++ b/scalding-serialization-macros/src/test/scala/com/twitter/scalding/serialization/macros/MacroOrderingProperties.scala @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package com.twitter.scalding.macros +package com.twitter.scalding.serialization.macros import org.scalatest.{ FunSuite, ShouldMatchers } import org.scalatest.prop.Checkers @@ -25,14 +25,13 @@ import java.nio.ByteBuffer import org.scalacheck.Arbitrary.{ arbitrary => arb } import java.io.{ ByteArrayOutputStream, InputStream } -import com.twitter.bijection.Bufferable import org.scalacheck.{ Arbitrary, Gen, Prop } import com.twitter.scalding.serialization.JavaStreamEnrichments import scala.collection.immutable.Queue trait LowerPriorityImplicit { - implicit def primitiveOrderedBufferSupplier[T] = macro com.twitter.scalding.macros.impl.OrderedSerializationProviderImpl[T] + implicit def primitiveOrderedBufferSupplier[T] = macro impl.OrderedSerializationProviderImpl[T] } object LawTester { @@ -87,9 +86,10 @@ class MyData(override val _1: Int, override val _2: Option[Long]) extends Produc } object MacroOpaqueContainer { + def getOrdSer[T]: OrderedSerialization[T] = macro impl.OrderedSerializationProviderImpl[T] import java.io._ implicit val myContainerOrderedSerializer = new OrderedSerialization[MacroOpaqueContainer] { - val intOrderedSerialization = _root_.com.twitter.scalding.macros.Macros.orderedBufferSupplier[Int] + val intOrderedSerialization = getOrdSer[Int] override def hash(s: MacroOpaqueContainer) = intOrderedSerialization.hash(s.myField) ^ Int.MaxValue override def compare(a: MacroOpaqueContainer, b: MacroOpaqueContainer) = intOrderedSerialization.compare(a.myField, b.myField) diff --git a/scalding-macros/src/test/scala/com/twitter/scalding/macros/TraversableHelperLaws.scala b/scalding-serialization-macros/src/test/scala/com/twitter/scalding/serialization/macros/TraversableHelperLaws.scala similarity index 93% rename from scalding-macros/src/test/scala/com/twitter/scalding/macros/TraversableHelperLaws.scala rename to scalding-serialization-macros/src/test/scala/com/twitter/scalding/serialization/macros/TraversableHelperLaws.scala index 5b54e7d6b4..922c74f51f 100644 --- a/scalding-macros/src/test/scala/com/twitter/scalding/macros/TraversableHelperLaws.scala +++ b/scalding-serialization-macros/src/test/scala/com/twitter/scalding/serialization/macros/TraversableHelperLaws.scala @@ -14,13 +14,13 @@ See the License for the specific language governing permissions and limitations under the License. */ -package com.twitter.scalding.macros +package com.twitter.scalding.serialization.macros import org.scalatest.{ FunSuite, ShouldMatchers } import org.scalatest.prop.Checkers import org.scalatest.prop.PropertyChecks -import com.twitter.scalding.macros.impl.ordered_serialization.runtime_helpers.TraversableHelpers._ +import impl.ordered_serialization.runtime_helpers.TraversableHelpers._ class TraversableHelperLaws extends FunSuite with PropertyChecks with ShouldMatchers { test("Iterator ordering should be Iterable ordering") { diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Boxed.scala b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/Boxed.scala similarity index 69% rename from scalding-core/src/main/scala/com/twitter/scalding/serialization/Boxed.scala rename to scalding-serialization/src/main/scala/com/twitter/scalding/serialization/Boxed.scala index edf2eb97ce..5e57a3427e 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Boxed.scala +++ b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/Boxed.scala @@ -18,9 +18,6 @@ package com.twitter.scalding.serialization import java.util.concurrent.atomic.AtomicReference import java.io.{ InputStream, OutputStream } -import com.esotericsoftware.kryo.{ Serializer => KSerializer, DefaultSerializer, Kryo } -import com.esotericsoftware.kryo.io.{ Input, Output } - /** * This interface is a way of wrapping a value in a marker class * whose class identity is used to control which serialization we @@ -31,312 +28,204 @@ trait Boxed[+K] { def get: K } -class BoxedDefaultSerialization extends KSerializer[Boxed[_]] { - override def write(kryo: Kryo, output: Output, t: Boxed[_]) { - sys.error(s"Kryo should never be used to serialize a boxed instance: $t") - } - override def read(kryo: Kryo, input: Input, t: Class[Boxed[_]]): Boxed[_] = - sys.error("Kryo should never be used to serialize a boxed instance, class: $t") -} - -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed0[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed1[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed2[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed3[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed4[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed5[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed6[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed7[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed8[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed9[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed10[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed11[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed12[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed13[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed14[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed15[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed16[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed17[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed18[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed19[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed20[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed21[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed22[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed23[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed24[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed25[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed26[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed27[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed28[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed29[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed30[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed31[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed32[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed33[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed34[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed35[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed36[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed37[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed38[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed39[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed40[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed41[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed42[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed43[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed44[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed45[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed46[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed47[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed48[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed49[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed50[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed51[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed52[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed53[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed54[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed55[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed56[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed57[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed58[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed59[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed60[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed61[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed62[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed63[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed64[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed65[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed66[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed67[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed68[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed69[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed70[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed71[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed72[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed73[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed74[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed75[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed76[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed77[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed78[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed79[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed80[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed81[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed82[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed83[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed84[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed85[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed86[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed87[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed88[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed89[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed90[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed91[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed92[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed93[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed94[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed95[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed96[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed97[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed98[K](override val get: K) extends Boxed[K] -@DefaultSerializer(classOf[BoxedDefaultSerialization]) class Boxed99[K](override val get: K) extends Boxed[K] case class BoxedOrderedSerialization[K](box: K => Boxed[K], diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Hasher.scala b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/Hasher.scala similarity index 100% rename from scalding-core/src/main/scala/com/twitter/scalding/serialization/Hasher.scala rename to scalding-serialization/src/main/scala/com/twitter/scalding/serialization/Hasher.scala diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/JavaStreamEnrichments.scala b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/JavaStreamEnrichments.scala similarity index 100% rename from scalding-core/src/main/scala/com/twitter/scalding/serialization/JavaStreamEnrichments.scala rename to scalding-serialization/src/main/scala/com/twitter/scalding/serialization/JavaStreamEnrichments.scala diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Laws.scala b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/Laws.scala similarity index 100% rename from scalding-core/src/main/scala/com/twitter/scalding/serialization/Laws.scala rename to scalding-serialization/src/main/scala/com/twitter/scalding/serialization/Laws.scala diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/MurmurHashUtils.scala b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/MurmurHashUtils.scala similarity index 100% rename from scalding-core/src/main/scala/com/twitter/scalding/serialization/MurmurHashUtils.scala rename to scalding-serialization/src/main/scala/com/twitter/scalding/serialization/MurmurHashUtils.scala diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/OrderedSerialization.scala b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/OrderedSerialization.scala similarity index 100% rename from scalding-core/src/main/scala/com/twitter/scalding/serialization/OrderedSerialization.scala rename to scalding-serialization/src/main/scala/com/twitter/scalding/serialization/OrderedSerialization.scala diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/PositionInputStream.scala b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/PositionInputStream.scala similarity index 100% rename from scalding-core/src/main/scala/com/twitter/scalding/serialization/PositionInputStream.scala rename to scalding-serialization/src/main/scala/com/twitter/scalding/serialization/PositionInputStream.scala diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Reader.scala b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/Reader.scala similarity index 100% rename from scalding-core/src/main/scala/com/twitter/scalding/serialization/Reader.scala rename to scalding-serialization/src/main/scala/com/twitter/scalding/serialization/Reader.scala diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization.scala b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/Serialization.scala similarity index 100% rename from scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization.scala rename to scalding-serialization/src/main/scala/com/twitter/scalding/serialization/Serialization.scala diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization2.scala b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/Serialization2.scala similarity index 100% rename from scalding-core/src/main/scala/com/twitter/scalding/serialization/Serialization2.scala rename to scalding-serialization/src/main/scala/com/twitter/scalding/serialization/Serialization2.scala diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/StringOrderedSerialization.scala b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/StringOrderedSerialization.scala similarity index 100% rename from scalding-core/src/main/scala/com/twitter/scalding/serialization/StringOrderedSerialization.scala rename to scalding-serialization/src/main/scala/com/twitter/scalding/serialization/StringOrderedSerialization.scala diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/UnsignedComparisons.scala b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/UnsignedComparisons.scala similarity index 100% rename from scalding-core/src/main/scala/com/twitter/scalding/serialization/UnsignedComparisons.scala rename to scalding-serialization/src/main/scala/com/twitter/scalding/serialization/UnsignedComparisons.scala diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/Writer.scala b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/Writer.scala similarity index 100% rename from scalding-core/src/main/scala/com/twitter/scalding/serialization/Writer.scala rename to scalding-serialization/src/main/scala/com/twitter/scalding/serialization/Writer.scala diff --git a/scalding-core/src/test/scala/com/twitter/scalding/serialization/JavaStreamEnrichmentsProperties.scala b/scalding-serialization/src/test/scala/com/twitter/scalding/serialization/JavaStreamEnrichmentsProperties.scala similarity index 100% rename from scalding-core/src/test/scala/com/twitter/scalding/serialization/JavaStreamEnrichmentsProperties.scala rename to scalding-serialization/src/test/scala/com/twitter/scalding/serialization/JavaStreamEnrichmentsProperties.scala diff --git a/scalding-core/src/test/scala/com/twitter/scalding/serialization/SerializationProperties.scala b/scalding-serialization/src/test/scala/com/twitter/scalding/serialization/SerializationProperties.scala similarity index 100% rename from scalding-core/src/test/scala/com/twitter/scalding/serialization/SerializationProperties.scala rename to scalding-serialization/src/test/scala/com/twitter/scalding/serialization/SerializationProperties.scala diff --git a/scalding-core/src/test/scala/com/twitter/scalding/serialization/UnsignedComparisonLaws.scala b/scalding-serialization/src/test/scala/com/twitter/scalding/serialization/UnsignedComparisonLaws.scala similarity index 100% rename from scalding-core/src/test/scala/com/twitter/scalding/serialization/UnsignedComparisonLaws.scala rename to scalding-serialization/src/test/scala/com/twitter/scalding/serialization/UnsignedComparisonLaws.scala diff --git a/scalding-core/src/test/scala/com/twitter/scalding/serialization/WriterReaderProperties.scala b/scalding-serialization/src/test/scala/com/twitter/scalding/serialization/WriterReaderProperties.scala similarity index 100% rename from scalding-core/src/test/scala/com/twitter/scalding/serialization/WriterReaderProperties.scala rename to scalding-serialization/src/test/scala/com/twitter/scalding/serialization/WriterReaderProperties.scala From e26bed4c11a8bd575d59a738d33be8ee83d56192 Mon Sep 17 00:00:00 2001 From: Joe Nievelt Date: Wed, 13 May 2015 10:06:56 -0700 Subject: [PATCH 167/177] Reverting elephantbird to 4.6 --- project/Build.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/Build.scala b/project/Build.scala index 0faa717b1d..737b4ba60d 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -26,7 +26,7 @@ object ScaldingBuild extends Build { val cascadingAvroVersion = "2.1.2" val chillVersion = "0.6.0" val dfsDatastoresVersion = "1.3.4" - val elephantbirdVersion = "4.7" + val elephantbirdVersion = "4.6" val hadoopLzoVersion = "0.4.16" val hadoopVersion = "1.2.1" val hbaseVersion = "0.94.10" From addcee386617f42f2e7c2e2564d2b4f047c926f8 Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Thu, 14 May 2015 10:34:53 -0700 Subject: [PATCH 168/177] Make sure to throw on Boxed hitting Kryo --- .../scala/com/twitter/scalding/serialization/KryoHadoop.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala index 0a4420262a..5f27104ee2 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala @@ -50,6 +50,8 @@ class KryoHadoop(config: Config) extends KryoInstantiator { newK.register(classOf[com.twitter.algebird.HyperLogLogMonoid], new HLLMonoidSerializer) newK.register(classOf[com.twitter.algebird.Moments], new MomentsSerializer) newK.addDefaultSerializer(classOf[com.twitter.algebird.HLL], new HLLSerializer) + // Don't serialize Boxed instances using Kryo. + newK.addDefaultSerializer(classOf[com.twitter.scalding.serialization.Boxed[_]], new ThrowingSerializer) /** * AdaptiveVector is IndexedSeq, which picks up the chill IndexedSeq serializer * (which is its own bug), force using the fields serializer here From 925291cf3c75b4aa4eecdaa53809d8a8c87f6c68 Mon Sep 17 00:00:00 2001 From: Ruban Monu Date: Thu, 14 May 2015 11:40:59 -0700 Subject: [PATCH 169/177] remove unneeded assert, cache the converter --- .../scalding/commons/source/LzoGenericScheme.scala | 14 +++++++++++--- .../scalding/commons/source/LzoGenericSource.scala | 2 +- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericScheme.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericScheme.scala index 2e3d882d6b..1ed2c6ad59 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericScheme.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericScheme.scala @@ -64,12 +64,20 @@ private[source] object ConfigBinaryConverterProvider { */ private[source] class ConfigBinaryConverterProvider[M] extends BinaryConverterProvider[M] { import ConfigBinaryConverterProvider._ + + private[this] var cached: Option[(String, BinaryConverter[M])] = None + override def getConverter(conf: Configuration): BinaryConverter[M] = { val data = conf.get(ProviderConfKey) require(data != null, s"$ProviderConfKey is not set in configuration") - - val extern = ExternalizerSerializer.inj.invert(data).get - extern.get.asInstanceOf[BinaryConverter[M]] + cached match { + case Some((data, conv)) => conv + case _ => + val extern = ExternalizerSerializer.inj.invert(data).get + val conv = extern.get.asInstanceOf[BinaryConverter[M]] + cached = Some((data, conv)) + conv + } } } diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericSource.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericSource.scala index 49eb70cea2..439726d1c4 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericSource.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericSource.scala @@ -37,6 +37,6 @@ object LzoGenericSource { new LzoGenericSource[T] { override val conv: BinaryConverter[T] = passedConv override val hdfsPaths = paths - override val localPaths = { assert(paths.size == 1, "Cannot use multiple input files on local mode"); paths } + override val localPaths = paths } } From 7ae7270beefa129592b2c95dc37f27a799db4d70 Mon Sep 17 00:00:00 2001 From: Ruban Monu Date: Thu, 14 May 2015 12:03:30 -0700 Subject: [PATCH 170/177] bump elephantbird to 4.8 --- project/Build.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/Build.scala b/project/Build.scala index 737b4ba60d..9e258143dd 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -26,7 +26,7 @@ object ScaldingBuild extends Build { val cascadingAvroVersion = "2.1.2" val chillVersion = "0.6.0" val dfsDatastoresVersion = "1.3.4" - val elephantbirdVersion = "4.6" + val elephantbirdVersion = "4.8" val hadoopLzoVersion = "0.4.16" val hadoopVersion = "1.2.1" val hbaseVersion = "0.94.10" From 98f2528de1b17b7baa1078ff76794376e241282c Mon Sep 17 00:00:00 2001 From: Ian O Connell Date: Fri, 15 May 2015 09:53:26 -0700 Subject: [PATCH 171/177] Adds a test that should break --- project/Build.scala | 3 +- .../scalding/platform/LocalCluster.scala | 1 + .../OrderedSerializationTest.scala | 58 +++++++++++++++++++ 3 files changed, 61 insertions(+), 1 deletion(-) create mode 100644 scalding-hadoop-test/src/test/scala/com/twitter/scalding/ordered_serialization/OrderedSerializationTest.scala diff --git a/project/Build.scala b/project/Build.scala index 2ee09b339c..6e0c9869f8 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -426,13 +426,14 @@ object ScaldingBuild extends Build { libraryDependencies <++= (scalaVersion) { scalaVersion => Seq( ("org.apache.hadoop" % "hadoop-core" % hadoopVersion), ("org.apache.hadoop" % "hadoop-minicluster" % hadoopVersion), + "com.twitter" %% "chill-algebird" % chillVersion, "org.slf4j" % "slf4j-api" % slf4jVersion, "org.slf4j" % "slf4j-log4j12" % slf4jVersion, "org.scalacheck" %% "scalacheck" % scalaCheckVersion, "org.scalatest" %% "scalatest" % scalaTestVersion ) } - ).dependsOn(scaldingCore) + ).dependsOn(scaldingCore, scaldingSerializationMacros % "test") lazy val scaldingMacros = module("macros").settings( libraryDependencies <++= (scalaVersion) { scalaVersion => Seq( diff --git a/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/LocalCluster.scala b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/LocalCluster.scala index 5cd14dcc76..bb83b6aaf1 100644 --- a/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/LocalCluster.scala +++ b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/LocalCluster.scala @@ -119,6 +119,7 @@ class LocalCluster(mutex: Boolean = true) { classOf[com.twitter.scalding.RichDate], classOf[cascading.tuple.TupleException], classOf[com.twitter.chill.Externalizer[_]], + classOf[com.twitter.chill.algebird.AveragedValueSerializer], classOf[com.twitter.algebird.Semigroup[_]], classOf[com.twitter.chill.KryoInstantiator], classOf[org.jgrapht.ext.EdgeNameProvider[_]], diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/ordered_serialization/OrderedSerializationTest.scala b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/ordered_serialization/OrderedSerializationTest.scala new file mode 100644 index 0000000000..5365589d7b --- /dev/null +++ b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/ordered_serialization/OrderedSerializationTest.scala @@ -0,0 +1,58 @@ +package com.twitter.scalding.ordered_serialization + +import com.twitter.scalding._ +import com.twitter.scalding.platform.{ HadoopPlatformJobTest, HadoopPlatformTest } +import com.twitter.scalding.serialization.OrderedSerialization + +import org.scalacheck.{ Arbitrary, Gen } +import org.scalatest.FunSuite + +import scala.language.experimental.macros +import scala.math.Ordering + +object OrderedSerializationTest { + implicit val genASGK = Arbitrary { + for { + ts <- Arbitrary.arbitrary[Long] + b <- Gen.nonEmptyListOf(Gen.alphaNumChar).map (_.mkString) + } yield NestedCaseClass(RichDate(ts), (b, b)) + } + + def sample[T: Arbitrary]: T = Arbitrary.arbitrary[T].sample.get + val data = sample[List[NestedCaseClass]].take(1000) +} + +case class NestedCaseClass(day: RichDate, key: (String, String)) + +class OrderedSerializationTest extends FunSuite with HadoopPlatformTest { + import OrderedSerializationTest._ + test("A test job with a fork and join, had previously not had boxed serializations on all branches") { + val fn = (arg: Args) => new ComplexJob(data, arg) + HadoopPlatformJobTest(fn, cluster) + .arg("output1", "output1") + .arg("output2", "output2") + .sink[String](TypedTsv[String]("output2")) { + actual => () + }.sink[String](TypedTsv[String]("output1")) { x => () } + .run + } +} + +class ComplexJob(input: List[NestedCaseClass], args: Args) extends Job(args) { + implicit def primitiveOrderedBufferSupplier[T]: OrderedSerialization[T] = macro com.twitter.scalding.serialization.macros.impl.OrderedSerializationProviderImpl[T] + + val ds1 = TypedPipe.from(input).map(_ -> 1L).distinct.group + + val ds2 = TypedPipe.from(input).map(_ -> 1L).distinct.group + + ds2 + .keys + .map(s => s.toString) + .write(TypedTsv[String](args("output1"))) + + ds2.join(ds1) + .values + .map(_.toString) + .write(TypedTsv[String](args("output2"))) +} + From f07ddd8bf526de9a611217747587679a4fe0dea0 Mon Sep 17 00:00:00 2001 From: Ian O Connell Date: Fri, 15 May 2015 10:55:18 -0700 Subject: [PATCH 172/177] Fixes applying of boxed serializations This ensures as Pipe's exit our typed pipe eco system we apply all boxed serializations. To do this cleanly and it seemed best to provide a final toPipe method in TypedPipe, and then an abstract asPipe method --- .../com/twitter/scalding/FlowState.scala | 7 ++- .../com/twitter/scalding/RichFlowDef.scala | 4 +- .../scala/com/twitter/scalding/RichPipe.scala | 40 +++++++++++++++++ .../twitter/scalding/typed/CoGrouped.scala | 2 +- .../com/twitter/scalding/typed/Grouped.scala | 44 +++++-------------- .../twitter/scalding/typed/TypedPipe.scala | 23 +++++++--- 6 files changed, 75 insertions(+), 45 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/FlowState.scala b/scalding-core/src/main/scala/com/twitter/scalding/FlowState.scala index 89f306524c..98a4139d76 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/FlowState.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/FlowState.scala @@ -21,9 +21,12 @@ import java.util.WeakHashMap /** * Immutable state that we attach to the Flow using the FlowStateMap */ -case class FlowState(sourceMap: Map[String, Source] = Map.empty) { +case class FlowState(sourceMap: Map[String, Source] = Map.empty, flowBoxes: Set[(String, String)] = Set()) { def addSource(id: String, s: Source): FlowState = - FlowState(sourceMap + (id -> s)) + copy(sourceMap = sourceMap + (id -> s)) + + def addBoxed(k: String, v: String): FlowState = + copy(flowBoxes = flowBoxes + ((k, v))) def getSourceNamed(name: String): Option[Source] = sourceMap.get(name) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/RichFlowDef.scala b/scalding-core/src/main/scala/com/twitter/scalding/RichFlowDef.scala index 49a3404470..4a1f99ef5d 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/RichFlowDef.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/RichFlowDef.scala @@ -84,7 +84,7 @@ class RichFlowDef(val fd: FlowDef) { .foreach { oFS => FlowStateMap.mutate(fd) { current => // overwrite the items from o with current - (FlowState(oFS.sourceMap ++ current.sourceMap), ()) + (current.copy(sourceMap = oFS.sourceMap ++ current.sourceMap), ()) } } } @@ -147,7 +147,7 @@ class RichFlowDef(val fd: FlowDef) { if (headNames(name)) newfs + kv else newfs } - FlowStateMap.mutate(newFd) { _ => (FlowState(subFlowState), ()) } + FlowStateMap.mutate(newFd) { oldFS => (oldFS.copy(sourceMap = subFlowState), ()) } } newFd } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/RichPipe.scala b/scalding-core/src/main/scala/com/twitter/scalding/RichPipe.scala index 62ad3c3ce0..0b14ba594c 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/RichPipe.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/RichPipe.scala @@ -24,6 +24,7 @@ import cascading.tuple._ import scala.util.Random import java.util.concurrent.atomic.AtomicInteger +import scala.collection.immutable.Queue object RichPipe extends java.io.Serializable { private val nextPipe = new AtomicInteger(-1) @@ -660,6 +661,45 @@ class RichPipe(val pipe: Pipe) extends java.io.Serializable with JoinAlgorithms .flatten .toSet + /** + * This finds all the boxed serializations stored in the flow state map for this + * flowdef. We then find all the pipes back in the DAG from this pipe and apply + * those serializations. + */ + private[scalding] def applyBoxedSerializations(flowDef: FlowDef): Pipe = { + case class ToVisit[T](queue: Queue[T], inQueue: Set[T]) { + def maybeAdd(t: T): ToVisit[T] = if (inQueue(t)) this else { + ToVisit(queue :+ t, inQueue + t) + } + def next: Option[(T, ToVisit[T])] = + if (inQueue.isEmpty) None + else Some((queue.head, ToVisit(queue.tail, inQueue - queue.head))) + } + + @annotation.tailrec + def go(p: Pipe, visited: Set[Pipe], toVisit: ToVisit[Pipe]): Set[Pipe] = { + val notSeen: Set[Pipe] = p.getPrevious.filter(i => !visited.contains(i)).toSet + val nextVisited: Set[Pipe] = visited + p + val nextToVisit = notSeen.foldLeft(toVisit) { case (prev, n) => prev.maybeAdd(n) } + + nextToVisit.next match { + case Some((h, innerNextToVisit)) => go(h, nextVisited, innerNextToVisit) + case _ => nextVisited + } + } + val allPipes = go(pipe, Set[Pipe](), ToVisit[Pipe](Queue.empty, Set.empty)) + + FlowStateMap.get(flowDef).foreach { fstm => + fstm.flowBoxes.foreach { + case (k, v) => + allPipes.foreach { p => + p.getStepConfigDef().setProperty(k, v) + } + } + } + pipe + } + } /** diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/CoGrouped.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/CoGrouped.scala index 9625ce180b..91400fd8be 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/CoGrouped.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/CoGrouped.scala @@ -204,7 +204,7 @@ trait CoGrouped[K, +R] extends KeyedListLike[K, R, CoGrouped] with CoGroupable[K val ord = keyOrdering TypedPipeFactory({ (flowDef, mode) => - val newPipe = Grouped.maybeBox[K, Any](ord) { (tupset, ordKeyField) => + val newPipe = Grouped.maybeBox[K, Any](ord, flowDef) { (tupset, ordKeyField) => if (firstCount == inputs.size) { /** * This is a self-join diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/Grouped.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/Grouped.scala index 4e821417bd..d98d4cf7c8 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/Grouped.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/Grouped.scala @@ -93,46 +93,24 @@ object Grouped { * If we are using OrderedComparable, we need to box the key * to prevent other serializers from handling the key */ - def maybeBox[K, V](ord: Ordering[K])(op: (TupleSetter[(K, V)], Fields) => Pipe): Pipe = ord match { + def maybeBox[K, V](ord: Ordering[K], flowDef: FlowDef)(op: (TupleSetter[(K, V)], Fields) => Pipe): Pipe = ord match { case ordser: OrderedSerialization[K] => val (boxfn, cls) = Boxed.next[K] - val ts = tup2Setter[(Boxed[K], V)].contraMap { kv1: (K, V) => (boxfn(kv1._1), kv1._2) } val boxordSer = BoxedOrderedSerialization(boxfn, ordser) - val keyF = new Fields("key") - keyF.setComparator("key", new CascadingBinaryComparator(boxordSer)) - val pipe = op(ts, keyF) - - case class ToVisit[T](queue: Queue[T], inQueue: Set[T]) { - def maybeAdd(t: T): ToVisit[T] = if (inQueue(t)) this else { - ToVisit(queue :+ t, inQueue + t) - } - def next: Option[(T, ToVisit[T])] = - if (inQueue.isEmpty) None - else Some((queue.head, ToVisit(queue.tail, inQueue - queue.head))) - } - - @annotation.tailrec - def go(p: Pipe, visited: Set[Pipe], toVisit: ToVisit[Pipe]): Set[Pipe] = { - val notSeen: Set[Pipe] = p.getPrevious.filter(i => !visited.contains(i)).toSet - val nextVisited: Set[Pipe] = visited + p - val nextToVisit = notSeen.foldLeft(toVisit) { case (prev, n) => prev.maybeAdd(n) } - - nextToVisit.next match { - case Some((h, innerNextToVisit)) => go(h, nextVisited, innerNextToVisit) - case _ => nextVisited - } - } - - val allPipes = go(pipe, Set[Pipe](), ToVisit[Pipe](Queue.empty, Set.empty)) WrappedSerialization.rawSetBinary(List((cls, boxordSer)), { - case (k, v) => - allPipes.foreach { p => - p.getStepConfigDef().setProperty(k + cls, v) + case (k: String, v: String) => + FlowStateMap.mutate(flowDef) { st => + val newSt = st.addBoxed(k + cls, v) + (newSt, ()) } }) - pipe + + val ts = tup2Setter[(Boxed[K], V)].contraMap { kv1: (K, V) => (boxfn(kv1._1), kv1._2) } + val keyF = new Fields("key") + keyF.setComparator("key", new CascadingBinaryComparator(boxordSer)) + op(ts, keyF) case _ => val ts = tup2Setter[(K, V)] val keyF = Field.singleOrdered("key")(ord) @@ -204,7 +182,7 @@ sealed trait ReduceStep[K, V1] extends KeyedPipe[K] { // make the pipe and group it, only here because it is common protected def groupOp[V2](gb: GroupBuilder => GroupBuilder): TypedPipe[(K, V2)] = TypedPipeFactory({ (fd, mode) => - val pipe = Grouped.maybeBox[K, V1](keyOrdering) { (tupleSetter, fields) => + val pipe = Grouped.maybeBox[K, V1](keyOrdering, fd) { (tupleSetter, fields) => mapped .toPipe(Grouped.kvFields)(fd, mode, tupleSetter) .groupBy(fields)(gb) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipe.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipe.scala index dcbc1474eb..6d95acabed 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipe.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipe.scala @@ -142,7 +142,16 @@ trait TypedPipe[+T] extends Serializable { * Fields API or with Cascading code. * Avoid this if possible. Prefer to write to TypedSink. */ - def toPipe[U >: T](fieldNames: Fields)(implicit flowDef: FlowDef, mode: Mode, setter: TupleSetter[U]): Pipe + final def toPipe[U >: T](fieldNames: Fields)(implicit flowDef: FlowDef, mode: Mode, setter: TupleSetter[U]): Pipe = { + import Dsl._ + // Ensure we hook into all pipes coming out of the typed API to apply the boxed serializations + asPipe[U](fieldNames).applyBoxedSerializations(flowDef) + } + + /** + * Provide the internal implementation to get from a typed pipe to a cascading Pipe + */ + protected def asPipe[U >: T](fieldNames: Fields)(implicit flowDef: FlowDef, mode: Mode, setter: TupleSetter[U]): Pipe ///////////////////////////////////////////// // @@ -707,7 +716,7 @@ final case object EmptyTypedPipe extends TypedPipe[Nothing] { override def ++[U >: Nothing](other: TypedPipe[U]): TypedPipe[U] = other - override def toPipe[U >: Nothing](fieldNames: Fields)(implicit fd: FlowDef, mode: Mode, setter: TupleSetter[U]): Pipe = + override def asPipe[U >: Nothing](fieldNames: Fields)(implicit fd: FlowDef, mode: Mode, setter: TupleSetter[U]): Pipe = IterableSource(Iterable.empty, fieldNames)(setter, singleConverter[U]).read(fd, mode) override def toIterableExecution: Execution[Iterable[Nothing]] = Execution.from(Iterable.empty) @@ -793,7 +802,7 @@ final case class IterablePipe[T](iterable: Iterable[T]) extends TypedPipe[T] { }) } - override def toPipe[U >: T](fieldNames: Fields)(implicit flowDef: FlowDef, mode: Mode, setter: TupleSetter[U]): Pipe = + override def asPipe[U >: T](fieldNames: Fields)(implicit flowDef: FlowDef, mode: Mode, setter: TupleSetter[U]): Pipe = // It is slightly more efficient to use this rather than toSourcePipe.toPipe(fieldNames) IterableSource[U](iterable, fieldNames)(setter, singleConverter[U]).read(flowDef, mode) @@ -851,7 +860,7 @@ class TypedPipeFactory[T] private (@transient val next: NoStackAndThen[(FlowDef, override def sumByLocalKeys[K, V](implicit ev: T <:< (K, V), sg: Semigroup[V]) = andThen(_.sumByLocalKeys[K, V]) - override def toPipe[U >: T](fieldNames: Fields)(implicit flowDef: FlowDef, mode: Mode, setter: TupleSetter[U]) = + override def asPipe[U >: T](fieldNames: Fields)(implicit flowDef: FlowDef, mode: Mode, setter: TupleSetter[U]) = // unwrap in a loop, without recursing unwrap(this).toPipe[U](fieldNames)(flowDef, mode, setter) @@ -926,7 +935,7 @@ class TypedPipeInst[T] private[scalding] (@transient inpipe: Pipe, * This approach is more efficient than untyped scalding because we * don't use TupleConverters/Setters after each map. */ - override def toPipe[U >: T](fieldNames: Fields)(implicit flowDef: FlowDef, m: Mode, setter: TupleSetter[U]): Pipe = { + override def asPipe[U >: T](fieldNames: Fields)(implicit flowDef: FlowDef, m: Mode, setter: TupleSetter[U]): Pipe = { import Dsl.flowDefToRichFlowDef checkMode(m) flowDef.mergeFrom(localFlowDef) @@ -993,7 +1002,7 @@ final case class MergedTypedPipe[T](left: TypedPipe[T], right: TypedPipe[T]) ext case Nil => acc } - override def toPipe[U >: T](fieldNames: Fields)(implicit flowDef: FlowDef, mode: Mode, setter: TupleSetter[U]): Pipe = { + override def asPipe[U >: T](fieldNames: Fields)(implicit flowDef: FlowDef, mode: Mode, setter: TupleSetter[U]): Pipe = { /* * Cascading can't handle duplicate pipes in merges. What we do here is see if any pipe appears * multiple times and if it does we can do self merges using flatMap. @@ -1026,7 +1035,7 @@ final case class MergedTypedPipe[T](left: TypedPipe[T], right: TypedPipe[T]) ext } class WithOnComplete[T](typedPipe: TypedPipe[T], fn: () => Unit) extends TypedPipe[T] { - override def toPipe[U >: T](fieldNames: Fields)(implicit flowDef: FlowDef, mode: Mode, setter: TupleSetter[U]) = { + override def asPipe[U >: T](fieldNames: Fields)(implicit flowDef: FlowDef, mode: Mode, setter: TupleSetter[U]) = { val pipe = typedPipe.toPipe[U](fieldNames)(flowDef, mode, setter) new Each(pipe, Fields.ALL, new CleanupIdentityFunction(fn), Fields.REPLACE) } From 1b09e19920fd5e91ce75b16a6b25b4afce31f1d0 Mon Sep 17 00:00:00 2001 From: Joe Nievelt Date: Thu, 14 May 2015 13:18:29 -0700 Subject: [PATCH 173/177] Prepare for release of 0.14.0 --- CHANGES.md | 46 +++++++++++++++++++ README.md | 2 +- project/Build.scala | 2 +- .../src/main/scala/com/twitter/package.scala | 2 +- .../ExecutionTutorial.scala | 2 +- version.sbt | 2 +- 6 files changed, 51 insertions(+), 5 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index a6bdf9a5d0..9d066bc5b2 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,5 +1,51 @@ # Scalding # +### Version 0.14.0 ### +* add .unit to Execution object #1189 +* Override hashCode for Args #1190 +* Put a value in a exception message #1191 +* Add an exclusiveUpper method to DateRange #1194 +* Covert LzoTextDelimited to Cascading scheme. #1179 +* Remove Travis IRC notifications #1200 +* add LookupJoin and LookupJoinTest changes from summingbird #1199 +* Add a new ExecutionApp tutorial #1196 +* Move main simple example to be the typed API, and put the .'s at the sta... #1193 +* Add Execution.withArgs #1205 +* Config/Cascading updater #1197 +* Remove algebird serializers #1206 +* remove warnings in CumulativeSum #1215 +* Implicit execution context / easier switching between modes #1113 +* add row l1 normalize #1214 +* provide Args as an implicit val #1219 +* call sourceConfInit when reading from taps in local mode #1228 +* Add distinctCount and distinctValues helper methods to KeyedList. #1232 +* import hygiene: remove unused imports and remove JavaConversions use #1239 +* Swap hash and filename for filename-extension-sensitive code #1243 +* Remove more unused imports #1240 +* Provide useHdfsLocalMode for an easy switch to mapreduce local mode #1244 +* upgrade scalacheck and scalatest #1246 +* Optimize string and (hopefully) number comparisons a bit #1241 +* Note the active FlowProcess for Joiners #1235 +* Make sure Executions are executed at most once #1253 +* Fix Config.getUniqueIDs #1254 +* Add MustHasReducers trait. #1252 +* Make sure the EvalCache thread isDaemon #1255 +* Use non-regex split function #1251 +* make InputSizeReducerEstimator work for any CompositeTap #1256 +* TimePathedSource helper methods #1257 +* Fix for reducer estimation not working correctly if withReducers is set to 1 reducer #1263 +* Add make(dest) to TypedPipe #1217 +* Fix SimpleDateFormat caching by default #1265 +* upgrade sbt and sbt launcher script #1270 +* Add TypedPipeDiff for comparing typed pipes #1266 +* Change separator from \1 to \u0001 #1271 +* Disable reducer estimation for map-only steps #1276 +* Local sources support multiple paths #1275 +* fix the spelling of the cumulativeSumTest file #1281 +* Hydrate both sides of sampledCounts in skewJoinWithSmaller #1278 +* Bijection 0.8.0, algebird 0.10.0, chill 0.6.0, scala 2.10.5 #1287 +* Remove some deprecated items #1288 + ### Version 0.13.1 ### * Back out 4 changes to be binary compatible: https://github.com/twitter/scalding/pull/1187 * Use java.util.Random instead of scala.util.Random: https://github.com/twitter/scalding/pull/1186 diff --git a/README.md b/README.md index fc1b882d4d..595d146b82 100644 --- a/README.md +++ b/README.md @@ -4,7 +4,7 @@ Scalding is a Scala library that makes it easy to specify Hadoop MapReduce jobs. ![Scalding Logo](https://raw.github.com/twitter/scalding/develop/logo/scalding.png) -Current version: `0.13.1` +Current version: `0.14.0` ## Word Count diff --git a/project/Build.scala b/project/Build.scala index 2ee09b339c..ace352f4c8 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -226,7 +226,7 @@ object ScaldingBuild extends Build { Some(subProj) .filterNot(unreleasedModules.contains(_)) .map { - s => "com.twitter" % ("scalding-" + s + "_2.10") % "0.13.0" + s => "com.twitter" % ("scalding-" + s + "_2.10") % "0.14.0" } def module(name: String) = { diff --git a/scalding-core/src/main/scala/com/twitter/package.scala b/scalding-core/src/main/scala/com/twitter/package.scala index f712d2d353..45904fe714 100644 --- a/scalding-core/src/main/scala/com/twitter/package.scala +++ b/scalding-core/src/main/scala/com/twitter/package.scala @@ -34,7 +34,7 @@ package object scalding { /** * Make sure this is in sync with version.sbt */ - val scaldingVersion: String = "0.13.1" + val scaldingVersion: String = "0.14.0" object RichPathFilter { implicit def toRichPathFilter(f: PathFilter) = new RichPathFilter(f) diff --git a/tutorial/execution-tutorial/ExecutionTutorial.scala b/tutorial/execution-tutorial/ExecutionTutorial.scala index f27ee2da8e..b4b57345ea 100644 --- a/tutorial/execution-tutorial/ExecutionTutorial.scala +++ b/tutorial/execution-tutorial/ExecutionTutorial.scala @@ -30,7 +30,7 @@ To test it, first build the assembly jar from root directory: ./sbt execution-tutorial/assembly Run: - scala -classpath tutorial/execution-tutorial/target/execution-tutorial-assembly-0.13.1.jar \ + scala -classpath tutorial/execution-tutorial/target/execution-tutorial-assembly-0.14.0.jar \ com.twitter.scalding.tutorial.MyExecJob --local \ --input tutorial/data/hello.txt \ --output tutorial/data/execution_output.txt diff --git a/version.sbt b/version.sbt index beb9953b43..501eb5530c 100644 --- a/version.sbt +++ b/version.sbt @@ -1 +1 @@ -version in ThisBuild := "0.13.1" +version in ThisBuild := "0.14.0" From 6f3cc9290bc5e9428f997f77abf804953f90d2ae Mon Sep 17 00:00:00 2001 From: Ian O Connell Date: Fri, 15 May 2015 13:51:48 -0700 Subject: [PATCH 174/177] Review comments --- .../src/main/scala/com/twitter/scalding/FlowState.scala | 6 +++--- .../src/main/scala/com/twitter/scalding/Job.scala | 1 - .../src/main/scala/com/twitter/scalding/RichPipe.scala | 4 ++-- .../main/scala/com/twitter/scalding/typed/Grouped.scala | 4 ++-- .../main/scala/com/twitter/scalding/typed/TypedPipe.scala | 4 ++-- .../ordered_serialization/OrderedSerializationTest.scala | 8 +++++--- 6 files changed, 14 insertions(+), 13 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/FlowState.scala b/scalding-core/src/main/scala/com/twitter/scalding/FlowState.scala index 98a4139d76..85b6b74718 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/FlowState.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/FlowState.scala @@ -21,12 +21,12 @@ import java.util.WeakHashMap /** * Immutable state that we attach to the Flow using the FlowStateMap */ -case class FlowState(sourceMap: Map[String, Source] = Map.empty, flowBoxes: Set[(String, String)] = Set()) { +case class FlowState(sourceMap: Map[String, Source] = Map.empty, flowConfigUpdates: Set[(String, String)] = Set()) { def addSource(id: String, s: Source): FlowState = copy(sourceMap = sourceMap + (id -> s)) - def addBoxed(k: String, v: String): FlowState = - copy(flowBoxes = flowBoxes + ((k, v))) + def addConfigSetting(k: String, v: String): FlowState = + copy(flowConfigUpdates = flowConfigUpdates + ((k, v))) def getSourceNamed(name: String): Option[Source] = sourceMap.get(name) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Job.scala b/scalding-core/src/main/scala/com/twitter/scalding/Job.scala index 14ac2ce0a1..d4363f7af9 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Job.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Job.scala @@ -470,7 +470,6 @@ abstract class ExecutionJob[+T](args: Args) extends Job(args) { } } - /* * 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 diff --git a/scalding-core/src/main/scala/com/twitter/scalding/RichPipe.scala b/scalding-core/src/main/scala/com/twitter/scalding/RichPipe.scala index 0b14ba594c..12f8e2c10f 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/RichPipe.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/RichPipe.scala @@ -666,7 +666,7 @@ class RichPipe(val pipe: Pipe) extends java.io.Serializable with JoinAlgorithms * flowdef. We then find all the pipes back in the DAG from this pipe and apply * those serializations. */ - private[scalding] def applyBoxedSerializations(flowDef: FlowDef): Pipe = { + private[scalding] def applyFlowConfigProperties(flowDef: FlowDef): Pipe = { case class ToVisit[T](queue: Queue[T], inQueue: Set[T]) { def maybeAdd(t: T): ToVisit[T] = if (inQueue(t)) this else { ToVisit(queue :+ t, inQueue + t) @@ -690,7 +690,7 @@ class RichPipe(val pipe: Pipe) extends java.io.Serializable with JoinAlgorithms val allPipes = go(pipe, Set[Pipe](), ToVisit[Pipe](Queue.empty, Set.empty)) FlowStateMap.get(flowDef).foreach { fstm => - fstm.flowBoxes.foreach { + fstm.flowConfigUpdates.foreach { case (k, v) => allPipes.foreach { p => p.getStepConfigDef().setProperty(k, v) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/Grouped.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/Grouped.scala index d98d4cf7c8..f60184b0ac 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/Grouped.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/Grouped.scala @@ -93,7 +93,7 @@ object Grouped { * If we are using OrderedComparable, we need to box the key * to prevent other serializers from handling the key */ - def maybeBox[K, V](ord: Ordering[K], flowDef: FlowDef)(op: (TupleSetter[(K, V)], Fields) => Pipe): Pipe = ord match { + private[scalding] def maybeBox[K, V](ord: Ordering[K], flowDef: FlowDef)(op: (TupleSetter[(K, V)], Fields) => Pipe): Pipe = ord match { case ordser: OrderedSerialization[K] => val (boxfn, cls) = Boxed.next[K] val boxordSer = BoxedOrderedSerialization(boxfn, ordser) @@ -102,7 +102,7 @@ object Grouped { { case (k: String, v: String) => FlowStateMap.mutate(flowDef) { st => - val newSt = st.addBoxed(k + cls, v) + val newSt = st.addConfigSetting(k + cls, v) (newSt, ()) } }) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipe.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipe.scala index 6d95acabed..bbaebf555f 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipe.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipe.scala @@ -144,8 +144,8 @@ trait TypedPipe[+T] extends Serializable { */ final def toPipe[U >: T](fieldNames: Fields)(implicit flowDef: FlowDef, mode: Mode, setter: TupleSetter[U]): Pipe = { import Dsl._ - // Ensure we hook into all pipes coming out of the typed API to apply the boxed serializations - asPipe[U](fieldNames).applyBoxedSerializations(flowDef) + // Ensure we hook into all pipes coming out of the typed API to apply the FlowState's properties on their pipes + asPipe[U](fieldNames).applyFlowConfigProperties(flowDef) } /** diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/ordered_serialization/OrderedSerializationTest.scala b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/ordered_serialization/OrderedSerializationTest.scala index 5365589d7b..e06872c470 100644 --- a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/ordered_serialization/OrderedSerializationTest.scala +++ b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/ordered_serialization/OrderedSerializationTest.scala @@ -31,9 +31,11 @@ class OrderedSerializationTest extends FunSuite with HadoopPlatformTest { HadoopPlatformJobTest(fn, cluster) .arg("output1", "output1") .arg("output2", "output2") - .sink[String](TypedTsv[String]("output2")) { - actual => () - }.sink[String](TypedTsv[String]("output1")) { x => () } + // Here we are just testing that we hit no exceptions in the course of this run + // the previous issue would have caused OOM or other exceptions. If we get to the end + // then we are good. + .sink[String](TypedTsv[String]("output2")) { x => () } + .sink[String](TypedTsv[String]("output1")) { x => () } .run } } From 2ec95d100ab6c35eb35d3546cdc1d3d11d73b189 Mon Sep 17 00:00:00 2001 From: Ruban Monu Date: Tue, 19 May 2015 16:32:33 -0700 Subject: [PATCH 175/177] fix cached converter, drop classtag, add serialization test --- .../commons/source/LzoGenericScheme.scala | 16 +++++++--- .../commons/source/LzoGenericSource.scala | 8 +++-- .../commons/source/LzoGenericSourceSpec.scala | 30 +++++++++++++++++++ 3 files changed, 47 insertions(+), 7 deletions(-) create mode 100644 scalding-commons/src/test/scala/com/twitter/scalding/commons/source/LzoGenericSourceSpec.scala diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericScheme.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericScheme.scala index 1ed2c6ad59..aad3b696c3 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericScheme.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericScheme.scala @@ -71,7 +71,7 @@ private[source] class ConfigBinaryConverterProvider[M] extends BinaryConverterPr val data = conf.get(ProviderConfKey) require(data != null, s"$ProviderConfKey is not set in configuration") cached match { - case Some((data, conv)) => conv + case Some((d, conv)) if d == data => conv case _ => val extern = ExternalizerSerializer.inj.invert(data).get val conv = extern.get.asInstanceOf[BinaryConverter[M]] @@ -81,11 +81,19 @@ private[source] class ConfigBinaryConverterProvider[M] extends BinaryConverterPr } } +object LzoGenericScheme { + def apply[M: ClassTag](conv: BinaryConverter[M]): LzoGenericScheme[M] = + new LzoGenericScheme(conv, implicitly[ClassTag[M]].runtimeClass.asInstanceOf[Class[M]]) + + def apply[M](conv: BinaryConverter[M], clazz: Class[M]): LzoGenericScheme[M] = + new LzoGenericScheme(conv, clazz) +} + /** * Generic scheme for data stored as lzo-compressed protobuf messages. * Serialization is performed using the supplied BinaryConverter. */ -class LzoGenericScheme[M: ClassTag](@transient conv: BinaryConverter[M]) extends LzoBinaryScheme[M, GenericWritable[M]] { +class LzoGenericScheme[M](@transient conv: BinaryConverter[M], clazz: Class[M]) extends LzoBinaryScheme[M, GenericWritable[M]] { override protected def prepareBinaryWritable(): GenericWritable[M] = new GenericWritable(conv) @@ -103,7 +111,7 @@ class LzoGenericScheme[M: ClassTag](@transient conv: BinaryConverter[M]) extends conf.set(ConfigBinaryConverterProvider.ProviderConfKey, ExternalizerSerializer.inj(extern)) - MultiInputFormat.setClassConf(implicitly[ClassTag[M]].runtimeClass, conf) + MultiInputFormat.setClassConf(clazz, conf) MultiInputFormat.setGenericConverterClassConf(classOf[ConfigBinaryConverterProvider[_]], conf) DelegateCombineFileInputFormat.setDelegateInputFormat(conf, classOf[MultiInputFormat[_]]) @@ -119,7 +127,7 @@ class LzoGenericScheme[M: ClassTag](@transient conv: BinaryConverter[M]) extends case e: Exception => throw new RuntimeException("Unable to roundtrip the BinaryConverter in the Externalizer.", e) } - LzoGenericBlockOutputFormat.setClassConf(implicitly[ClassTag[M]].runtimeClass, conf) + LzoGenericBlockOutputFormat.setClassConf(clazz, conf) conf.set(ConfigBinaryConverterProvider.ProviderConfKey, ExternalizerSerializer.inj(extern)) LzoGenericBlockOutputFormat.setGenericConverterClassConf(classOf[ConfigBinaryConverterProvider[_]], conf) DeprecatedOutputFormatWrapper.setOutputFormat(classOf[LzoGenericBlockOutputFormat[_]], conf) diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericSource.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericSource.scala index 439726d1c4..c75448b90e 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericSource.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericSource.scala @@ -26,16 +26,18 @@ import cascading.scheme.Scheme /** * Generic source with an underlying GenericScheme that uses the supplied BinaryConverter. */ -abstract class LzoGenericSource[T: ClassTag] extends FileSource with SingleMappable[T] with TypedSink[T] with LocalTapSource { +abstract class LzoGenericSource[T] extends FileSource with SingleMappable[T] with TypedSink[T] with LocalTapSource { + def clazz: Class[T] def conv: BinaryConverter[T] override def setter[U <: T] = TupleSetter.asSubSetter[T, U](TupleSetter.singleSetter[T]) - override def hdfsScheme = HadoopSchemeInstance((new LzoGenericScheme[T](conv)).asInstanceOf[Scheme[_, _, _, _, _]]) + override def hdfsScheme = HadoopSchemeInstance(LzoGenericScheme[T](conv, clazz).asInstanceOf[Scheme[_, _, _, _, _]]) } object LzoGenericSource { - def apply[T: ClassTag](passedConv: BinaryConverter[T], paths: String*) = + def apply[T](passedConv: BinaryConverter[T], passedClass: Class[T], paths: String*) = new LzoGenericSource[T] { override val conv: BinaryConverter[T] = passedConv + override val clazz = passedClass override val hdfsPaths = paths override val localPaths = paths } diff --git a/scalding-commons/src/test/scala/com/twitter/scalding/commons/source/LzoGenericSourceSpec.scala b/scalding-commons/src/test/scala/com/twitter/scalding/commons/source/LzoGenericSourceSpec.scala new file mode 100644 index 0000000000..6b87a9edc5 --- /dev/null +++ b/scalding-commons/src/test/scala/com/twitter/scalding/commons/source/LzoGenericSourceSpec.scala @@ -0,0 +1,30 @@ +/* +Copyright 2015 Twitter, Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + +http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ +package com.twitter.scalding.commons.source + +import com.twitter.bijection.JavaSerializationInjection +import org.scalatest.{ Matchers, WordSpec } +import scala.util.Success + +class LzoGenericSourceSpec extends WordSpec with Matchers { + "LzoGenericScheme" should { + "be serializable" in { + val scheme = LzoGenericScheme[Array[Byte]](IdentityBinaryConverter) + val inj = JavaSerializationInjection[LzoGenericScheme[Array[Byte]]] + inj.invert(inj.apply(scheme)) shouldBe Success(scheme) + } + } +} From b940f2a04d7bb06ff5d7a0702101555712a8e7c2 Mon Sep 17 00:00:00 2001 From: Ruban Monu Date: Wed, 20 May 2015 10:51:00 -0700 Subject: [PATCH 176/177] Add serialization modules to aggregate list --- project/Build.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/project/Build.scala b/project/Build.scala index ae4d7ca59d..8b33f920ff 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -206,7 +206,9 @@ object ScaldingBuild extends Build { scaldingHadoopTest, scaldingMacros, maple, - executionTutorial + executionTutorial, + scaldingSerialization, + scaldingSerializationMacros ) lazy val formattingPreferences = { From df045987899d5dc18803180ce0793e1caa5f92eb Mon Sep 17 00:00:00 2001 From: Katya Gonina Date: Thu, 21 May 2015 16:10:38 -0700 Subject: [PATCH 177/177] bumping version to 0.15.0; algebird to 0.10.1 --- CHANGES.md | 6 ++++++ README.md | 2 +- project/Build.scala | 4 ++-- scalding-core/src/main/scala/com/twitter/package.scala | 2 +- tutorial/execution-tutorial/ExecutionTutorial.scala | 2 +- version.sbt | 2 +- 6 files changed, 12 insertions(+), 6 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index 9d066bc5b2..c585238a4b 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,5 +1,11 @@ # Scalding # +### Version 0.15.0 ### +* Move OrderedSerialization into zero-dep scalding-serialization module #1289 +* bump elephantbird to 4.8 #1292 +* Fix OrderedSerialization for some forked graphs #1293 +* Add serialization modules to aggregate list #1298 + ### Version 0.14.0 ### * add .unit to Execution object #1189 * Override hashCode for Args #1190 diff --git a/README.md b/README.md index 595d146b82..39681da214 100644 --- a/README.md +++ b/README.md @@ -4,7 +4,7 @@ Scalding is a Scala library that makes it easy to specify Hadoop MapReduce jobs. ![Scalding Logo](https://raw.github.com/twitter/scalding/develop/logo/scalding.png) -Current version: `0.14.0` +Current version: `0.15.0` ## Word Count diff --git a/project/Build.scala b/project/Build.scala index 8b33f920ff..6d40f82720 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -20,7 +20,7 @@ object ScaldingBuild extends Build { } def isScala210x(scalaVersion: String) = scalaBinaryVersion(scalaVersion) == "2.10" - val algebirdVersion = "0.10.0" + val algebirdVersion = "0.10.1" val avroVersion = "1.7.4" val bijectionVersion = "0.8.0" val cascadingAvroVersion = "2.1.2" @@ -228,7 +228,7 @@ object ScaldingBuild extends Build { Some(subProj) .filterNot(unreleasedModules.contains(_)) .map { - s => "com.twitter" % ("scalding-" + s + "_2.10") % "0.14.0" + s => "com.twitter" % ("scalding-" + s + "_2.10") % "0.15.0" } def module(name: String) = { diff --git a/scalding-core/src/main/scala/com/twitter/package.scala b/scalding-core/src/main/scala/com/twitter/package.scala index 45904fe714..a9b0009608 100644 --- a/scalding-core/src/main/scala/com/twitter/package.scala +++ b/scalding-core/src/main/scala/com/twitter/package.scala @@ -34,7 +34,7 @@ package object scalding { /** * Make sure this is in sync with version.sbt */ - val scaldingVersion: String = "0.14.0" + val scaldingVersion: String = "0.15.0" object RichPathFilter { implicit def toRichPathFilter(f: PathFilter) = new RichPathFilter(f) diff --git a/tutorial/execution-tutorial/ExecutionTutorial.scala b/tutorial/execution-tutorial/ExecutionTutorial.scala index b4b57345ea..2cb786d077 100644 --- a/tutorial/execution-tutorial/ExecutionTutorial.scala +++ b/tutorial/execution-tutorial/ExecutionTutorial.scala @@ -30,7 +30,7 @@ To test it, first build the assembly jar from root directory: ./sbt execution-tutorial/assembly Run: - scala -classpath tutorial/execution-tutorial/target/execution-tutorial-assembly-0.14.0.jar \ + scala -classpath tutorial/execution-tutorial/target/execution-tutorial-assembly-0.15.0.jar \ com.twitter.scalding.tutorial.MyExecJob --local \ --input tutorial/data/hello.txt \ --output tutorial/data/execution_output.txt diff --git a/version.sbt b/version.sbt index 501eb5530c..b2771948a7 100644 --- a/version.sbt +++ b/version.sbt @@ -1 +1 @@ -version in ThisBuild := "0.14.0" +version in ThisBuild := "0.15.0"