diff --git a/.gitignore b/.gitignore index 8862a09c9e..efb48fa0af 100644 --- a/.gitignore +++ b/.gitignore @@ -32,3 +32,4 @@ tutorial/data/tmp3.tsv tutorial/data/jsonoutput0.tsv tutorial/data/avrooutput0.avro .scalding_repl +scalding-hadoop-test/NOTICE diff --git a/.travis.yml b/.travis.yml index 20ac631d63..2fb94e5371 100644 --- a/.travis.yml +++ b/.travis.yml @@ -1,12 +1,3 @@ -# Generate base tests::: -# for s in 2.10.4 2.9.3; do -# for t in `ls -d scalding-*`; do -# echo " - scala: $s" -# echo " env: BUILD=\"base\" TEST_TARGET=\"$t\"" -# echo " script: \"scripts/run_test.sh\"" -# echo "" -# done -# done language: scala jdk: oraclejdk7 sudo: false @@ -17,133 +8,90 @@ matrix: include: #BASE TESTS - scala: 2.10.4 - env: BUILD="base" TEST_TARGET="scalding-args" + env: BUILD="base" TEST_TARGET="scalding-args scalding-date" script: "scripts/run_test.sh" - - scala: 2.10.4 - env: BUILD="base" TEST_TARGET="scalding-avro" + - scala: 2.11.4 + env: BUILD="base" TEST_TARGET="scalding-args scalding-date" script: "scripts/run_test.sh" - scala: 2.10.4 - env: BUILD="base" TEST_TARGET="scalding-commons" + env: BUILD="base" TEST_TARGET="scalding-avro scalding-hraven scalding-commons" + script: "scripts/run_test.sh" + + - scala: 2.11.4 + env: BUILD="base" TEST_TARGET="scalding-avro scalding-hraven scalding-commons" script: "scripts/run_test.sh" - scala: 2.10.4 env: BUILD="base" TEST_TARGET="scalding-core" script: "scripts/run_test.sh" - - scala: 2.10.4 - env: BUILD="base" TEST_TARGET="scalding-date" + - scala: 2.11.4 + env: BUILD="base" TEST_TARGET="scalding-core" script: "scripts/run_test.sh" - scala: 2.10.4 env: BUILD="base" TEST_TARGET="scalding-hadoop-test" script: "scripts/run_test.sh" - - scala: 2.10.4 - env: BUILD="base" TEST_TARGET="scalding-hraven" + - scala: 2.11.4 + env: BUILD="base" TEST_TARGET="scalding-hadoop-test" script: "scripts/run_test.sh" - scala: 2.10.4 - env: BUILD="base" TEST_TARGET="scalding-jdbc" + env: BUILD="base" TEST_TARGET="scalding-jdbc scalding-json" script: "scripts/run_test.sh" - - scala: 2.10.4 - env: BUILD="base" TEST_TARGET="scalding-json" + - scala: 2.11.4 + env: BUILD="base" TEST_TARGET="scalding-jdbc scalding-json" script: "scripts/run_test.sh" - scala: 2.10.4 - env: BUILD="base" TEST_TARGET="scalding-parquet" + env: BUILD="base" TEST_TARGET="scalding-parquet scalding-parquet-scrooge" script: "scripts/run_test.sh" - - scala: 2.10.4 - env: BUILD="base" TEST_TARGET="scalding-parquet-scrooge" + - scala: 2.11.4 + env: BUILD="base" TEST_TARGET="scalding-parquet scalding-parquet-scrooge" script: "scripts/run_test.sh" - scala: 2.10.4 env: BUILD="base" TEST_TARGET="scalding-repl" script: "scripts/run_test.sh" - - scala: 2.9.3 - env: BUILD="base" TEST_TARGET="scalding-args" - script: "scripts/run_test.sh" - - - scala: 2.9.3 - env: BUILD="base" TEST_TARGET="scalding-avro" - script: "scripts/run_test.sh" - - - scala: 2.9.3 - env: BUILD="base" TEST_TARGET="scalding-commons" - script: "scripts/run_test.sh" - - - scala: 2.9.3 - env: BUILD="base" TEST_TARGET="scalding-core" - script: "scripts/run_test.sh" - - - scala: 2.9.3 - env: BUILD="base" TEST_TARGET="scalding-date" - script: "scripts/run_test.sh" - - - scala: 2.9.3 - env: BUILD="base" TEST_TARGET="scalding-hadoop-test" - script: "scripts/run_test.sh" - - - scala: 2.9.3 - env: BUILD="base" TEST_TARGET="scalding-hraven" - script: "scripts/run_test.sh" - - - scala: 2.9.3 - env: BUILD="base" TEST_TARGET="scalding-jdbc" - script: "scripts/run_test.sh" - - - scala: 2.9.3 - env: BUILD="base" TEST_TARGET="scalding-json" - script: "scripts/run_test.sh" - - - scala: 2.9.3 - env: BUILD="base" TEST_TARGET="scalding-parquet" - script: "scripts/run_test.sh" - - - scala: 2.9.3 - env: BUILD="base" TEST_TARGET="scalding-repl" - script: "scripts/run_test.sh" - - scala: 2.10.4 env: BUILD="test tutorials" script: - "scripts/build_assembly_no_test.sh scalding" - "scripts/test_tutorials.sh" - - scala: 2.10.4 - env: BUILD="test matrix tutorials" + + - scala: 2.11.4 + env: BUILD="test tutorials" script: - "scripts/build_assembly_no_test.sh scalding" - - "scripts/test_matrix_tutorials.sh" - - scala: 2.10.4 - env: BUILD="test repl tutorials" - script: - - "scripts/build_assembly_no_test.sh scalding-repl" - - "scripts/test_repl_tutorial.sh" + - "scripts/test_tutorials.sh" + - scala: 2.10.4 - env: BUILD="test typed tutorials" - script: - - "scripts/build_assembly_no_test.sh scalding-core" - - "scripts/test_typed_tutorials.sh" - - scala: 2.9.3 - env: BUILD="test tutorials" + env: BUILD="test matrix tutorials" script: - "scripts/build_assembly_no_test.sh scalding" - - "scripts/test_tutorials.sh" - - scala: 2.9.3 + - "scripts/test_matrix_tutorials.sh" + + - scala: 2.11.4 env: BUILD="test matrix tutorials" script: - "scripts/build_assembly_no_test.sh scalding" - "scripts/test_matrix_tutorials.sh" - - scala: 2.9.3 - env: BUILD="test repl tutorials" + + - scala: 2.10.4 + env: BUILD="test repl and typed tutorials" script: - "scripts/build_assembly_no_test.sh scalding-repl" - "scripts/test_repl_tutorial.sh" - - scala: 2.9.3 + - "scripts/build_assembly_no_test.sh scalding-core" + - "scripts/test_typed_tutorials.sh" + + - scala: 2.11.4 env: BUILD="test typed tutorials" script: - "scripts/build_assembly_no_test.sh scalding-core" @@ -151,4 +99,3 @@ matrix: notifications: irc: "chat.freenode.net#scalding" - diff --git a/CHANGES.md b/CHANGES.md index da644fbaab..1e84e51bd5 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,5 +1,35 @@ # Scalding # +### Version 0.13.0 ### +* Covert LzoTextDelimited to Cascading scheme.: https://github.com/twitter/scalding/pull/1179 +* Make TraceUtil support versions of cascading older than 2.6: https://github.com/twitter/scalding/pull/1180 +* Add support for more LzoTextDeilmited parameters in LzoTraits.: https://github.com/twitter/scalding/pull/1178 +* Use latest algebird, bijection, chill, elephantbird, and scala 2.11.5: https://github.com/twitter/scalding/pull/1174 +* Cascading 2.6 tracing: https://github.com/twitter/scalding/pull/1156 +* use Cascading 2.6.1 and cascading-jdbc 2.6.0: https://github.com/twitter/scalding/pull/1110 +* add reducer option to LookupJoin: https://github.com/twitter/scalding/pull/1160 +* Add dump to ValuePipe in the REPL: https://github.com/twitter/scalding/pull/1157 +* Ianoc/type descriptor: https://github.com/twitter/scalding/pull/1147 +* Refactor around the macro definitions into 3 files. Both converter and setter support Options: https://github.com/twitter/scalding/pull/1145 +* Fix a few random typos: https://github.com/twitter/scalding/pull/1144 +* Fix two issues found by static analysis: https://github.com/twitter/scalding/pull/1143 +* Add implicit helpers for numeric arguments: https://github.com/twitter/scalding/pull/1138 +* Add a fields macro: https://github.com/twitter/scalding/pull/1132 +* Ianoc/case class tuple converters: https://github.com/twitter/scalding/pull/1131 +* Some minor changes, cleanup pulled from jco's macro branch: https://github.com/twitter/scalding/pull/1130 +* Adds a typedjson source: https://github.com/twitter/scalding/pull/1129 +* Pulls all external 3rdparty versions up to the top of the build file: https://github.com/twitter/scalding/pull/1128 +* remove transitive pig and elephantbird dependencies for parquet-cascading: https://github.com/twitter/scalding/pull/1127 +* Some minor clean up in the build file: https://github.com/twitter/scalding/pull/1123 +* Ianoc/scalding 210: https://github.com/twitter/scalding/pull/1116 +* Decrease test count: https://github.com/twitter/scalding/pull/1117 +* Removes scala 2.9.3: https://github.com/twitter/scalding/pull/1106 +* Fix some typos in TypedPipe docs, expand flatMap docs: https://github.com/twitter/scalding/pull/1115 +* Implicit execution context / easier switching between modes: https://github.com/twitter/scalding/pull/1113 +* Add more documentation to TypedPipe: https://github.com/twitter/scalding/pull/1111 +* Update the README: https://github.com/twitter/scalding/pull/1114 +* Fixed comment in LookupJoin.scala: https://github.com/twitter/scalding/pull/1108 + ### Version 0.12.0 ### * Fix long compile time for MultiJoin helpers: https://github.com/twitter/scalding/pull/1109 * Allows reducer estimation to operate on all hfs taps: https://github.com/twitter/scalding/pull/1080 diff --git a/README.md b/README.md index 9d7a524e53..25e4308b44 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.12.0` +Current version: `0.13.0` ## Word Count @@ -15,11 +15,12 @@ package com.twitter.scalding.examples import com.twitter.scalding._ -class WordCountJob(args : Args) extends Job(args) { - TextLine( args("input") ) - .flatMap('line -> 'word) { line : String => tokenize(line) } - .groupBy('word) { _.size } - .write( Tsv( args("output") ) ) +class WordCountJob(args: Args) extends Job(args) { + TypedPipe.from(TextLine(args("input"))) + .flatMap { line => tokenize(line) } + .groupBy { word => word } // use each word for a key + .size // in each group, get the size + .write(TypedTsv[(String, Long)](args("output"))) // Split a piece of text into individual words. def tokenize(text : String) : Array[String] = { @@ -36,10 +37,12 @@ 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) +* [**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: - * [Fields-based API Reference](https://github.com/twitter/scalding/wiki/Fields-based-API-Reference) * [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 @@ -47,6 +50,9 @@ You can find more example code under [examples/](https://github.com/twitter/scal Please feel free to use the beautiful [Scalding logo](https://drive.google.com/folderview?id=0B3i3pDi3yVgNbm9pMUdDcHFKVEk&usp=sharing) artwork anywhere. +## Code of Conduct +This, and all github.com/twitter projects, are under the [Twitter Open Source Code of Conduct](https://engineering.twitter.com/opensource/code-of-conduct). Additionally, see the [Typelevel Code of Conduct](http://typelevel.org/conduct) for specific examples of harassing behavior that are not tolerated. + ## Building There is a script (called sbt) in the root that loads the correct sbt version to build: @@ -63,25 +69,21 @@ Please refer to [FAQ page](https://github.com/twitter/scalding/wiki/Frequently-a We use [Travis CI](http://travis-ci.org/) to verify the build: [![Build Status](https://secure.travis-ci.org/twitter/scalding.png)](http://travis-ci.org/twitter/scalding) +We use [Coveralls](https://coveralls.io/r/twitter/scalding) for code coverage results: +[![Coverage Status](https://coveralls.io/repos/twitter/scalding/badge.png?branch=develop)](https://coveralls.io/r/twitter/scalding?branch=develop) + Scalding modules are available from maven central. -The current groupid and version for all modules is, respectively, `"com.twitter"` and `0.11.0`. +The current groupid and version for all modules is, respectively, `"com.twitter"` and `0.12.0`. Current published artifacts are -* `scalding-core_2.9.3` * `scalding-core_2.10` -* `scalding-args_2.9.3` * `scalding-args_2.10` -* `scalding-date_2.9.3` * `scalding-date_2.10` -* `scalding-commons_2.9.3` * `scalding-commons_2.10` -* `scalding-avro_2.9.3` * `scalding-avro_2.10` -* `scalding-parquet_2.9.3` * `scalding-parquet_2.10` -* `scalding-repl_2.9.3` * `scalding-repl_2.10` diff --git a/project/Build.scala b/project/Build.scala index 5073e91989..c349eafed7 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -14,9 +14,31 @@ import scala.collection.JavaConverters._ object ScaldingBuild extends Build { def scalaBinaryVersion(scalaVersion: String) = scalaVersion match { - case version if version startsWith "2.9" => "2.9" case version if version startsWith "2.10" => "2.10" + case version if version startsWith "2.11" => "2.11" + case _ => sys.error("unknown error") } + def isScala210x(scalaVersion: String) = scalaBinaryVersion(scalaVersion) == "2.10" + + val scalaTestVersion = "2.2.2" + val scalaCheckVersion = "1.11.5" + val hadoopVersion = "1.2.1" + val algebirdVersion = "0.9.0" + val bijectionVersion = "0.7.2" + val chillVersion = "0.5.2" + val slf4jVersion = "1.6.6" + val parquetVersion = "1.6.0rc4" + val dfsDatastoresVersion = "1.3.4" + 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 printDependencyClasspath = taskKey[Unit]("Prints location of the dependencies") @@ -25,7 +47,7 @@ object ScaldingBuild extends Build { scalaVersion := "2.10.4", - crossScalaVersions := Seq("2.9.3", "2.10.4"), + crossScalaVersions := Seq("2.10.4", "2.11.5"), ScalariformKeys.preferences := formattingPreferences, @@ -34,9 +56,10 @@ object ScaldingBuild extends Build { javacOptions in doc := Seq("-source", "1.6"), libraryDependencies ++= Seq( - "org.scalacheck" %% "scalacheck" % "1.10.0" % "test", - "org.scala-tools.testing" %% "specs" % "1.6.9" % "test", - "org.mockito" % "mockito-all" % "1.8.5" % "test" + "org.mockito" % "mockito-all" % "1.8.5" % "test", + "org.scalacheck" %% "scalacheck" % scalaCheckVersion % "test", + "org.scalatest" %% "scalatest" % scalaTestVersion % "test", + "org.slf4j" % "slf4j-log4j12" % slf4jVersion % "test" ), resolvers ++= Seq( @@ -64,7 +87,17 @@ object ScaldingBuild extends Build { parallelExecution in Test := false, - scalacOptions ++= Seq("-unchecked", "-deprecation"), + scalacOptions ++= Seq("-unchecked", "-deprecation", "-language:implicitConversions", "-language:higherKinds", "-language:existentials"), + + scalacOptions <++= (scalaVersion) map { sv => + if (isScala210x(sv)) + Seq("-Xdivergence211") + else + Seq() + }, + + // Enables full stack traces in scalatest + testOptions in Test += Tests.Argument(TestFrameworks.ScalaTest, "-oF"), // Uncomment if you don't want to run all the tests before building assembly // test in assembly := {}, @@ -167,6 +200,7 @@ object ScaldingBuild extends Build { scaldingJson, scaldingJdbc, scaldingHadoopTest, + scaldingMacros, maple ) @@ -187,7 +221,7 @@ object ScaldingBuild extends Build { Some(subProj) .filterNot(unreleasedModules.contains(_)) .map { - s => "com.twitter" % ("scalding-" + s + "_2.9.3") % "0.11.0" + s => "com.twitter" % ("scalding-" + s + "_2.10") % "0.13.0" } def module(name: String) = { @@ -203,16 +237,10 @@ object ScaldingBuild extends Build { lazy val scaldingDate = module("date") lazy val cascadingVersion = - System.getenv.asScala.getOrElse("SCALDING_CASCADING_VERSION", "2.5.5") + System.getenv.asScala.getOrElse("SCALDING_CASCADING_VERSION", "2.6.1") lazy val cascadingJDBCVersion = - System.getenv.asScala.getOrElse("SCALDING_CASCADING_JDBC_VERSION", "2.5.4") - - val hadoopVersion = "1.2.1" - val algebirdVersion = "0.7.1" - val bijectionVersion = "0.6.3" - val chillVersion = "0.4.0" - val slf4jVersion = "1.6.6" + System.getenv.asScala.getOrElse("SCALDING_CASCADING_JDBC_VERSION", "2.6.0") lazy val scaldingCore = module("core").settings( libraryDependencies ++= Seq( @@ -233,78 +261,74 @@ object ScaldingBuild extends Build { lazy val scaldingCommons = module("commons").settings( libraryDependencies ++= Seq( - "com.backtype" % "dfs-datastores-cascading" % "1.3.4", - "com.backtype" % "dfs-datastores" % "1.3.4", + "com.backtype" % "dfs-datastores-cascading" % dfsDatastoresVersion, + "com.backtype" % "dfs-datastores" % dfsDatastoresVersion, // TODO: split into scalding-protobuf - "com.google.protobuf" % "protobuf-java" % "2.4.1", + "com.google.protobuf" % "protobuf-java" % protobufVersion, "com.twitter" %% "bijection-core" % bijectionVersion, "com.twitter" %% "algebird-core" % algebirdVersion, "com.twitter" %% "chill" % chillVersion, - "com.twitter.elephantbird" % "elephant-bird-cascading2" % "4.4", - "com.hadoop.gplcompression" % "hadoop-lzo" % "0.4.16", + "com.twitter.elephantbird" % "elephant-bird-cascading2" % elephantbirdVersion, + "com.twitter.elephantbird" % "elephant-bird-core" % elephantbirdVersion, + "com.hadoop.gplcompression" % "hadoop-lzo" % hadoopLzoVersion, // TODO: split this out into scalding-thrift - "org.apache.thrift" % "libthrift" % "0.5.0", + "org.apache.thrift" % "libthrift" % thriftVersion, "org.slf4j" % "slf4j-api" % slf4jVersion, - "org.slf4j" % "slf4j-log4j12" % slf4jVersion % "provided", - "org.scalacheck" %% "scalacheck" % "1.10.0" % "test", - "org.scala-tools.testing" %% "specs" % "1.6.9" % "test" + "org.slf4j" % "slf4j-log4j12" % slf4jVersion % "provided" ) ).dependsOn(scaldingArgs, scaldingDate, scaldingCore) lazy val scaldingAvro = module("avro").settings( libraryDependencies ++= Seq( - "cascading.avro" % "avro-scheme" % "2.1.2", - "org.apache.avro" % "avro" % "1.7.4", + "cascading.avro" % "avro-scheme" % cascadingAvroVersion, + "org.apache.avro" % "avro" % avroVersion, "org.slf4j" % "slf4j-api" % slf4jVersion, - "org.apache.hadoop" % "hadoop-core" % hadoopVersion % "provided", - "org.slf4j" % "slf4j-log4j12" % slf4jVersion % "test", - "org.scalacheck" %% "scalacheck" % "1.10.0" % "test", - "org.scala-tools.testing" %% "specs" % "1.6.9" % "test" + "org.apache.hadoop" % "hadoop-core" % hadoopVersion % "provided" ) ).dependsOn(scaldingCore) lazy val scaldingParquet = module("parquet").settings( libraryDependencies ++= Seq( - "com.twitter" % "parquet-cascading" % "1.6.0rc2", + // see https://issues.apache.org/jira/browse/PARQUET-143 for exclusions + "com.twitter" % "parquet-cascading" % parquetVersion + exclude("com.twitter", "parquet-pig") + exclude("com.twitter.elephantbird", "elephant-bird-pig") + 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", - "org.slf4j" % "slf4j-log4j12" % slf4jVersion % "test", - "org.scalacheck" %% "scalacheck" % "1.10.0" % "test", - "org.scala-tools.testing" %% "specs" % "1.6.9" % "test" + "org.apache.hadoop" % "hadoop-core" % hadoopVersion % "provided" ) ).dependsOn(scaldingCore) def scaldingParquetScroogeDeps(version: String) = { - if (scalaBinaryVersion(version) == "2.9") - Seq() - else + if (isScala210x(version)) Seq( - "com.twitter" % "parquet-cascading" % "1.6.0rc2", - "com.twitter" %% "parquet-scrooge" % "1.6.0rc2", + // see https://issues.apache.org/jira/browse/PARQUET-143 for exclusions + "com.twitter" % "parquet-cascading" % parquetVersion + exclude("com.twitter", "parquet-pig") + exclude("com.twitter.elephantbird", "elephant-bird-pig") + exclude("com.twitter.elephantbird", "elephant-bird-core"), + "com.twitter" %% "parquet-scrooge" % parquetVersion, "org.slf4j" % "slf4j-api" % slf4jVersion, - "org.apache.hadoop" % "hadoop-core" % hadoopVersion % "provided", - "org.slf4j" % "slf4j-log4j12" % slf4jVersion % "test", - "org.scalacheck" %% "scalacheck" % "1.10.0" % "test", - "org.scala-tools.testing" %% "specs" % "1.6.9" % "test" + "org.apache.hadoop" % "hadoop-core" % hadoopVersion % "provided" ) + else + Seq() } lazy val scaldingParquetScrooge = module("parquet-scrooge").settings( - skip in compile := scalaBinaryVersion(scalaVersion.value) == "2.9", - skip in test := scalaBinaryVersion(scalaVersion.value) == "2.9", - publishArtifact := !(scalaBinaryVersion(scalaVersion.value) == "2.9"), + skip in compile := !(isScala210x(scalaVersion.value)), + skip in test := !(isScala210x(scalaVersion.value)), + publishArtifact := isScala210x(scalaVersion.value), libraryDependencies ++= scaldingParquetScroogeDeps(scalaVersion.value) ).dependsOn(scaldingCore, scaldingParquet % "compile->compile;test->test") lazy val scaldingHRaven = module("hraven").settings( libraryDependencies ++= Seq( - "com.twitter.hraven" % "hraven-core" % "0.9.13", - "org.apache.hbase" % "hbase" % "0.94.10", + "com.twitter.hraven" % "hraven-core" % hravenVersion, + "org.apache.hbase" % "hbase" % hbaseVersion, "org.slf4j" % "slf4j-api" % slf4jVersion, - "org.apache.hadoop" % "hadoop-core" % hadoopVersion % "provided", - "org.slf4j" % "slf4j-log4j12" % slf4jVersion % "test", - "org.scalacheck" %% "scalacheck" % "1.10.0" % "test", - "org.scala-tools.testing" %% "specs" % "1.6.9" % "test" + "org.apache.hadoop" % "hadoop-core" % hadoopVersion % "provided" ) ).dependsOn(scaldingCore) @@ -317,21 +341,27 @@ object ScaldingBuild extends Build { lazy val scaldingRepl = module("repl") .configs(Unprovided) // include 'unprovided' as config option .settings( + skip in compile := !isScala210x(scalaVersion.value), + skip in test := !isScala210x(scalaVersion.value), + publishArtifact := isScala210x(scalaVersion.value), initialCommands in console := """ import com.twitter.scalding._ import com.twitter.scalding.ReplImplicits._ import com.twitter.scalding.ReplImplicitContext._ """, libraryDependencies <++= (scalaVersion) { scalaVersion => Seq( - "org.scala-lang" % "jline" % scalaVersion, + "jline" % "jline" % scalaVersion.take(4), "org.scala-lang" % "scala-compiler" % scalaVersion, + "org.scala-lang" % "scala-reflect" % scalaVersion, "org.apache.hadoop" % "hadoop-core" % hadoopVersion % "provided", "org.apache.hadoop" % "hadoop-core" % hadoopVersion % "unprovided", "org.slf4j" % "slf4j-api" % slf4jVersion, "org.slf4j" % "slf4j-log4j12" % slf4jVersion % "provided", "org.slf4j" % "slf4j-log4j12" % slf4jVersion % "unprovided" ) - } + }, + // https://gist.github.com/djspiewak/976cd8ac65e20e136f05 + unmanagedSourceDirectories in Compile += (sourceDirectory in Compile).value / s"scala-${scalaBinaryVersion(scalaVersion.value)}" ).dependsOn(scaldingCore) // run with 'unprovided' config includes libs marked 'unprovided' in classpath .settings(inConfig(Unprovided)(Classpaths.configSettings ++ Seq( @@ -345,8 +375,10 @@ object ScaldingBuild extends Build { lazy val scaldingJson = module("json").settings( libraryDependencies <++= (scalaVersion) { scalaVersion => Seq( "org.apache.hadoop" % "hadoop-core" % hadoopVersion % "provided", - "com.fasterxml.jackson.module" %% "jackson-module-scala" % "2.2.3" - ) + "com.fasterxml.jackson.module" %% "jackson-module-scala" % jacksonVersion, + "org.json4s" %% "json4s-native" % json4SVersion, + "com.twitter.elephantbird" % "elephant-bird-cascading2" % elephantbirdVersion % "provided" + ) } ).dependsOn(scaldingCore) @@ -364,11 +396,23 @@ object ScaldingBuild extends Build { ("org.apache.hadoop" % "hadoop-core" % hadoopVersion), ("org.apache.hadoop" % "hadoop-minicluster" % hadoopVersion), "org.slf4j" % "slf4j-api" % slf4jVersion, - "org.slf4j" % "slf4j-log4j12" % slf4jVersion + "org.slf4j" % "slf4j-log4j12" % slf4jVersion, + "org.scalacheck" %% "scalacheck" % scalaCheckVersion, + "org.scalatest" %% "scalatest" % scalaTestVersion ) } ).dependsOn(scaldingCore) + lazy val scaldingMacros = module("macros").settings( + libraryDependencies <++= (scalaVersion) { scalaVersion => Seq( + "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()) + }, + addCompilerPlugin("org.scalamacros" % "paradise" % "2.0.1" cross CrossVersion.full) + ).dependsOn(scaldingCore, scaldingHadoopTest) + // This one uses a different naming convention lazy val maple = Project( id = "maple", @@ -381,7 +425,7 @@ object ScaldingBuild extends Build { autoScalaLibrary := false, libraryDependencies <++= (scalaVersion) { scalaVersion => Seq( "org.apache.hadoop" % "hadoop-core" % hadoopVersion % "provided", - "org.apache.hbase" % "hbase" % "0.94.5" % "provided", + "org.apache.hbase" % "hbase" % hbaseVersion % "provided", "cascading" % "cascading-hadoop" % cascadingVersion ) } diff --git a/project/travis-log4j.properties b/project/travis-log4j.properties index d0513fadd8..e45c815eb0 100644 --- a/project/travis-log4j.properties +++ b/project/travis-log4j.properties @@ -1,10 +1,12 @@ -log4j.debug=true -log4j.rootCategory=WARN, console -log4j.threshhold=ALL +log4j.rootCategory=DEBUG, console +log4j.threshold=ALL log4j.category.cascading=WARN log4j.category.com.twitter=INFO log4j.logger.org.apache.hadoop=ERROR +log4j.logger.cascading.flow=WARN +log4j.logger.cascading.tap=WARN + log4j.appender.console=org.apache.log4j.ConsoleAppender log4j.appender.console.layout=org.apache.log4j.PatternLayout 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 da1c5e3081..98f09c941b 100644 --- a/scalding-args/src/main/scala/com/twitter/scalding/Args.scala +++ b/scalding-args/src/main/scala/com/twitter/scalding/Args.scala @@ -15,6 +15,8 @@ limitations under the License. */ package com.twitter.scalding +import scala.util.control.NonFatal + case class ArgsException(message: String) extends RuntimeException(message) /** @@ -160,4 +162,56 @@ class Args(val m: Map[String, List[String]]) extends java.io.Serializable { case List(a) => Some(a) case _ => throw ArgsException("Please provide at most one value for --" + key) } + + def int(key: String, default: Int): Int = { + optional(key).map(value => try value.toInt catch { + case NonFatal(_) => throw ArgsException(s"Invalid value ${value} for -- ${key}") + }).getOrElse(default) + } + + def int(key: String): Int = { + val value = required(key) + try value.toInt catch { + case NonFatal(_) => throw ArgsException(s"Invalid value ${value} for -- ${key}") + } + } + + def long(key: String, default: Long): Long = { + optional(key).map(value => try value.toLong catch { + case NonFatal(_) => throw ArgsException(s"Invalid value ${value} for -- ${key}") + }).getOrElse(default) + } + + def long(key: String): Long = { + val value = required(key) + try value.toLong catch { + case NonFatal(_) => throw ArgsException(s"Invalid value ${value} for -- ${key}") + } + } + + def float(key: String, default: Float): Float = { + optional(key).map(value => try value.toFloat catch { + case NonFatal(_) => throw ArgsException(s"Invalid value ${value} for -- ${key}") + }).getOrElse(default) + } + + def float(key: String): Float = { + val value = required(key) + try value.toFloat catch { + case NonFatal(_) => throw ArgsException(s"Invalid value ${value} for -- ${key}") + } + } + + def double(key: String, default: Double): Double = { + optional(key).map(value => try value.toDouble catch { + case NonFatal(_) => throw ArgsException(s"Invalid value ${value} for -- ${key}") + }).getOrElse(default) + } + + def double(key: String): Double = { + val value = required(key) + try value.toDouble catch { + case NonFatal(_) => throw ArgsException(s"Invalid value ${value} for -- ${key}") + } + } } diff --git a/scalding-args/src/test/scala/com/twitter/scalding/ArgTest.scala b/scalding-args/src/test/scala/com/twitter/scalding/ArgTest.scala index c0174a929c..367c596be9 100644 --- a/scalding-args/src/test/scala/com/twitter/scalding/ArgTest.scala +++ b/scalding-args/src/test/scala/com/twitter/scalding/ArgTest.scala @@ -14,115 +14,145 @@ See the License for the specific language governing permissions and limitations under the License. */ package com.twitter.scalding -import org.specs._ +import org.scalatest.WordSpec -class ArgTest extends Specification { +class ArgTest extends WordSpec { "Tool.parseArgs" should { "handle the empty list" in { val map = Args(Array[String]()) - map.list("") must be_==(List()) + assert(map.list("").isEmpty) } "accept any number of dashed args" in { val map = Args(Array("--one", "1", "--two", "2", "--three", "3")) - map.list("") must be_==(List()) - map.optional("") must be_==(None) - - map.list("absent") must be_==(List()) - map.optional("absent") must be_==(None) - - map("one") must be_==("1") - map.list("one") must be_==(List("1")) - map.required("one") must be_==("1") - map.optional("one") must be_==(Some("1")) - - map("two") must be_==("2") - map.list("two") must be_==(List("2")) - map.required("two") must be_==("2") - map.optional("two") must be_==(Some("2")) - - map("three") must be_==("3") - map.list("three") must be_==(List("3")) - map.required("three") must be_==("3") - map.optional("three") must be_==(Some("3")) + assert(map.list("").isEmpty) + assert(map.optional("").isEmpty) + + assert(map.list("absent").isEmpty) + assert(map.optional("absent").isEmpty) + + assert(map("one") === "1") + assert(map.list("one") === List("1")) + assert(map.required("one") === "1") + assert(map.optional("one") === Some("1")) + + assert(map("two") === "2") + assert(map.list("two") === List("2")) + assert(map.required("two") === "2") + assert(map.optional("two") === Some("2")) + + assert(map("three") === "3") + assert(map.list("three") === List("3")) + assert(map.required("three") === "3") + assert(map.optional("three") === Some("3")) } "remove empty args in lists" in { val map = Args(Array("", "hello", "--one", "1", "", "\t", "--two", "2", "", "3")) - map("") must be_==("hello") - map.list("") must be_==(List("hello")) - map("one") must be_==("1") - map.list("one") must be_==(List("1")) - map.list("two") must be_==(List("2", "3")) + assert(map("") === "hello") + assert(map.list("") === List("hello")) + assert(map("one") === "1") + assert(map.list("one") === List("1")) + assert(map.list("two") === List("2", "3")) } "put initial args into the empty key" in { val map = Args(List("hello", "--one", "1")) - map("") must be_==("hello") - map.list("") must be_==(List("hello")) - map.required("") must be_==("hello") - map.optional("") must be_==(Some("hello")) + assert(map("") === "hello") + assert(map.list("") === List("hello")) + assert(map.required("") === "hello") + assert(map.optional("") === Some("hello")) - map("one") must be_==("1") - map.list("one") must be_==(List("1")) + assert(map("one") === "1") + assert(map.list("one") === List("1")) } "allow any number of args per key" in { val map = Args(Array("--one", "1", "--two", "2", "deux", "--zero")) - map("one") must be_==("1") - map.list("two") must be_==(List("2", "deux")) - map.boolean("zero") must be_==(true) + assert(map("one") === "1") + assert(map.list("two") === List("2", "deux")) + assert(map.boolean("zero")) } "allow any number of dashes" in { val map = Args(Array("-one", "1", "--two", "2", "---three", "3")) - map("three") must be_==("3") - map("two") must be_==("2") - map("one") must be_==("1") + assert(map("three") === "3") + assert(map("two") === "2") + assert(map("one") === "1") } "round trip to/from string" in { val a = Args("--you all every --body 1 2") - a must be_==(Args(a.toString)) - a must be_==(Args(a.toList)) + assert(a === Args(a.toString)) + assert(a === Args(a.toList)) } "handle positional arguments" in { val a = Args("p0 p1 p2 --f 1 2") - a.positional must be_==(List("p0", "p1", "p2")) - Args(a.toString) must be_==(a) - Args(a.toList) must be_==(a) + assert(a.positional === List("p0", "p1", "p2")) + assert(Args(a.toString) === a) + assert(Args(a.toList) === a) } "handle negative numbers in args" in { val a = Args("--a 1 -2.1 --b 1 -3 4 --c -5") - a.list("a") must_== List("1", "-2.1") - a.list("b") must_== List("1", "-3", "4") - a("c").toInt must_== -5 + assert(a.list("a") === List("1", "-2.1")) + assert(a.list("b") === List("1", "-3", "4")) + assert(a("c").toInt === -5) } "handle strange characters in the args" in { val a = Args("p-p --a-a 1-1 -b=b c=d e/f -5,2 5,3") - a.positional must be_==(List("p-p")) - a.list("a-a") must be_==(List("1-1")) - a.list("b=b") must be_==(List("c=d", "e/f")) - a.list("5,2") must be_==(List("5,3")) + assert(a.positional === List("p-p")) + assert(a.list("a-a") === List("1-1")) + assert(a.list("b=b") === List("c=d", "e/f")) + assert(a.list("5,2") === List("5,3")) } "access positional arguments using apply" in { val a = Args("a b c --d e") - a(0) must be_==("a") - a(1) must be_==("b") - a(2) must be_==("c") - a("d") must be_==("e") + assert(a(0) === "a") + assert(a(1) === "b") + assert(a(2) === "c") + assert(a("d") === "e") } "verify that args belong to an accepted key set" in { val a = Args("a --one --two b --three c d --scalding.tool.mode") a.restrictTo(Set("one", "two", "three", "four")) - a.restrictTo(Set("one", "two")) must throwA[java.lang.RuntimeException] + intercept[RuntimeException] { a.restrictTo(Set("one", "two")) } } + "correctly parse numeric args" in { + val map = Args(Array("--anInt", "-1", "--aLong", "21474836470", "--aDecimal", "3.141592654", "--aString", "foo")) + assert(map.int("anInt") == "-1".toInt) + assert(map.int("anInt", 2) == "-1".toInt) + assert(map.int("nothing", 2) == 2) + intercept[RuntimeException] { map.int("nothing") } + intercept[RuntimeException] { map.int("aString") } + intercept[RuntimeException] { map.int("aString", 2) } + + assert(map.long("aLong") == "21474836470".toLong) + assert(map.long("anInt", 2L) == "-1".toLong) + assert(map.long("nothing", 2L) == 2L) + intercept[RuntimeException] { map.long("nothing") } + intercept[RuntimeException] { map.long("aString") } + intercept[RuntimeException] { map.long("aString", 2L) } + + assert(map.float("aDecimal") == "3.141592654".toFloat) + assert(map.float("aDecimal", 2.71828f) == "3.141592654".toFloat) + assert(map.float("nothing", 2.71828f) == 2.71828f) + intercept[RuntimeException] { map.float("nothing") } + intercept[RuntimeException] { map.float("aString") } + intercept[RuntimeException] { map.float("aString", 2.71828f) } + + assert(map.double("aDecimal") == "3.141592654".toDouble) + assert(map.double("aDecimal", 2.71828d) == "3.141592654".toDouble) + assert(map.double("nothing", 2.71828d) == 2.71828d) + intercept[RuntimeException] { map.double("nothing") } + intercept[RuntimeException] { map.double("aString") } + intercept[RuntimeException] { map.double("aString", 2.71828d) } + } } } diff --git a/scalding-args/src/test/scala/com/twitter/scalding/RangedArgsSpec.scala b/scalding-args/src/test/scala/com/twitter/scalding/RangedArgsSpec.scala index f0b5daea75..079384398f 100644 --- a/scalding-args/src/test/scala/com/twitter/scalding/RangedArgsSpec.scala +++ b/scalding-args/src/test/scala/com/twitter/scalding/RangedArgsSpec.scala @@ -16,41 +16,41 @@ limitations under the License. package com.twitter.scalding -import org.specs._ +import org.scalatest.WordSpec -class RangeSpecs extends Specification { +class RangeSpecs extends WordSpec { "A Range" should { val testRange = Range(4, 5) "contain its endpoints" in { - testRange.lower must_== 4 - testRange.upper must_== 5 + assert(testRange.lower === 4) + assert(testRange.upper === 5) } "throw errors for misordered ranges" in { Range(4, 4) - Range(5, 4) must throwAn[AssertionError] + intercept[AssertionError] { Range(5, 4) } } "assert lower bounds" in { testRange.assertLowerBound(3) testRange.assertLowerBound(4) - testRange.assertLowerBound(5) must throwAn[AssertionError] + intercept[AssertionError] { testRange.assertLowerBound(5) } } "assert upper bounds" in { testRange.assertUpperBound(6) testRange.assertUpperBound(5) - testRange.assertUpperBound(4) must throwAn[AssertionError] + intercept[AssertionError] { testRange.assertUpperBound(4) } } - "print nicely with mkString" in { + "print nicely with mkString" should { "for trivial ranges" in { - Range(4, 4).mkString("_") must beEqualTo("4") + assert(Range(4, 4).mkString("_") === "4") } "for proper ranges" in { - testRange.mkString("_") must beEqualTo("4_5") - testRange.mkString("-") must beEqualTo("4-5") + assert(testRange.mkString("_") === "4_5") + assert(testRange.mkString("-") === "4-5") } } } diff --git a/scalding-avro/src/main/scala/com/twitter/scalding/avro/SchemaType.scala b/scalding-avro/src/main/scala/com/twitter/scalding/avro/SchemaType.scala index 51b49e709e..b053cdf2f5 100644 --- a/scalding-avro/src/main/scala/com/twitter/scalding/avro/SchemaType.scala +++ b/scalding-avro/src/main/scala/com/twitter/scalding/avro/SchemaType.scala @@ -72,7 +72,7 @@ object AvroSchemaType { // Avro SpecificRecord implicit def SpecificRecordSchema[T <: SpecificRecord](implicit mf: Manifest[T]) = new AvroSchemaType[T] { - def schema = mf.erasure.newInstance.asInstanceOf[SpecificRecord].getSchema + def schema = mf.runtimeClass.newInstance.asInstanceOf[SpecificRecord].getSchema } } \ No newline at end of file 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 d65e153854..c7bee55b4e 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 @@ -39,7 +39,7 @@ abstract class DailySuffixLzoCodec[T](prefix: String, dateRange: DateRange)(impl abstract class DailySuffixLzoProtobuf[T <: Message: Manifest](prefix: String, dateRange: DateRange) extends DailySuffixSource(prefix, dateRange) with LzoProtobuf[T] { - override def column = manifest[T].erasure + override def column = manifest[T].runtimeClass } abstract class DailySuffixMostRecentLzoProtobuf[T <: Message: Manifest](prefix: String, dateRange: DateRange) @@ -49,12 +49,12 @@ abstract class DailySuffixMostRecentLzoProtobuf[T <: Message: Manifest](prefix: abstract class DailySuffixLzoThrift[T <: TBase[_, _]: Manifest](prefix: String, dateRange: DateRange) extends DailySuffixSource(prefix, dateRange) with LzoThrift[T] { - override def column = manifest[T].erasure + override def column = manifest[T].runtimeClass } abstract class DailyPrefixSuffixLzoThrift[T <: TBase[_, _]: Manifest](prefix: String, suffix: String, dateRange: DateRange) extends DailyPrefixSuffixSource(prefix, suffix, dateRange) with LzoThrift[T] { - override def column = manifest[T].erasure + override def column = manifest[T].runtimeClass } abstract class TimePathedLongThriftSequenceFile[V <: TBase[_, _]: Manifest](f: Fields, prefix: String, dateFormat: String, dateRange: DateRange) 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 82281e96fa..a7aad7ee18 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 @@ -24,10 +24,10 @@ import org.apache.thrift.TBase abstract class FixedPathLzoThrift[T <: TBase[_, _]: Manifest](path: String*) extends FixedPathSource(path: _*) with LzoThrift[T] { - def column = manifest[T].erasure + def column = manifest[T].runtimeClass } abstract class FixedPathLzoProtobuf[T <: Message: Manifest](path: String) extends FixedPathSource(path) with LzoProtobuf[T] { - def column = manifest[T].erasure + def column = manifest[T].runtimeClass } 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 769bce3211..64d29df656 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 @@ -39,12 +39,12 @@ case class HourlySuffixLzoTsv(prefix: String, fs: Fields = Fields.ALL)(override abstract class HourlySuffixLzoThrift[T <: TBase[_, _]: Manifest](prefix: String, dateRange: DateRange) extends HourlySuffixSource(prefix, dateRange) with LzoThrift[T] { - override def column = manifest[T].erasure + override def column = manifest[T].runtimeClass } abstract class HourlySuffixLzoProtobuf[T <: Message: Manifest](prefix: String, dateRange: DateRange) extends HourlySuffixSource(prefix, dateRange) with LzoProtobuf[T] { - override def column = manifest[T].erasure + override def column = manifest[T].runtimeClass } abstract class HourlySuffixLzoText(prefix: String, dateRange: DateRange) 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 d3f6ea9e75..80d25fe45f 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 @@ -35,7 +35,7 @@ trait LongThriftTransformer[V <: TBase[_, _]] extends Source { val valueType = classOf[ThriftWritable[V]].asInstanceOf[Class[Writable]] override protected def transformForRead(pipe: Pipe): Pipe = { new RichPipe(pipe).mapTo(fields -> fields) { v: (LongWritable, ThriftWritable[V]) => - v._2.setConverter(mt.erasure.asInstanceOf[Class[V]]) + v._2.setConverter(mt.runtimeClass.asInstanceOf[Class[V]]) (v._1.get, v._2.get) } } @@ -46,5 +46,5 @@ trait LongThriftTransformer[V <: TBase[_, _]] extends Source { (key, value) } } - lazy val typeRef = ThriftUtils.getTypeRef(mt.erasure).asInstanceOf[TypeRef[TBase[_, _]]] + lazy val typeRef = ThriftUtils.getTypeRef(mt.runtimeClass).asInstanceOf[TypeRef[TBase[_, _]]] } 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 3fe4eab5c3..d69cfcb1ac 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 @@ -85,22 +85,22 @@ trait LzoText extends LocalTapSource with SingleMappable[String] with TypedSink[ } trait LzoTsv extends DelimitedScheme with LocalTapSource { - override def hdfsScheme = HadoopSchemeInstance(new LzoTextDelimited(fields, separator, types)) + override def hdfsScheme = HadoopSchemeInstance((new LzoTextDelimited(fields, skipHeader, writeHeader, separator, strict, quote, types, safe)).asInstanceOf[Scheme[_, _, _, _, _]]) } trait LzoTypedTsv[T] extends DelimitedScheme with Mappable[T] with TypedSink[T] with LocalTapSource { override def setter[U <: T] = TupleSetter.asSubSetter[T, U](TupleSetter.singleSetter[T]) - override def hdfsScheme = HadoopSchemeInstance(new LzoTextDelimited(fields, separator, types)) + override def hdfsScheme = HadoopSchemeInstance((new LzoTextDelimited(fields, skipHeader, writeHeader, separator, strict, quote, types, safe)).asInstanceOf[Scheme[_, _, _, _, _]]) def mf: Manifest[T] override val types: Array[Class[_]] = { - if (classOf[scala.Product].isAssignableFrom(mf.erasure)) { + if (classOf[scala.Product].isAssignableFrom(mf.runtimeClass)) { //Assume this is a Tuple: - mf.typeArguments.map { _.erasure }.toArray + mf.typeArguments.map { _.runtimeClass }.toArray } else { //Assume there is only a single item - Array(mf.erasure) + Array(mf.runtimeClass) } } } 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 e1e2adc4c9..3f700b3573 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 @@ -16,6 +16,8 @@ limitations under the License. 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 @@ -48,12 +50,12 @@ object PailSource { * SEE EXAMPLE : https://gist.github.com/krishnanraman/5224937 */ def sink[T](rootPath: String, - targetFn: (T) => List[String])(implicit cmf: ClassManifest[T], + targetFn: (T) => List[String])(implicit cmf: ClassTag[T], injection: Injection[T, Array[Byte]]): PailSource[T] = { val validator = ((x: List[String]) => true) val cps = new CodecPailStructure[T]() - cps.setParams(targetFn, validator, cmf.erasure.asInstanceOf[Class[T]], injection) + cps.setParams(targetFn, validator, cmf.runtimeClass.asInstanceOf[Class[T]], injection) sink(rootPath, cps) } @@ -70,12 +72,12 @@ object PailSource { * SEE EXAMPLE : https://gist.github.com/krishnanraman/5224937 */ def source[T](rootPath: String, - subPaths: Array[List[String]])(implicit cmf: ClassManifest[T], + subPaths: Array[List[String]])(implicit cmf: ClassTag[T], injection: Injection[T, Array[Byte]]): PailSource[T] = { val validator = ((x: List[String]) => true) val cps = new CodecPailStructure[T]() - cps.setParams(null, validator, cmf.erasure.asInstanceOf[Class[T]], injection) + cps.setParams(null, validator, cmf.runtimeClass.asInstanceOf[Class[T]], injection) source(rootPath, cps, subPaths) } @@ -102,14 +104,14 @@ object PailSource { /** * Alternate sink construction - * Using implicit injections & classmanifest for the type + * Using implicit injections & ClassTag for the type */ def sink[T](rootPath: String, targetFn: (T) => List[String], - validator: (List[String]) => Boolean)(implicit cmf: ClassManifest[T], + validator: (List[String]) => Boolean)(implicit cmf: ClassTag[T], injection: Injection[T, Array[Byte]]): PailSource[T] = { val cps = new CodecPailStructure[T]() - cps.setParams(targetFn, validator, cmf.erasure.asInstanceOf[Class[T]], injection) + cps.setParams(targetFn, validator, cmf.runtimeClass.asInstanceOf[Class[T]], injection) sink(rootPath, cps) } @@ -139,10 +141,10 @@ object PailSource { */ def source[T](rootPath: String, validator: (List[String]) => Boolean, - subPaths: Array[List[String]])(implicit cmf: ClassManifest[T], + subPaths: Array[List[String]])(implicit cmf: ClassTag[T], injection: Injection[T, Array[Byte]]): PailSource[T] = { val cps = new CodecPailStructure[T]() - cps.setParams(null, validator, cmf.erasure.asInstanceOf[Class[T]], injection) + cps.setParams(null, validator, cmf.runtimeClass.asInstanceOf[Class[T]], injection) source(rootPath, cps, subPaths) } } 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 e802db072c..5b51fb4f72 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 @@ -15,7 +15,7 @@ limitations under the License. */ package com.twitter.scalding.commons.source -import org.specs._ +import org.scalatest.{ Matchers, WordSpec } import com.twitter.scalding._ import com.twitter.bijection.Injection import com.google.common.io.Files @@ -52,9 +52,7 @@ class MoreComplexTypedWriteIncrementalJob(args: Args) extends Job(args) { .writeIncremental(VersionedKeyValSource[Int, Int]("output")) } -class VersionedKeyValSourceTest extends Specification { - noDetailedDiffs() - +class VersionedKeyValSourceTest extends WordSpec with Matchers { val input = (1 to 100).toList "A TypedWriteIncrementalJob" should { @@ -62,9 +60,9 @@ class VersionedKeyValSourceTest extends Specification { .source(TypedTsv[Int]("input"), input) .sink[(Int, Int)](VersionedKeyValSource[Array[Byte], Array[Byte]]("output")) { outputBuffer: Buffer[(Int, Int)] => "Outputs must be as expected" in { - outputBuffer.size must_== input.size + assert(outputBuffer.size === input.size) val singleInj = implicitly[Injection[Int, Array[Byte]]] - input.map{ k => (k, k) }.sortBy(_._1).toString must be_==(outputBuffer.sortBy(_._1).toList.toString) + assert(input.map{ k => (k, k) }.sortBy(_._1).toString === outputBuffer.sortBy(_._1).toList.toString) } } .run @@ -76,9 +74,9 @@ class VersionedKeyValSourceTest extends Specification { .source(TypedTsv[Int]("input"), input) .sink[(Int, Int)](VersionedKeyValSource[Array[Byte], Array[Byte]]("output")) { outputBuffer: Buffer[(Int, Int)] => "Outputs must be as expected" in { - outputBuffer.size must_== input.size + assert(outputBuffer.size === input.size) val singleInj = implicitly[Injection[Int, Array[Byte]]] - input.map{ k => (k, k) }.sortBy(_._1).toString must be_==(outputBuffer.sortBy(_._1).toList.toString) + assert(input.map{ k => (k, k) }.sortBy(_._1).toString === outputBuffer.sortBy(_._1).toList.toString) } } .run @@ -89,16 +87,15 @@ class VersionedKeyValSourceTest extends Specification { "Validate that explicitly provided versions exist" in { val path = setupLocalVersionStore(100L to 102L) - validateVersion(path, Some(103)) must throwA( - new InvalidSourceException("Version 103 does not exist. " + - "Currently available versions are: [102, 101, 100]")) + val thrown = the[InvalidSourceException] thrownBy { validateVersion(path, Some(103)) } + assert(thrown.getMessage === "Version 103 does not exist. " + + "Currently available versions are: [102, 101, 100]") // should not throw validateVersion(path, Some(101)) // should not throw validateVersion(path) - } } @@ -121,8 +118,7 @@ class VersionedKeyValSourceTest extends Specification { * Creates a VersionedKeyValSource using the provided version * and then validates it. */ - private def validateVersion(path: String, version: Option[Long] = None) = { + private def validateVersion(path: String, version: Option[Long] = None) = VersionedKeyValSource(path = path, sourceVersion = version) .validateTaps(Hdfs(false, new JobConf())) - } } diff --git a/scalding-commons/src/test/scala/com/twitter/scalding/commons/extensions/CheckpointSpec.scala b/scalding-commons/src/test/scala/com/twitter/scalding/commons/extensions/CheckpointSpec.scala index 4dc3e4aeb7..92e5c85511 100644 --- a/scalding-commons/src/test/scala/com/twitter/scalding/commons/extensions/CheckpointSpec.scala +++ b/scalding-commons/src/test/scala/com/twitter/scalding/commons/extensions/CheckpointSpec.scala @@ -17,7 +17,7 @@ limitations under the License. package com.twitter.scalding.commons.extensions import com.twitter.scalding._ -import org.specs._ +import org.scalatest.WordSpec import scala.collection.mutable.Buffer /** @@ -66,21 +66,19 @@ class TypedCheckpointJob(args: Args) extends Job(args) { out.write(TypedTsv[(Int, Int, Double)]("output")) } -class CheckpointSpec extends Specification { +class CheckpointSpec extends WordSpec { "A CheckpointJob" should { val in0 = Set((0, 0, 1), (0, 1, 1), (1, 0, 2), (2, 0, 4)) val in1 = Set((0, 1, 1), (1, 0, 2), (2, 4, 5)) val out = Set((0, 1, 2.0), (0, 0, 1.0), (1, 1, 4.0), (2, 1, 8.0)) // Verifies output when passed as a callback to JobTest.sink(). - def verifyOutput[A](expectedOutput: Set[A], actualOutput: Buffer[A]): Unit = { - val unordered = actualOutput.toSet - unordered must_== expectedOutput - } + def verifyOutput[A](expectedOutput: Set[A], actualOutput: Buffer[A]): Unit = + assert(actualOutput.toSet === expectedOutput) // Runs a test in both local test and hadoop test mode, verifies the final // output, and clears the local file set. - def runTest(test: JobTest) = { + def runTest(test: JobTest) = // runHadoop seems to have trouble with sequencefile format; use TSV. test .arg("checkpoint.format", "tsv") @@ -88,10 +86,9 @@ class CheckpointSpec extends Specification { .run .runHadoop .finish - } "run without checkpoints" in runTest { - JobTest("com.twitter.scalding.commons.extensions.CheckpointJob") + JobTest(new CheckpointJob(_)) .source(Tsv("input0"), in0) .source(Tsv("input1"), in1) } @@ -99,7 +96,7 @@ class CheckpointSpec extends Specification { "read c0, write c1 and c2" in runTest { // Adding filenames to Checkpoint.testFileSet makes Checkpoint think that // they exist. - JobTest("com.twitter.scalding.commons.extensions.CheckpointJob") + JobTest(new CheckpointJob(_)) .arg("checkpoint.file", "test") .registerFile("test_c0") .source(Tsv("test_c0"), in0) @@ -109,14 +106,14 @@ class CheckpointSpec extends Specification { } "read c2, skipping c0 and c1" in runTest { - JobTest("com.twitter.scalding.commons.extensions.CheckpointJob") + JobTest(new CheckpointJob(_)) .arg("checkpoint.file", "test") .registerFile("test_c2") .source(Tsv("test_c2"), out) } "clobber c0" in runTest { - JobTest("com.twitter.scalding.commons.extensions.CheckpointJob") + JobTest(new CheckpointJob(_)) .arg("checkpoint.file.c0", "test_c0") .arg("checkpoint.clobber", "") .registerFile("test_c0") @@ -126,7 +123,7 @@ class CheckpointSpec extends Specification { } "read c0 and clobber c1" in runTest { - JobTest("com.twitter.scalding.commons.extensions.CheckpointJob") + JobTest(new CheckpointJob(_)) .arg("checkpoint.file", "test") .arg("checkpoint.clobber.c1", "") .registerFile("test_c0") @@ -139,21 +136,19 @@ class CheckpointSpec extends Specification { } } -class TypedCheckpointSpec extends Specification { +class TypedCheckpointSpec extends WordSpec { "A TypedCheckpointJob" should { val in0 = Set((0, 0, 1), (0, 1, 1), (1, 0, 2), (2, 0, 4)) val in1 = Set((0, 1, 1), (1, 0, 2), (2, 4, 5)) val out = Set((0, 1, 2.0), (0, 0, 1.0), (1, 1, 4.0), (2, 1, 8.0)) // Verifies output when passed as a callback to JobTest.sink(). - def verifyOutput[A](expectedOutput: Set[A], actualOutput: Buffer[A]): Unit = { - val unordered = actualOutput.toSet - unordered must_== expectedOutput - } + def verifyOutput[A](expectedOutput: Set[A], actualOutput: Buffer[A]): Unit = + assert(actualOutput.toSet === expectedOutput) // Runs a test in both local test and hadoop test mode, verifies the final // output, and clears the local file set. - def runTest(test: JobTest) = { + def runTest(test: JobTest) = // runHadoop seems to have trouble with sequencefile format; use TSV. test .arg("checkpoint.format", "tsv") @@ -161,10 +156,9 @@ class TypedCheckpointSpec extends Specification { .run .runHadoop .finish - } "run without checkpoints" in runTest { - JobTest("com.twitter.scalding.commons.extensions.TypedCheckpointJob") + JobTest(new TypedCheckpointJob(_)) .source(TypedTsv[(Int, Int, Int)]("input0"), in0) .source(TypedTsv[(Int, Int, Int)]("input1"), in1) } @@ -172,7 +166,7 @@ class TypedCheckpointSpec extends Specification { "read c0, write c1 and c2" in runTest { // Adding filenames to Checkpoint.testFileSet makes Checkpoint think that // they exist. - JobTest("com.twitter.scalding.commons.extensions.TypedCheckpointJob") + JobTest(new TypedCheckpointJob(_)) .arg("checkpoint.file", "test") .registerFile("test_c0") .source(Tsv("test_c0"), in0) @@ -182,14 +176,14 @@ class TypedCheckpointSpec extends Specification { } "read c2, skipping c0 and c1" in runTest { - JobTest("com.twitter.scalding.commons.extensions.TypedCheckpointJob") + JobTest(new TypedCheckpointJob(_)) .arg("checkpoint.file", "test") .registerFile("test_c2") .source(Tsv("test_c2"), out) } "clobber c0" in runTest { - JobTest("com.twitter.scalding.commons.extensions.TypedCheckpointJob") + JobTest(new TypedCheckpointJob(_)) .arg("checkpoint.file.c0", "test_c0") .arg("checkpoint.clobber", "") .registerFile("test_c0") @@ -199,7 +193,7 @@ class TypedCheckpointSpec extends Specification { } "read c0 and clobber c1" in runTest { - JobTest("com.twitter.scalding.commons.extensions.TypedCheckpointJob") + JobTest(new TypedCheckpointJob(_)) .arg("checkpoint.file", "test") .arg("checkpoint.clobber.c1", "") .registerFile("test_c0") diff --git a/scalding-core/src/main/scala/com/twitter/package.scala b/scalding-core/src/main/scala/com/twitter/package.scala index 737ea6164c..027aa6acc1 100644 --- a/scalding-core/src/main/scala/com/twitter/package.scala +++ b/scalding-core/src/main/scala/com/twitter/package.scala @@ -33,7 +33,7 @@ package object scalding { /** * Make sure this is in sync with version.sbt */ - val scaldingVersion: String = "0.12.0" + val scaldingVersion: String = "0.13.0" object RichPathFilter { implicit def toRichPathFilter(f: PathFilter) = new RichPathFilter(f) 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 76321e5a43..e783525be8 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala @@ -40,7 +40,7 @@ import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapred.OutputCollector import org.apache.hadoop.mapred.RecordReader -import scala.util.control.Exception.allCatch +import scala.util.{ Try, Success, Failure } /** * A base class for sources that take a scheme trait. @@ -152,18 +152,21 @@ abstract class FileSource extends SchemedSource with LocalSourceOverride { case Write => CastHfsTap(new Hfs(hdfsScheme, hdfsWritePath, sinkMode)) } case _ => { - allCatch.opt( - TestTapFactory(this, hdfsScheme, sinkMode)).map { - _.createTap(readOrWrite) // these java types are invariant, so we cast here + val tryTtp = Try(TestTapFactory(this, hdfsScheme, sinkMode)).map { + // these java types are invariant, so we cast here + _.createTap(readOrWrite) + .asInstanceOf[Tap[Any, Any, Any]] + }.orElse { + Try(TestTapFactory(this, localScheme.getSourceFields, sinkMode)).map { + _.createTap(readOrWrite) .asInstanceOf[Tap[Any, Any, Any]] } - .orElse { - allCatch.opt( - TestTapFactory(this, localScheme.getSourceFields, sinkMode)).map { - _.createTap(readOrWrite) - .asInstanceOf[Tap[Any, Any, Any]] - } - }.getOrElse(sys.error("Failed to create a tap for: " + toString)) + } + + tryTtp match { + case Success(s) => s + case Failure(e) => throw new java.lang.IllegalArgumentException(s"Failed to create tap for: $toString, with error: ${e.getMessage}", e) + } } } } @@ -274,17 +277,19 @@ trait DelimitedScheme extends SchemedSource { //These should not be changed: override def localScheme = new CLTextDelimited(fields, skipHeader, writeHeader, separator, strict, quote, types, safe) - override def hdfsScheme = + override def hdfsScheme = { + assert( + types == null || fields.size == types.size, + "Fields [" + fields + "] of different size than types array [" + types.mkString(",") + "]") HadoopSchemeInstance(new CHTextDelimited(fields, null, skipHeader, writeHeader, separator, strict, quote, types, safe)) + } } trait SequenceFileScheme extends SchemedSource { //override these as needed: val fields = Fields.ALL // TODO Cascading doesn't support local mode yet - override def hdfsScheme = { - HadoopSchemeInstance(new CHSequenceFile(fields)) - } + override def hdfsScheme = HadoopSchemeInstance(new CHSequenceFile(fields)) } /** @@ -292,9 +297,8 @@ trait SequenceFileScheme extends SchemedSource { * as well as the requirements of [[FileSource.pathIsGood]] */ trait SuccessFileSource extends FileSource { - override protected def pathIsGood(p: String, conf: Configuration) = { + override protected def pathIsGood(p: String, conf: Configuration) = FileSource.globHasNonHiddenPaths(p, conf) && FileSource.globHasSuccessFile(p, conf) - } } /** 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 b98f8a87da..0b6d40409c 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/GroupBuilder.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/GroupBuilder.scala @@ -333,7 +333,7 @@ class GroupBuilder(val groupFields: Fields) extends FoldOperations[GroupBuilder] def pass: GroupBuilder = takeWhile(0) { (t: TupleEntry) => true } /** - * begining of block with access to expensive nonserializable state. The state object should + * beginning of block with access to expensive nonserializable state. The state object should * contain a function release() for resource management purpose. */ def using[C <: { def release() }](bf: => C) = new { 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 bda1b9d355..076fd3979b 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Job.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Job.scala @@ -77,6 +77,7 @@ object Job { * these functions can be combined Monadically using algebird.monad.Reader. */ class Job(val args: Args) extends FieldConversions with java.io.Serializable { + Tracing.init() // Set specific Mode implicit def mode: Mode = Mode.getMode(args).getOrElse(sys.error("No Mode defined")) @@ -156,7 +157,7 @@ class Job(val args: Args) extends FieldConversions with java.io.Serializable { /** Override this to control how dates are parsed */ implicit def dateParser: DateParser = DateParser.default - // Generated the MD5 hex of the the bytes in the job classfile + // Generated the MD5 hex of the bytes in the job classfile def classIdentifier: String = Config.md5Identifier(getClass) /** diff --git a/scalding-core/src/main/scala/com/twitter/scalding/JobTest.scala b/scalding-core/src/main/scala/com/twitter/scalding/JobTest.scala index 7528aef2ff..75f4b5897b 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/JobTest.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/JobTest.scala @@ -34,7 +34,7 @@ object JobTest { } def apply[T <: Job: Manifest] = { val cons = { (args: Args) => - manifest[T].erasure + manifest[T].runtimeClass .getConstructor(classOf[Args]) .newInstance(args) .asInstanceOf[Job] 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 cdf823bc39..a867cad1e2 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/RichPipe.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/RichPipe.scala @@ -396,7 +396,7 @@ class RichPipe(val pipe: Pipe) extends java.io.Serializable with JoinAlgorithms * * == Note == * Using mapTo is the same as using map followed by a project for - * selecting just the ouput fields + * selecting just the output fields */ def map[A, T](fs: (Fields, Fields))(fn: A => T)(implicit conv: TupleConverter[A], setter: TupleSetter[T]): Pipe = { conv.assertArityMatches(fs._1) 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 07fc94efbf..b7f98a5057 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Source.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Source.scala @@ -105,7 +105,7 @@ abstract class Source extends java.io.Serializable { */ val uuid = java.util.UUID.randomUUID val srcName = sourceId + uuid.toString - assert(!sources.containsKey(srcName), "Source %s had collision in uuid: %".format(this, uuid)) + assert(!sources.containsKey(srcName), "Source %s had collision in uuid: %s".format(this, uuid)) sources.put(srcName, createTap(Read)(mode)) FlowStateMap.mutate(flowDef) { st => (st.addSource(srcName, this), ()) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Tracing.scala b/scalding-core/src/main/scala/com/twitter/scalding/Tracing.scala new file mode 100644 index 0000000000..4f0adc7b6f --- /dev/null +++ b/scalding-core/src/main/scala/com/twitter/scalding/Tracing.scala @@ -0,0 +1,94 @@ +/* +Copyright 2012 Twitter, Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + +http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package com.twitter.scalding + +import java.lang.reflect.InvocationTargetException + +import org.slf4j.{ Logger, LoggerFactory => LogManager } + +/** + * Calling init registers "com.twitter.scalding" as a "tracing boundary" for + * Cascading. That means that when Cascading sends trace information to + * a DocumentService such as Driven, the trace will have information about + * the caller of Scalding instead of about the internals of Scalding. + * com.twitter.scalding.Job and its subclasses will automatically + * initialize Tracing. + * + * register and unregister methods are provided for testing, but + * should not be needed for most development + */ +object Tracing { + private val LOG: Logger = LogManager.getLogger(this.getClass) + + // TODO: remove this once we no longer want backwards compatiblity + // with cascading versions pre 2.6 + private val traceUtilClassName = "cascading.util.TraceUtil" + + /** + * Put a barrier at com.twitter.scalding, but exclude things like Tool + * that are common entry points for calling user code + */ + private val defaultRegex = """^com\.twitter\.scalding\.(?!Tool|Job|ExecutionContext).*""" + + register() + + /** + * Forces the initialization of the Tracing object which in turn causes + * the one time registration of "com.twitter.scalding" as a + * tracing boundary in Cascading + */ + def init() { /* do nothing */ } + + /** + * Explicitly registers "com.twitter.scalding" as a Cascading + * tracing boundary. Normally not needed, but may be useful + * after a call to unregister() + */ + def register(regex: String = defaultRegex) = invokeStaticMethod(traceUtilClassName, "registerApiBoundary", regex) + + /** + * Unregisters "com.twitter.scalding" as a Cascading + * tracing bounardy. After calling this, Cascading DocumentServices + * such as Driven will show nodes as being created by Scalding + * class such as RichPipe instead of end user written code. This + * should normally not be called but can be useful in testing + * the development of Scalding internals + */ + def unregister(regex: String = defaultRegex) = invokeStaticMethod(traceUtilClassName, "unregisterApiBoundary", regex) + + /** + * Use reflection to register/unregister tracing boundaries so that cascading versions prior to 2.6 can be used + * without completely breaking + */ + private def invokeStaticMethod(clazz: String, methodName: String, args: AnyRef*) { + try { + val argTypes = args map (_.getClass()) + Class.forName(clazz).getMethod(methodName, argTypes: _*).invoke(null, args: _*) + } catch { + case e @ (_: NoSuchMethodException | + _: SecurityException | + _: IllegalAccessException | + _: IllegalArgumentException | + _: InvocationTargetException | + _: NullPointerException | + _: ClassNotFoundException) => LOG.warn("There was an error initializing tracing. " + + "Tracing information in DocumentServices such as Driven may point to Scalding code instead of " + + "user code. The most likely cause is a mismatch in Cascading library version. Upgrading the " + + "Cascading library to at least 2.6 should fix this issue.The cause was [" + e + "]") + } + } +} \ No newline at end of file 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 74335e8c75..efd6547f94 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/TuplePacker.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/TuplePacker.scala @@ -69,12 +69,12 @@ class ReflectionTupleConverter[T](fields: Fields)(implicit m: Manifest[T]) exten def validate { //We can't touch setters because that shouldn't be accessed until map/reduce side, not //on submitter. - val missing = Dsl.asList(fields).filter { f => !getSetters.contains(f.toString) }.headOption + val missing = Dsl.asList(fields).find { f => !getSetters.contains(f.toString) } assert(missing.isEmpty, "Field: " + missing.get.toString + " not in setters") } validate - def getSetters = m.erasure + def getSetters = m.runtimeClass .getDeclaredMethods .filter { _.getName.startsWith("set") } .groupBy { setterToFieldName(_) } @@ -86,7 +86,7 @@ class ReflectionTupleConverter[T](fields: Fields)(implicit m: Manifest[T]) exten lazy val setters = getSetters override def apply(input: TupleEntry): T = { - val newInst = m.erasure.newInstance() + val newInst = m.runtimeClass.newInstance() val fields = input.getFields (0 until fields.size).map { idx => val thisField = fields.get(idx) @@ -108,7 +108,7 @@ class OrderedConstructorConverter[T](fields: Fields)(implicit mf: Manifest[T]) e override val arity = fields.size // Keep this as a method, so we can validate by calling, but don't serialize it, and keep it lazy // below - def getConstructor = mf.erasure + def getConstructor = mf.runtimeClass .getConstructors .filter { _.getParameterTypes.size == fields.size } .head.asInstanceOf[Constructor[T]] 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 c15cbb6570..82fa09bb20 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/TupleUnpacker.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/TupleUnpacker.scala @@ -77,7 +77,7 @@ class ReflectionTupleUnpacker[T](implicit m: Manifest[T]) extends TupleUnpacker[ // A Fields object representing all of m's // fields, in the declared field order. // Lazy because we need this twice or not at all. - lazy val allFields = new Fields(ReflectionUtils.fieldsOf(m.erasure).toSeq: _*) + lazy val allFields = new Fields(ReflectionUtils.fieldsOf(m.runtimeClass).toSeq: _*) /** * A helper to check the passed-in @@ -105,7 +105,7 @@ class ReflectionSetter[T](fields: Fields)(implicit m: Manifest[T]) extends Tuple // Methods and Fields are not serializable so we // make these defs instead of vals // TODO: filter by isAccessible, which somehow seems to fail - def methodMap = m.erasure + def methodMap = m.runtimeClass .getDeclaredMethods // Keep only methods with 0 parameter types .filter { m => m.getParameterTypes.length == 0 } @@ -113,7 +113,7 @@ class ReflectionSetter[T](fields: Fields)(implicit m: Manifest[T]) extends Tuple .mapValues { _.head } // TODO: filter by isAccessible, which somehow seems to fail - def fieldMap = m.erasure + def fieldMap = m.runtimeClass .getDeclaredFields .groupBy { _.getName } .mapValues { _.head } @@ -141,7 +141,7 @@ class ReflectionSetter[T](fields: Fields)(implicit m: Manifest[T]) extends Tuple .orElse(getValueFromMethod(fieldName)) .orElse(getValueFromField(fieldName)) .getOrElse( - throw new TupleUnpackerException("Unrecognized field: " + fieldName + " for class: " + m.erasure.getName)) + throw new TupleUnpackerException("Unrecognized field: " + fieldName + " for class: " + m.runtimeClass.getName)) } private def getValueFromField(fieldName: String): Option[(T => AnyRef)] = { diff --git a/scalding-core/src/main/scala/com/twitter/scalding/TypeDescriptor.scala b/scalding-core/src/main/scala/com/twitter/scalding/TypeDescriptor.scala new file mode 100644 index 0000000000..ae20dd88f9 --- /dev/null +++ b/scalding-core/src/main/scala/com/twitter/scalding/TypeDescriptor.scala @@ -0,0 +1,32 @@ + +/* +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 + +import cascading.tuple.Fields + +/** + * This class is used to bind together a Fields instance which may contain a type array via getTypes, + * a TupleConverter and TupleSetter, which are inverses of one another. Note the size of the Fields + * object and the arity values for the converter and setter are all the same. Note in the scalding-macros + * package there are macros to generate this for case classes, which may be very convenient. + */ + +trait TypeDescriptor[T] extends java.io.Serializable { + def setter: TupleSetter[T] + def converter: TupleConverter[T] + def fields: Fields +} 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 d7064ba7cd..e9ea62b217 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/TypedDelimited.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/TypedDelimited.scala @@ -108,12 +108,12 @@ trait TypedDelimited[T] extends DelimitedScheme override def setter[U <: T] = TupleSetter.asSubSetter[T, U](tset) override val types: Array[Class[_]] = - if (classOf[scala.Product].isAssignableFrom(mf.erasure)) { + if (classOf[scala.Product].isAssignableFrom(mf.runtimeClass)) { //Assume this is a Tuple: - mf.typeArguments.map { _.erasure }.toArray + mf.typeArguments.map { _.runtimeClass }.toArray } else { //Assume there is only a single item - Array(mf.erasure) + Array(mf.runtimeClass) } // This is used to add types to a Field, which Cascading now supports. While we do not do this much generally diff --git a/scalding-core/src/main/scala/com/twitter/scalding/WritableSequenceFile.scala b/scalding-core/src/main/scala/com/twitter/scalding/WritableSequenceFile.scala index dddb36865f..182829c61f 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/WritableSequenceFile.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/WritableSequenceFile.scala @@ -34,7 +34,7 @@ trait WritableSequenceFileScheme extends SchemedSource { // TODO Cascading doesn't support local mode yet override def hdfsScheme = - HadoopSchemeInstance(new CHWritableSequenceFile(fields, keyType, valueType)) + HadoopSchemeInstance(new CHWritableSequenceFile(fields, keyType, valueType).asInstanceOf[cascading.scheme.Scheme[_, _, _, _, _]]) } object WritableSequenceFile { @@ -54,8 +54,8 @@ case class WritableSequenceFile[K <: Writable: Manifest, V <: Writable: Manifest with TypedSource[(K, V)] { override val fields = f - override val keyType = manifest[K].erasure.asInstanceOf[Class[_ <: Writable]] - override val valueType = manifest[V].erasure.asInstanceOf[Class[_ <: Writable]] + override val keyType = manifest[K].runtimeClass.asInstanceOf[Class[_ <: Writable]] + override val valueType = manifest[V].runtimeClass.asInstanceOf[Class[_ <: Writable]] def setter[U <: (K, V)]: TupleSetter[U] = TupleSetter.asSubSetter[(K, V), U](TupleSetter.tup2Setter[(K, V)]) @@ -83,8 +83,8 @@ case class MultipleWritableSequenceFiles[K <: Writable: Manifest, V <: Writable: with TypedSource[(K, V)] { override val fields = f - override val keyType = manifest[K].erasure.asInstanceOf[Class[_ <: Writable]] - override val valueType = manifest[V].erasure.asInstanceOf[Class[_ <: Writable]] + override val keyType = manifest[K].runtimeClass.asInstanceOf[Class[_ <: Writable]] + override val valueType = manifest[V].runtimeClass.asInstanceOf[Class[_ <: Writable]] def converter[U >: (K, V)]: TupleConverter[U] = TupleConverter.asSuperConverter(TupleConverter.tuple2Converter[K, V]) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/examples/PageRank.scala b/scalding-core/src/main/scala/com/twitter/scalding/examples/PageRank.scala index 990978e79a..66f16598c0 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/examples/PageRank.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/examples/PageRank.scala @@ -6,7 +6,7 @@ import com.twitter.scalding._ /** * Options: * --input: the three column TSV with node, comma-sep-out-neighbors, initial pagerank (set to 1.0 first) - * --ouput: the name for the TSV you want to write to, same as above. + * --output: the name for the TSV you want to write to, same as above. * optional arguments: * --errorOut: name of where to write the L1 error between the input page-rank and the output * if this is omitted, we don't compute the error 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 b283bf3e85..7f2be86a29 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 @@ -177,7 +177,7 @@ sealed trait Matrix2[R, C, V] extends Serializable { } /** - * This trait allows users to plug in join algoritms + * This trait allows users to plug in join algorithms * where they are needed to improve products and propagations. * The default works well in most cases, but highly skewed matrices may need some * special handling diff --git a/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Poisson.scala b/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Poisson.scala index dc98cccace..e9112f983f 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Poisson.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Poisson.scala @@ -4,7 +4,7 @@ import scala.util.Random /** * Generating Poisson-distributed random variables - * according to Donald Knuth's algorith as shown on Wikipedia's + * according to Donald Knuth's algorithm as shown on Wikipedia's * Poisson Distribution page */ @@ -22,4 +22,4 @@ class Poisson(fraction: Double, seed: Int) { } while (p > L) k - 1 } -} \ No newline at end of file +} diff --git a/scalding-core/src/main/scala/com/twitter/scalding/source/MaxFailuresCheck.scala b/scalding-core/src/main/scala/com/twitter/scalding/source/MaxFailuresCheck.scala index 4abd0b3cd5..5f602c5c66 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/source/MaxFailuresCheck.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/source/MaxFailuresCheck.scala @@ -28,7 +28,7 @@ class MaxFailuresCheck[T, U](val maxFailures: Int)(implicit override val injecti try { Some(injection.invert(input).get) } catch { - case e => + case e: Exception => // TODO: use proper logging e.printStackTrace() assert( 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 260696882d..c25f0f7ca9 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 @@ -189,7 +189,7 @@ case class IdentityReduce[K, V1]( override lazy val toTypedPipe = reducers match { case None => mapped // free case case Some(reds) => - // This is wierd, but it is sometimes used to force a partition + // This is weird, but it is sometimes used to force a partition groupOp { _.reducers(reds) } } @@ -259,7 +259,7 @@ case class UnsortedIdentityReduce[K, V1]( override lazy val toTypedPipe = reducers match { case None => mapped // free case case Some(reds) => - // This is wierd, but it is sometimes used to force a partition + // This is weird, but it is sometimes used to force a partition groupOp { _.reducers(reds) } } 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 252ccf9027..aab6db8429 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 @@ -124,7 +124,7 @@ trait KeyedListLike[K, +T, +This[K, +T] <: KeyedListLike[K, T, This]] */ def aggregate[B, C](agg: Aggregator[T, B, C]): This[K, C] = mapValues[B](agg.prepare(_)) - .reduce[B](agg.reduce _) + .sum[B](agg.semigroup) .mapValues[C](agg.present(_)) /** 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 b6e63f01fb..1ada8894e8 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 @@ -30,7 +30,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 - * 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)). @@ -38,7 +38,7 @@ import java.io.Serializable * 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. * @@ -54,7 +54,11 @@ import java.io.Serializable * else, the service will return Some(joinedV). */ object LookupJoin extends Serializable { - def apply[T: Ordering, K: Ordering, V, JoinedV](left: TypedPipe[(T, (K, V))], right: TypedPipe[(T, (K, JoinedV))]): TypedPipe[(T, (K, (V, Option[JoinedV])))] = { + 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])))] = { + /** * Implicit ordering on an either that doesn't care about the * actual container values, puts the lookups before the service @@ -75,6 +79,7 @@ object LookupJoin extends Serializable { 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])) }) .group + .withReducers(reducers.getOrElse(-1)) // -1 means default in scalding .sortBy(identity) // time then left before right /** * Grouping by K leaves values of (T, Either[V, JoinedV]). Sort 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 f50e0b3d78..554e8a0bb5 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 @@ -57,12 +57,12 @@ case class PartitionedDelimitedSource[P, T]( "The number of fields needs to be the same as the arity of the value setter") val types: Array[Class[_]] = { - if (classOf[scala.Product].isAssignableFrom(mt.erasure)) { + if (classOf[scala.Product].isAssignableFrom(mt.runtimeClass)) { //Assume this is a Tuple: - mt.typeArguments.map { _.erasure }.toArray + mt.typeArguments.map { _.runtimeClass }.toArray } else { //Assume there is only a single item - Array(mt.erasure) + Array(mt.runtimeClass) } } 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 2f3041f1c9..47783a71b6 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 @@ -15,8 +15,38 @@ limitations under the License. */ package com.twitter.scalding.typed -import com.twitter.algebird.{ CMS, MurmurHash128 } +import com.twitter.algebird.{ CMS, CMSHasher } + +object Sketched { + + // TODO: there are more efficient orderings we could use here if this turns + // out to be a bottleneck, and this should actually never end up gettings used. + // We may be able to remove this after some refactoring in Algebird. + implicit val byteArrayOrdering = Ordering.by((_: Array[Byte]).toIterable) + + /** + * This is based on the CMSHasherBigInt found in algebird (see docs for in depth explanation): + * https://github.com/twitter/algebird/blob/develop/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala#L1086 + * + * TODO: We need to move this hasher to CMSHasherImplicits in algebird: + * https://github.com/twitter/algebird/blob/develop/algebird-core/src/main/scala/com/twitter/algebird/CountMinSketch.scala#L1054 + * See: https://github.com/twitter/scalding/issues/1177 + */ + implicit object CMSHasherByteArray extends CMSHasher[Array[Byte]] { + override def hash(a: Int, b: Int, width: Int)(x: Array[Byte]): Int = { + val hash: Int = scala.util.hashing.MurmurHash3.arrayHash(x, a) + // We only want positive integers for the subsequent modulo. This method mimics Java's Hashtable + // implementation. The Java code uses `0x7FFFFFFF` for the bit-wise AND, which is equal to Int.MaxValue. + val positiveHash = hash & Int.MaxValue + positiveHash % width + } + } +} +/** + * This class is generally only created by users + * with the TypedPipe.sketch method + */ case class Sketched[K, V](pipe: TypedPipe[(K, V)], numReducers: Int, delta: Double, @@ -24,25 +54,37 @@ case class Sketched[K, V](pipe: TypedPipe[(K, V)], seed: Int)(implicit serialization: K => Array[Byte], ordering: Ordering[K]) extends HasReducers { + import Sketched._ - val reducers = Some(numReducers) + def serialize(k: K): Array[Byte] = serialization(k) - private lazy val murmurHash = MurmurHash128(seed) - def hash(key: K): Long = murmurHash(serialization(key))._1 + val reducers = Some(numReducers) - private lazy implicit val cms = CMS.monoid(eps, delta, seed) - lazy val sketch: TypedPipe[CMS] = + private lazy implicit val cms = CMS.monoid[Array[Byte]](eps, delta, seed) + lazy val sketch: TypedPipe[CMS[Array[Byte]]] = pipe - .map{ kv => cms.create(hash(kv._1)) } + .map { case (k, v) => cms.create(serialization(k)) } .groupAll .sum .values .forceToDisk + /** + * Like a hashJoin, this joiner does not see all the values V at one time, only one at a time. + * This is sufficient to implement join and leftJoin + */ def cogroup[V2, R](right: TypedPipe[(K, V2)])(joiner: (K, V, Iterable[V2]) => Iterator[R]): SketchJoined[K, V, V2, R] = new SketchJoined(this, right, numReducers)(joiner) + /** + * Does a logical inner join but replicates the heavy keys of the left hand side + * across the reducers + */ def join[V2](right: TypedPipe[(K, V2)]) = cogroup(right)(Joiner.hashInner2) + /** + * Does a logical left join but replicates the heavy keys of the left hand side + * across the reducers + */ def leftJoin[V2](right: TypedPipe[(K, V2)]) = cogroup(right)(Joiner.hashLeft2) } @@ -60,7 +102,7 @@ case class SketchJoined[K: Ordering, V, V2, R](left: Sketched[K, V], pipe.cross(left.sketch).flatMap{ case (v, cms) => val maxPerReducer = (cms.totalCount / numReducers) * maxReducerFraction + 1 - val maxReplicas = (cms.frequency(left.hash(v._1)).estimate.toDouble / maxPerReducer) + val maxReplicas = (cms.frequency(left.serialize(v._1)).estimate.toDouble / maxPerReducer) //if the frequency is 0, maxReplicas.ceil will be 0 so we will filter out this key entirely //if it's < maxPerReducer, the ceil will round maxReplicas up to 1 to ensure we still see it 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 be05ecb920..48b6b0eb19 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 @@ -40,28 +40,46 @@ import scala.concurrent.Future object TypedPipe extends Serializable { import Dsl.flowDefToRichFlowDef + /** + * Create a TypedPipe from a cascading Pipe, some Fields and the type T + * Avoid this if you can. Prefer from(TypedSource). + */ def from[T](pipe: Pipe, fields: Fields)(implicit flowDef: FlowDef, mode: Mode, conv: TupleConverter[T]): TypedPipe[T] = { val localFlow = flowDef.onlyUpstreamFrom(pipe) new TypedPipeInst[T](pipe, fields, localFlow, mode, Converter(conv)) } + /** + * Create a TypedPipe from a TypedSource. This is the preferred way to make a TypedPipe + */ def from[T](source: TypedSource[T]): TypedPipe[T] = TypedPipeFactory({ (fd, mode) => val pipe = source.read(fd, mode) from(pipe, source.sourceFields)(fd, mode, source.converter) }) - // It might pay to use a view here, but you should experiment + /** + * Create a TypedPipe from an Iterable in memory. + */ def from[T](iter: Iterable[T]): TypedPipe[T] = IterablePipe[T](iter) - /** Input must be a Pipe with exactly one Field */ + /** + * Input must be a Pipe with exactly one Field + * Avoid this method and prefer from(TypedSource) if possible + */ def fromSingleField[T](pipe: Pipe)(implicit fd: FlowDef, mode: Mode): TypedPipe[T] = from(pipe, new Fields(0))(fd, mode, singleConverter[T]) + /** + * Create an empty TypedPipe. This is sometimes useful when a method must return + * a TypedPipe, but sometimes at runtime we can check a condition and see that + * it should be empty. + * This is the zero of the Monoid[TypedPipe] + */ def empty: TypedPipe[Nothing] = EmptyTypedPipe - /* + /** * This enables pipe.hashJoin(that) or pipe.join(that) syntax * This is a safe enrichment because hashJoinable and CoGroupable are * only used in the argument position or to give cogroup, join, leftJoin, rightJoin, outerJoin @@ -97,14 +115,32 @@ object TypedPipe extends Serializable { */ trait TypedPipe[+T] extends Serializable { - // Implements a cross product. The right side should be tiny + /** + * Implements a cross product. The right side should be tiny + * This gives the same results as + * {code for { l <- list1; l2 <- list2 } yield (l, l2) } + */ def cross[U](tiny: TypedPipe[U]): TypedPipe[(T, U)] + /** + * This is the fundamental mapper operation. + * It behaves in a way similar to List.flatMap, which means that each + * item is fed to the input function, which can return 0, 1, or many outputs + * (as a TraversableOnce) per input. The returned results will be iterated through once + * and then flattened into a single TypedPipe which is passed to the next step in the + * pipeline. + * + * This behavior makes it a powerful operator -- it can be used to filter records + * (by returning 0 items for a given input), it can be used the way map is used + * (by returning 1 item per input), it can be used to explode 1 input into many outputs, + * or even a combination of all of the above at once. + */ def flatMap[U](f: T => TraversableOnce[U]): TypedPipe[U] /** * Export back to a raw cascading Pipe. useful for interop with the scalding * Fields API or with Cascading code. + * Avoid this if possible. Prefer to write to TypedSink. */ def toPipe[U >: T](fieldNames: Fields)(implicit flowDef: FlowDef, mode: Mode, setter: TupleSetter[U]): Pipe @@ -128,6 +164,11 @@ trait TypedPipe[+T] extends Serializable { } /** + * Aggregate all items in this pipe into a single ValuePipe + * + * Aggregators are composable reductions that allow you to glue together + * several reductions and process them in one pass. + * * Same as groupAll.aggregate.values */ def aggregate[B, C](agg: Aggregator[T, B, C]): ValuePipe[C] = @@ -171,6 +212,17 @@ trait TypedPipe[+T] extends Serializable { /** * Returns the set of distinct elements in the TypedPipe + * This is the same as: .map((_, ())).group.sum.keys + * If you want a distinct while joining, consider: + * instead of: + * {@code + * a.join(b.distinct.asKeys) + * } + * manually do the distinct: + * {@code + * a.join(b.asKeys.sum) + * } + * The latter creates 1 map/reduce phase rather than 2 */ @annotation.implicitNotFound(msg = "For distinct method to work, the type in TypedPipe must have an Ordering.") def distinct(implicit ord: Ordering[_ >: T]): TypedPipe[T] = @@ -221,9 +273,13 @@ trait TypedPipe[+T] extends Serializable { def fork: TypedPipe[T] = onRawSingle(identity) /** - * limit the output to at most count items. + * WARNING This is dangerous, and may not be what you think. + * + * limit the output to AT MOST count items. * useful for debugging, but probably that's about it. - * The number may be less than count, and not sampled particular method + * The number may be less than count, and not sampled by any particular method + * + * This may change in the future to be exact, but that will add 1 MR step */ def limit(count: Int): TypedPipe[T] = onRawSingle(_.limit(count)) @@ -240,8 +296,13 @@ trait TypedPipe[+T] extends Serializable { def filter(f: T => Boolean): TypedPipe[T] = flatMap { t => if (f(t)) Iterator(t) else Iterator.empty } + // This is just to appease for comprehension + def withFilter(f: T => Boolean): TypedPipe[T] = filter(f) + /** * If T is a (K, V) for some V, then we can use this function to filter. + * Prefer to use this if your filter only touches the key. + * * This is here to match the function in KeyedListLike, where it is optimized */ def filterKeys[K](fn: K => Boolean)(implicit ev: T <:< (K, Any)): TypedPipe[T] = @@ -276,6 +337,8 @@ trait TypedPipe[+T] extends Serializable { /** * Force a materialization of this pipe prior to the next operation. * This is useful if you filter almost everything before a hashJoin, for instance. + * This is useful for experts who see some heuristic of the planner causing + * slower performance. */ def forceToDisk: TypedPipe[T] = onRawSingle(_.forceToDisk) @@ -325,7 +388,16 @@ trait TypedPipe[+T] extends Serializable { } private[this] def defaultSeed: Long = System.identityHashCode(this) * 2654435761L ^ System.currentTimeMillis + /** + * Sample uniformly independently at random each element of the pipe + * does not require a reduce step. + */ def sample(percent: Double): TypedPipe[T] = sample(percent, defaultSeed) + /** + * Sample uniformly independently at random each element of the pipe with + * a given seed. + * Does not require a reduce step. + */ def sample(percent: Double, seed: Long): TypedPipe[T] = { // Make sure to fix the seed, otherwise restarts cause subtle errors val rand = new Random(seed) @@ -365,8 +437,8 @@ trait TypedPipe[+T] extends Serializable { groupRandomly(partitions).forceToReducers.values /** - * Reasonably common shortcut for cases of associative/commutative reduction - * returns a typed pipe with only one element. + * Reasonably common shortcut for cases of total associative/commutative reduction + * returns a ValuePipe with only one element if there is any input, otherwise EmptyValue. */ def sum[U >: T](implicit plus: Semigroup[U]): ValuePipe[U] = ComputedValue(groupAll.sum[U].values) @@ -376,7 +448,12 @@ trait TypedPipe[+T] extends Serializable { def sumByKey[K, V](implicit ev: T <:< (K, V), ord: Ordering[K], plus: Semigroup[V]): UnsortedGrouped[K, V] = group[K, V].sum[V] - /* + /** + * This is used when you are working with Execution[T] to create loops. + * You might do this to checkpoint and then flatMap Execution to continue + * from there. Probably only useful if you need to flatMap it twice to fan + * out the data into two children jobs. + * * This writes the current TypedPipe into a temporary file * and then opens it after complete so that you can continue from that point */ @@ -404,6 +481,14 @@ trait TypedPipe[+T] extends Serializable { } } + /** + * This gives an Execution that when run evaluates the TypedPipe, + * writes it to disk, and then gives you an Iterable that reads from + * disk on the submit node each time .iterator is called. + * Because of how scala Iterables work, mapping/flatMapping/filtering + * the Iterable forces a read of the entire thing. If you need it to + * be lazy, call .iterator and use the Iterator inside instead. + */ def toIterableExecution: Execution[Iterable[T]] /** use a TupleUnpacker to flatten U out into a cascading Tuple */ @@ -412,6 +497,13 @@ trait TypedPipe[+T] extends Serializable { toPipe[U](fieldNames)(fd, mode, setter) } + /** + * This attaches a function that is called at the end of the map phase on + * EACH of the tasks that are executing. + * This is for expert use only. You probably won't ever need it. Try hard + * to avoid it. Execution also has onComplete that can run when an Execution + * has completed. + */ def onComplete(fn: () => Unit): TypedPipe[T] = new WithOnComplete[T](this, fn) /** @@ -428,7 +520,8 @@ trait TypedPipe[+T] extends Serializable { /** * This is the functionally pure approach to building jobs. Note, - * that you have to call run on the result for anything to happen here. + * that you have to call run on the result or flatMap/zip it + * into an Execution that is run for anything to happen here. */ def writeExecution(dest: TypedSink[T]): Execution[Unit] = Execution.fromFn { (conf: Config, m: Mode) => @@ -474,15 +567,45 @@ trait TypedPipe[+T] extends Serializable { def leftCross[V](thatPipe: TypedPipe[V]): TypedPipe[(T, Option[V])] = map(((), _)).hashLeftJoin(thatPipe.groupAll).values - /** common pattern of attaching a value and then map */ + /** + * common pattern of attaching a value and then map + * recommended style: + * {@code + * mapWithValue(vpu) { + * case (t, Some(u)) => op(t, u) + * case (t, None) => // if you never expect this: + * sys.error("unexpected empty value pipe") + * } + * } + */ def mapWithValue[U, V](value: ValuePipe[U])(f: (T, Option[U]) => V): TypedPipe[V] = leftCross(value).map(t => f(t._1, t._2)) - /** common pattern of attaching a value and then flatMap */ + /** + * common pattern of attaching a value and then flatMap + * recommended style: + * {@code + * flatMapWithValue(vpu) { + * case (t, Some(u)) => op(t, u) + * case (t, None) => // if you never expect this: + * sys.error("unexpected empty value pipe") + * } + * } + */ def flatMapWithValue[U, V](value: ValuePipe[U])(f: (T, Option[U]) => TraversableOnce[V]): TypedPipe[V] = leftCross(value).flatMap(t => f(t._1, t._2)) - /** common pattern of attaching a value and then filter */ + /** + * common pattern of attaching a value and then filter + * recommended style: + * {@code + * filterWithValue(vpu) { + * case (t, Some(u)) => op(t, u) + * case (t, None) => // if you never expect this: + * sys.error("unexpected empty value pipe") + * } + * } + */ def filterWithValue[U](value: ValuePipe[U])(f: (T, Option[U]) => Boolean): TypedPipe[T] = leftCross(value).filter(t => f(t._1, t._2)).map(_._1) @@ -515,7 +638,21 @@ trait TypedPipe[+T] extends Serializable { .hashLeftJoin(grouped) .map { case (t, (_, optV)) => (t, optV) } - /** Build a sketch of this TypedPipe so that you can do a skew-join with another Grouped */ + /** + * Enables joining when this TypedPipe has some keys with many many values and + * but many with very few values. For instance, a graph where some nodes have + * millions of neighbors, but most have only a few. + * + * We build a (count-min) sketch of each key's frequency, and we use that + * to shard the heavy keys across many reducers. + * This increases communication cost in order to reduce the maximum time needed + * to complete the join. + * + * {@code pipe.sketch(100).join(thatPipe) } + * will add an extra map/reduce job over a standard join to create the count-min-sketch. + * This will generally only be beneficial if you have really heavy skew, where without + * this you have 1 or 2 reducers taking hours longer than the rest. + */ def sketch[K, V](reducers: Int, eps: Double = 1.0E-5, //272k width = 1MB per row delta: Double = 0.01, //5 rows (= 5 hashes) @@ -524,7 +661,9 @@ trait TypedPipe[+T] extends Serializable { ordering: Ordering[K]): Sketched[K, V] = Sketched(ev(this), reducers, delta, eps, seed) - // If any errors happen below this line, but before a groupBy, write to a TypedSink + /** + * If any errors happen below this line, but before a groupBy, write to a TypedSink + */ def addTrap[U >: T](trapSink: Source with TypedSink[T])(implicit conv: TupleConverter[U]): TypedPipe[U] = TypedPipeFactory({ (flowDef, mode) => val fields = trapSink.sinkFields @@ -535,6 +674,9 @@ trait TypedPipe[+T] extends Serializable { }) } +/** + * This object is the EmptyTypedPipe. Prefer to create it with TypedPipe.empty + */ final case object EmptyTypedPipe extends TypedPipe[Nothing] { import Dsl._ @@ -553,14 +695,8 @@ final case object EmptyTypedPipe extends TypedPipe[Nothing] { override def leftCross[V](p: ValuePipe[V]) = this - /** - * limit the output to at most count items. - * useful for debugging, but probably that's about it. - * The number may be less than count, and not sampled particular method - */ override def limit(count: Int) = this - // prints the current pipe to either stdout or stderr override def debug: TypedPipe[Nothing] = this override def ++[U >: Nothing](other: TypedPipe[U]): TypedPipe[U] = other @@ -568,7 +704,7 @@ final case object EmptyTypedPipe extends TypedPipe[Nothing] { override def toPipe[U >: Nothing](fieldNames: Fields)(implicit fd: FlowDef, mode: Mode, setter: TupleSetter[U]): Pipe = IterableSource(Iterable.empty, fieldNames)(setter, singleConverter[U]).read(fd, mode) - def toIterableExecution: Execution[Iterable[Nothing]] = Execution.from(Iterable.empty) + override def toIterableExecution: Execution[Iterable[Nothing]] = Execution.from(Iterable.empty) override def forceToDiskExecution: Execution[TypedPipe[Nothing]] = Execution.from(this) @@ -581,7 +717,8 @@ final case object EmptyTypedPipe extends TypedPipe[Nothing] { } /** - * You should use a view here + * Creates a TypedPipe from an Iterable[T]. Prefer TypedPipe.from. + * * If you avoid toPipe, this class is more efficient than IterableSource. */ final case class IterablePipe[T](iterable: Iterable[T]) extends TypedPipe[T] { @@ -599,7 +736,6 @@ final case class IterablePipe[T](iterable: Iterable[T]) extends TypedPipe[T] { case _ => MergedTypedPipe(this, other) } - // Implements a cross product. override def cross[U](tiny: TypedPipe[U]) = tiny.flatMap { u => iterable.map { (_, u) } } @@ -659,9 +795,12 @@ final case class IterablePipe[T](iterable: Iterable[T]) extends TypedPipe[T] { TypedPipe.from( IterableSource[T](iterable, new Fields("0"))(singleSetter, singleConverter)) - def toIterableExecution: Execution[Iterable[T]] = Execution.from(iterable) + override def toIterableExecution: Execution[Iterable[T]] = Execution.from(iterable) } +/** + * This is an implementation detail (and should be marked private) + */ object TypedPipeFactory { def apply[T](next: (FlowDef, Mode) => TypedPipe[T]): TypedPipeFactory[T] = { val memo = new java.util.WeakHashMap[FlowDef, (Mode, TypedPipe[T])]() @@ -696,9 +835,9 @@ class TypedPipeFactory[T] private (@transient val next: NoStackAndThen[(FlowDef, private[this] def andThen[U](fn: TypedPipe[T] => TypedPipe[U]): TypedPipe[U] = new TypedPipeFactory(next.andThen(fn)) - def cross[U](tiny: TypedPipe[U]) = andThen(_.cross(tiny)) + override def cross[U](tiny: TypedPipe[U]) = andThen(_.cross(tiny)) override def filter(f: T => Boolean): TypedPipe[T] = andThen(_.filter(f)) - def flatMap[U](f: T => TraversableOnce[U]): TypedPipe[U] = andThen(_.flatMap(f)) + override def flatMap[U](f: T => TraversableOnce[U]): TypedPipe[U] = andThen(_.flatMap(f)) override def map[U](f: T => U): TypedPipe[U] = andThen(_.map(f)) override def limit(count: Int) = andThen(_.limit(count)) @@ -706,11 +845,11 @@ 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]) - def toPipe[U >: T](fieldNames: Fields)(implicit flowDef: FlowDef, mode: Mode, setter: TupleSetter[U]) = + override def toPipe[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) - def toIterableExecution: Execution[Iterable[T]] = Execution.factory { (conf, mode) => + 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 @@ -755,7 +894,6 @@ class TypedPipeInst[T] private[scalding] (@transient inpipe: Pipe, assert(m == mode, "Cannot switch Mode between TypedSource.read and toPipe calls. Pipe: %s, call: %s".format(mode, m)) - // Implements a cross product. The right side should be tiny (< 100MB) override def cross[U](tiny: TypedPipe[U]): TypedPipe[(T, U)] = tiny match { case EmptyTypedPipe => EmptyTypedPipe case MergedTypedPipe(l, r) => MergedTypedPipe(cross(l), cross(r)) @@ -774,6 +912,9 @@ class TypedPipeInst[T] private[scalding] (@transient inpipe: Pipe, new TypedPipeInst[U](inpipe, fields, localFlowDef, mode, flatMapFn.map(f)) /** + * Avoid this method if possible. Prefer to stay in the TypedAPI until + * you write out. + * * This actually runs all the pure map functions in one Cascading Each * This approach is more efficient than untyped scalding because we * don't use TupleConverters/Setters after each map. @@ -785,7 +926,7 @@ class TypedPipeInst[T] private[scalding] (@transient inpipe: Pipe, RichPipe(inpipe).flatMapTo[TupleEntry, U](fields -> fieldNames)(flatMapFn) } - def toIterableExecution: Execution[Iterable[T]] = Execution.factory { (conf, m) => + override def toIterableExecution: Execution[Iterable[T]] = Execution.factory { (conf, m) => // To convert from java iterator to scala below import scala.collection.JavaConverters._ checkMode(m) @@ -807,20 +948,18 @@ class TypedPipeInst[T] private[scalding] (@transient inpipe: Pipe, final case class MergedTypedPipe[T](left: TypedPipe[T], right: TypedPipe[T]) extends TypedPipe[T] { import Dsl._ - // Implements a cross project. The right side should be tiny - def cross[U](tiny: TypedPipe[U]): TypedPipe[(T, U)] = tiny match { + override def cross[U](tiny: TypedPipe[U]): TypedPipe[(T, U)] = tiny match { case EmptyTypedPipe => EmptyTypedPipe case _ => MergedTypedPipe(left.cross(tiny), right.cross(tiny)) } - // prints the current pipe to either stdout or stderr override def debug: TypedPipe[T] = MergedTypedPipe(left.debug, right.debug) override def filter(f: T => Boolean): TypedPipe[T] = MergedTypedPipe(left.filter(f), right.filter(f)) - def flatMap[U](f: T => TraversableOnce[U]): TypedPipe[U] = + override def flatMap[U](f: T => TraversableOnce[U]): TypedPipe[U] = MergedTypedPipe(left.flatMap(f), right.flatMap(f)) override def sample(percent: Double, seed: Long): TypedPipe[T] = @@ -835,7 +974,7 @@ 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. */ @@ -883,7 +1022,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. */ - def toIterableExecution: Execution[Iterable[T]] = + override def toIterableExecution: Execution[Iterable[T]] = left.toIterableExecution.zip(right.toIterableExecution) .map { case (l, r) => l ++ r } @@ -891,17 +1030,6 @@ final case class MergedTypedPipe[T](left: TypedPipe[T], right: TypedPipe[T]) ext MergedTypedPipe(left.hashCogroup(smaller)(joiner), right.hashCogroup(smaller)(joiner)) } -class MappablePipeJoinEnrichment[T](pipe: TypedPipe[T]) { - def joinBy[K, U](smaller: TypedPipe[U])(g: (T => K), h: (U => K), reducers: Int = -1)(implicit ord: Ordering[K]): CoGrouped[K, (T, U)] = pipe.groupBy(g).withReducers(reducers).join(smaller.groupBy(h)) - def leftJoinBy[K, U](smaller: TypedPipe[U])(g: (T => K), h: (U => K), reducers: Int = -1)(implicit ord: Ordering[K]): CoGrouped[K, (T, Option[U])] = pipe.groupBy(g).withReducers(reducers).leftJoin(smaller.groupBy(h)) - def rightJoinBy[K, U](smaller: TypedPipe[U])(g: (T => K), h: (U => K), reducers: Int = -1)(implicit ord: Ordering[K]): CoGrouped[K, (Option[T], U)] = pipe.groupBy(g).withReducers(reducers).rightJoin(smaller.groupBy(h)) - def outerJoinBy[K, U](smaller: TypedPipe[U])(g: (T => K), h: (U => K), reducers: Int = -1)(implicit ord: Ordering[K]): CoGrouped[K, (Option[T], Option[U])] = pipe.groupBy(g).withReducers(reducers).outerJoin(smaller.groupBy(h)) -} - -object Syntax { - implicit def joinOnMappablePipe[T](p: TypedPipe[T]): MappablePipeJoinEnrichment[T] = new MappablePipeJoinEnrichment(p) -} - 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]) = { val pipe = typedPipe.toPipe[U](fieldNames)(flowDef, mode, setter) @@ -912,3 +1040,23 @@ class WithOnComplete[T](typedPipe: TypedPipe[T], fn: () => Unit) extends TypedPi override def toIterableExecution: Execution[Iterable[T]] = forceToDiskExecution.flatMap(_.toIterableExecution) } + +/** + * This class is for the syntax enrichment enabling + * .joinBy on TypedPipes. To access this, do + * import Syntax.joinOnMappablePipe + */ +class MappablePipeJoinEnrichment[T](pipe: TypedPipe[T]) { + def joinBy[K, U](smaller: TypedPipe[U])(g: (T => K), h: (U => K), reducers: Int = -1)(implicit ord: Ordering[K]): CoGrouped[K, (T, U)] = pipe.groupBy(g).withReducers(reducers).join(smaller.groupBy(h)) + def leftJoinBy[K, U](smaller: TypedPipe[U])(g: (T => K), h: (U => K), reducers: Int = -1)(implicit ord: Ordering[K]): CoGrouped[K, (T, Option[U])] = pipe.groupBy(g).withReducers(reducers).leftJoin(smaller.groupBy(h)) + def rightJoinBy[K, U](smaller: TypedPipe[U])(g: (T => K), h: (U => K), reducers: Int = -1)(implicit ord: Ordering[K]): CoGrouped[K, (Option[T], U)] = pipe.groupBy(g).withReducers(reducers).rightJoin(smaller.groupBy(h)) + def outerJoinBy[K, U](smaller: TypedPipe[U])(g: (T => K), h: (U => K), reducers: Int = -1)(implicit ord: Ordering[K]): CoGrouped[K, (Option[T], Option[U])] = pipe.groupBy(g).withReducers(reducers).outerJoin(smaller.groupBy(h)) +} + +/** + * These are named syntax extensions that users can optionally import. + * Avoid import Syntax._ + */ +object Syntax { + implicit def joinOnMappablePipe[T](p: TypedPipe[T]): MappablePipeJoinEnrichment[T] = new MappablePipeJoinEnrichment(p) +} diff --git a/scalding-core/src/test/scala/com/twitter/scalding/AlgebraicReductionsTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/AlgebraicReductionsTest.scala index d29b7b6126..ce8e2aa2cb 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/AlgebraicReductionsTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/AlgebraicReductionsTest.scala @@ -15,9 +15,8 @@ limitations under the License. */ package com.twitter.scalding -import org.specs._ -/** - */ +import org.scalatest.{ Matchers, WordSpec } + class AlgebraJob(args: Args) extends Job(args) { Tsv("input", ('x, 'y, 'z, 'w)) .map('w -> 'w) { w: Int => Set(w) } @@ -39,17 +38,16 @@ class ComplicatedAlgebraJob(args: Args) extends Job(args) { .write(Tsv("output")) } -class AlgebraJobTest extends Specification { - noDetailedDiffs() +class AlgebraJobTest extends WordSpec with Matchers { import Dsl._ val inputData = List((1, 2, 3, 5), (1, 4, 5, 7), (2, 1, 0, 7)) val correctOutput = List((1, 6, 8, Set(5, 7), 8, 15, (6 + 20)), (2, 1, 0, Set(7), 1, 0, 0)) "A AlgebraJob" should { - JobTest("com.twitter.scalding.AlgebraJob") + JobTest(new AlgebraJob(_)) .source(Tsv("input", ('x, 'y, 'z, 'w)), inputData) .sink[(Int, Int, Int, Set[Int], Int, Int, Int)](Tsv("output")) { buf => "correctly do algebra" in { - buf.toList must be_==(correctOutput) + buf.toList shouldBe correctOutput } } .run @@ -59,11 +57,11 @@ class AlgebraJobTest extends Specification { val inputData2 = List((1, 2, 3, 5, 1.2), (1, 4, 5, 7, 0.1), (2, 1, 0, 7, 3.2)) val correctOutput2 = List((1, 6, 8, Set(5, 7), 1.3), (2, 1, 0, Set(7), 3.2)) "A ComplicatedAlgebraJob" should { - JobTest("com.twitter.scalding.ComplicatedAlgebraJob") + JobTest(new ComplicatedAlgebraJob(_)) .source(Tsv("input", ('x, 'y, 'z, 'w, 'v)), inputData2) .sink[(Int, Int, Int, Set[Int], Double)](Tsv("output")) { buf => "correctly do complex algebra" in { - buf.toList must be_==(correctOutput2) + buf.toList shouldBe correctOutput2 } } .run 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 2a86cfd966..c6353973cd 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/BlockJoinTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/BlockJoinTest.scala @@ -15,7 +15,7 @@ limitations under the License. */ package com.twitter.scalding -import org.specs._ +import org.scalatest.{ Matchers, WordSpec } import cascading.pipe.joiner._ @@ -44,9 +44,7 @@ class InnerProductJob(args: Args) extends Job(args) { .write(Tsv("output")) } -class BlockJoinPipeTest extends Specification { - noDetailedDiffs() - +class BlockJoinPipeTest extends WordSpec with Matchers { "An InnerProductJob" should { val in1 = List(("0", "0", "1"), ("0", "1", "1"), ("1", "0", "2"), ("2", "0", "4")) @@ -54,7 +52,7 @@ class BlockJoinPipeTest extends Specification { val correctOutput = Set((0, 1, 2.0), (0, 0, 1.0), (1, 1, 4.0), (2, 1, 8.0)) def runJobWithArguments(left: Int = 1, right: Int = 1, joiner: String = "i")(callback: Buffer[(Int, Int, Double)] => Unit) { - JobTest("com.twitter.scalding.InnerProductJob") + JobTest(new InnerProductJob(_)) .source(Tsv("input0"), in1) .source(Tsv("input1"), in2) .arg("left", left.toString) @@ -69,35 +67,32 @@ class BlockJoinPipeTest extends Specification { "correctly compute product with 1 left block and 1 right block" in { runJobWithArguments() { outBuf => - val unordered = outBuf.toSet - unordered must_== correctOutput + outBuf.toSet shouldBe correctOutput } } "correctly compute product with multiple left and right blocks" in { runJobWithArguments(left = 3, right = 7) { outBuf => - val unordered = outBuf.toSet - unordered must_== correctOutput + outBuf.toSet shouldBe correctOutput } } "correctly compute product with a valid LeftJoin" in { runJobWithArguments(right = 7, joiner = "l") { outBuf => - val unordered = outBuf.toSet - unordered must_== correctOutput + outBuf.toSet shouldBe correctOutput } } "throw an exception when used with OuterJoin" in { - runJobWithArguments(joiner = "o") { g => g } must throwA[InvocationTargetException] + an[InvalidJoinModeException] should be thrownBy runJobWithArguments(joiner = "o") { _ => } } "throw an exception when used with an invalid LeftJoin" in { - runJobWithArguments(joiner = "l", left = 2) { g => g } must throwA[InvocationTargetException] + an[InvalidJoinModeException] should be thrownBy runJobWithArguments(joiner = "l", left = 2) { _ => } } "throw an exception when used with an invalid RightJoin" in { - runJobWithArguments(joiner = "r", right = 2) { g => g } must throwA[InvocationTargetException] + an[InvalidJoinModeException] should be thrownBy runJobWithArguments(joiner = "r", right = 2) { _ => } } } } 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 8f85186e21..d0a280d7f1 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/CascadeTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/CascadeTest.scala @@ -15,11 +15,12 @@ limitations under the License. */ package com.twitter.scalding +import org.scalatest.{ Matchers, WordSpec } + import java.io.BufferedWriter import java.io.FileWriter import scala.io.Source.fromFile import java.io.File -import org.specs._ import cascading.cascade.Cascade import cascading.flow.FlowSkipIfSinkNotStale import cascading.tuple.Fields @@ -46,7 +47,7 @@ class CascadeTestJob(args: Args) extends CascadeJob(args) { } -class TwoPhaseCascadeTest extends Specification with FieldConversions { +class TwoPhaseCascadeTest extends WordSpec with Matchers with FieldConversions { "A Cascade job" should { CascadeTest("com.twitter.scalding.CascadeTestJob") .arg("input0", "input0") @@ -55,7 +56,7 @@ class TwoPhaseCascadeTest extends Specification with FieldConversions { .source(Tsv("input0", ('line)), List(Tuple1("line1"), Tuple1("line2"), Tuple1("line3"), Tuple1("line4"))) .sink[String](Tsv("output1")) { ob => "verify output got changed by both flows" in { - ob.toList must_== List("job2job1:line1", "job2job1:line2", "job2job1:line3", "job2job1:line4") + ob.toList shouldBe List("job2job1:line1", "job2job1:line2", "job2job1:line3", "job2job1:line4") } } .runHadoop @@ -86,7 +87,7 @@ class TwoPhaseCascadeTest extends Specification with FieldConversions { val lines = fromFile(output1.getAbsolutePath).getLines.toList "verify output got changed by both flows" in { - lines must_== List("job2job1:a", "job2job1:b", "job2job1:c", "job2job1:d", "job2job1:e") + lines shouldBe List("job2job1:a", "job2job1:b", "job2job1:c", "job2job1:d", "job2job1:e") } input0.delete() 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 17d170387a..c8b9cfc659 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/CoGroupTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/CoGroupTest.scala @@ -16,7 +16,7 @@ limitations under the License. package com.twitter.scalding import cascading.pipe.joiner._ -import org.specs._ +import org.scalatest.{ WordSpec, Matchers } class StarJoinJob(args: Args) extends Job(args) { val in0 = Tsv("input0").read.mapTo((0, 1) -> ('x0, 'a)) { input: (Int, Int) => input } @@ -33,10 +33,9 @@ class StarJoinJob(args: Args) extends Job(args) { .write(Tsv("output")) } -class CoGroupTest extends Specification { - noDetailedDiffs() +class CoGroupTest extends WordSpec with Matchers { "A StarJoinJob" should { - JobTest("com.twitter.scalding.StarJoinJob") + JobTest(new StarJoinJob(_)) .source(Tsv("input0"), List((0, 1), (1, 1), (2, 1), (3, 2))) .source(Tsv("input1"), List((0, 1), (2, 5), (3, 2))) .source(Tsv("input2"), List((1, 1), (2, 8))) @@ -45,7 +44,7 @@ class CoGroupTest extends Specification { "be able to work" in { val out = outputBuf.toSet val expected = Set((0, 1, 1, 0, 9), (1, 1, 0, 1, 0), (2, 1, 5, 8, 11), (3, 2, 2, 0, 0)) - out must_== expected + out shouldBe expected } } .run 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 048979442b..9cc897317e 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/ConfigTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/ConfigTest.scala @@ -15,7 +15,7 @@ limitations under the License. */ package com.twitter.scalding -import org.specs._ +import org.scalatest.{ WordSpec, Matchers } import org.scalacheck.Arbitrary import org.scalacheck.Arbitrary.arbitrary import org.scalacheck.Properties @@ -24,28 +24,28 @@ import org.scalacheck.Gen._ import scala.util.Success -class ConfigTest extends Specification { +class ConfigTest extends WordSpec with Matchers { "A Config" should { "cascadingAppJar works" in { val cls = getClass Config.default.setCascadingAppJar(cls) - .getCascadingAppJar must be_==(Some(Success(cls))) + .getCascadingAppJar should contain (Success(cls)) } "default has serialization set" in { val sers = Config.default.get("io.serializations").get.split(",").toList - sers.last must be_==(classOf[com.twitter.chill.hadoop.KryoSerialization].getName) + sers.last shouldBe (classOf[com.twitter.chill.hadoop.KryoSerialization].getName) } "default has chill configured" in { - Config.default.get(com.twitter.chill.config.ConfiguredInstantiator.KEY).isDefined must beTrue + Config.default.get(com.twitter.chill.config.ConfiguredInstantiator.KEY) should not be empty } "setting timestamp twice does not change it" in { val date = RichDate.now val (oldDate, newConf) = Config.empty.maybeSetSubmittedTimestamp(date) - oldDate.isEmpty must beTrue - newConf.getSubmittedTimestamp must be_==(Some(date)) + oldDate shouldBe empty + newConf.getSubmittedTimestamp should contain (date) val (stillOld, new2) = newConf.maybeSetSubmittedTimestamp(date + Seconds(1)) - stillOld must be_==(Some(date)) - new2 must be_==(newConf) + stillOld should contain (date) + new2 shouldBe newConf } } } 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 20887b2b75..1244e42771 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/CoreTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/CoreTest.scala @@ -15,12 +15,13 @@ limitations under the License. */ package com.twitter.scalding +import org.scalatest.{ WordSpec, Matchers } + import cascading.tuple.Fields import cascading.tuple.TupleEntry import java.util.concurrent.TimeUnit import com.twitter.scalding.source.DailySuffixTsv -import org.specs._ import java.lang.{ Integer => JInt } class NumberJoinerJob(args: Args) extends Job(args) { @@ -30,20 +31,20 @@ class NumberJoinerJob(args: Args) extends Job(args) { .write(Tsv("output")) } -class NumberJoinTest extends Specification { +class NumberJoinTest extends WordSpec with Matchers { import Dsl._ "A NumberJoinerJob" should { //Set up the job: "not throw when joining longs with ints" in { - JobTest("com.twitter.scalding.NumberJoinerJob") + JobTest(new NumberJoinerJob(_)) .source(TypedTsv[(Int, Int)]("input0"), List((0, 1), (1, 2), (2, 4))) .source(Tsv("input1"), List(("0", "1"), ("1", "3"), ("2", "9"))) .sink[(Int, Int, Long, Long)](Tsv("output")) { outBuf => val unordered = outBuf.toSet - unordered.size must be_==(3) - unordered((0, 1, 0L, 1L)) must be_==(true) - unordered((1, 2, 1L, 3L)) must be_==(true) - unordered((2, 4, 2L, 9L)) must be_==(true) + unordered should have size 3 + unordered should contain (0, 1, 0L, 1L) + unordered should contain (1, 2, 1L, 3L) + unordered should contain (2, 4, 2L, 9L) } .run .runHadoop @@ -59,7 +60,7 @@ class SpillingJob(args: Args) extends Job(args) { }.write(Tsv("output")) } -class SpillingTest extends Specification { +class SpillingTest extends WordSpec with Matchers { import Dsl._ "A SpillingJob" should { val src = (0 to 9).map(_ -> 1) ++ List(0 -> 4) @@ -73,7 +74,7 @@ class SpillingTest extends Specification { JobTest(new SpillingJob(_)) .source(TypedTsv[(Int, Int)]("input"), src) .sink[(Int, Int, Int)](Tsv("output")) { outBuf => - outBuf.toSet must be_==(result) + outBuf.toSet shouldBe result }.run .runHadoop .finish @@ -95,19 +96,19 @@ class GroupRandomlyJob(args: Args) extends Job(args) { .write(Tsv("fakeOutput")) } -class GroupRandomlyJobTest extends Specification { +class GroupRandomlyJobTest extends WordSpec with Matchers { import GroupRandomlyJob.NumShards - noDetailedDiffs() "A GroupRandomlyJob" should { val input = (0 to 10000).map { _.toString }.map { Tuple1(_) } - JobTest("com.twitter.scalding.GroupRandomlyJob") + JobTest(new GroupRandomlyJob(_)) .source(Tsv("fakeInput"), input) .sink[(Int)](Tsv("fakeOutput")) { outBuf => val numShards = outBuf(0) - numShards must be_==(NumShards) + numShards shouldBe NumShards } - .run.finish + .run + .finish } } @@ -120,19 +121,18 @@ class ShuffleJob(args: Args) extends Job(args) { .write(Tsv("fakeOutput")) } -class ShuffleJobTest extends Specification { - noDetailedDiffs() - +class ShuffleJobTest extends WordSpec with Matchers { val expectedShuffle: List[Int] = List(10, 5, 9, 12, 0, 1, 4, 8, 11, 6, 2, 3, 7) "A ShuffleJob" should { val input = (0 to 12).map { Tuple1(_) } - JobTest("com.twitter.scalding.ShuffleJob") + JobTest(new ShuffleJob(_)) .source(Tsv("fakeInput"), input) .sink[(List[Int])](Tsv("fakeOutput")) { outBuf => - outBuf(0) must be_==(expectedShuffle) + outBuf(0) shouldBe expectedShuffle } - .run.finish + .run + .finish } } @@ -147,8 +147,7 @@ class MapToGroupBySizeSumMaxJob(args: Args) extends Job(args) { write(Tsv(args("output"))) } -class MapToGroupBySizeSumMaxTest extends Specification { - noDetailedDiffs() +class MapToGroupBySizeSumMaxTest extends WordSpec with Matchers { "A MapToGroupBySizeSumMaxJob" should { val r = new java.util.Random //Here is our input data: @@ -168,21 +167,21 @@ class MapToGroupBySizeSumMaxTest extends Specification { (size, sum, max) } //Now we have the expected input and output: - JobTest("com.twitter.scalding.MapToGroupBySizeSumMaxJob"). - arg("input", "fakeInput"). - arg("output", "fakeOutput"). - source(TextLine("fakeInput"), input). - sink[(Boolean, Int, Double, Double)](Tsv("fakeOutput")) { outBuf => + JobTest(new MapToGroupBySizeSumMaxJob(_)) + .arg("input", "fakeInput") + .arg("output", "fakeOutput") + .source(TextLine("fakeInput"), input) + .sink[(Boolean, Int, Double, Double)](Tsv("fakeOutput")) { outBuf => val actualOutput = outBuf.map { case (k: Boolean, sz: Int, sm: Double, mx: Double) => (k, (sz, sm, mx)) }.toMap "produce correct size, sum, max" in { - goldenOutput must be_==(actualOutput) + goldenOutput shouldBe actualOutput } - }. - run. - finish + } + .run + .finish } } @@ -193,8 +192,7 @@ class PartitionJob(args: Args) extends Job(args) { .write(Tsv("output")) } -class PartitionJobTest extends Specification { - noDetailedDiffs() +class PartitionJobTest extends WordSpec with Matchers { "A PartitionJob" should { val input = List((3, 23), (23, 154), (15, 123), (53, 143), (7, 85), (19, 195), (42, 187), (35, 165), (68, 121), (13, 103), (17, 173), (2, 13)) @@ -209,9 +207,10 @@ class PartitionJobTest extends Specification { JobTest(new com.twitter.scalding.PartitionJob(_)) .source(Tsv("input", new Fields("age", "weight")), input) .sink[(Boolean, Double)](Tsv("output")) { outBuf => - outBuf.toMap must be_==(expectedOutput) + outBuf.toMap shouldBe expectedOutput } - .run.finish + .run + .finish } } @@ -232,26 +231,25 @@ class MRMJob(args: Args) extends Job(args) { .write(Tsv("outputSetTo")) } -class MRMTest extends Specification { - noDetailedDiffs() //Fixes an issue with scala 2.9 +class MRMTest extends WordSpec with Matchers { "A MRMJob" should { val input = List((0, 1), (0, 2), (1, 3), (1, 1)) - JobTest("com.twitter.scalding.MRMJob") + JobTest(new MRMJob(_)) .source(Tsv("input"), input) .sink[(Int, Int)](Tsv("outputXor")) { outBuf => "use reduce to compute xor" in { - outBuf.toList.sorted must be_==(List((0, 3), (1, 2))) + outBuf.toList.sorted shouldBe List((0, 3), (1, 2)) } } .sink[(Int, Int)](Tsv("outputSet")) { outBuf => "use mapReduceMap to round-trip input" in { - outBuf.toList.sorted must be_==(input.sorted) + outBuf.toList.sorted shouldBe (input.sorted) } } .sink[Int](Tsv("outputSetTo")) { outBuf => "use flattenTo" in { - outBuf.toList.sorted must be_==(input.map { _._2 }.sorted) + outBuf.toList.sorted shouldBe (input.map { _._2 }.sorted) } } .run @@ -271,14 +269,13 @@ class JoinJob(args: Args) extends Job(args) { .write(Tsv(args("output"))) } -class JoinTest extends Specification { - noDetailedDiffs() //Fixes an issue with scala 2.9 +class JoinTest extends WordSpec with Matchers { "A JoinJob" should { val input1 = List("a" -> 1, "b" -> 2, "c" -> 3) val input2 = List("b" -> -1, "c" -> 5, "d" -> 4) val correctOutput = Map("b" -> (2, -1), "c" -> (3, 5)) - JobTest("com.twitter.scalding.JoinJob") + JobTest(new JoinJob(_)) .arg("input1", "fakeInput1") .arg("input2", "fakeInput2") .arg("output", "fakeOutput") @@ -290,7 +287,7 @@ class JoinTest extends Specification { (k, (v1, v2)) }.toMap "join tuples with the same key" in { - correctOutput must be_==(actualOutput) + correctOutput shouldBe actualOutput } } .run @@ -313,14 +310,13 @@ class CollidingKeyJoinJob(args: Args) extends Job(args) { .write(Tsv(args("output"))) } -class CollidingKeyJoinTest extends Specification { - noDetailedDiffs() //Fixes an issue with scala 2.9 +class CollidingKeyJoinTest extends WordSpec with Matchers { "A CollidingKeyJoinJob" should { val input1 = List("a" -> 1, "b" -> 2, "c" -> 3) val input2 = List("b" -> -1, "c" -> 5, "d" -> 4) val correctOutput = Map("b" -> (2, "bb", -1, "bb"), "c" -> (3, "cc", 5, "cc")) - JobTest("com.twitter.scalding.CollidingKeyJoinJob") + JobTest(new CollidingKeyJoinJob(_)) .arg("input1", "fakeInput1") .arg("input2", "fakeInput2") .arg("output", "fakeOutput") @@ -332,7 +328,7 @@ class CollidingKeyJoinTest extends Specification { (k, (v1, k2, v2, k3)) }.toMap "join tuples with the same key" in { - correctOutput must be_==(actualOutput) + correctOutput shouldBe actualOutput } } .run @@ -352,14 +348,13 @@ class TinyJoinJob(args: Args) extends Job(args) { .write(Tsv(args("output"))) } -class TinyJoinTest extends Specification { - noDetailedDiffs() //Fixes an issue with scala 2.9 +class TinyJoinTest extends WordSpec with Matchers { "A TinyJoinJob" should { val input1 = List("a" -> 1, "b" -> 2, "c" -> 3) val input2 = List("b" -> -1, "c" -> 5, "d" -> 4) val correctOutput = Map("b" -> (2, -1), "c" -> (3, 5)) - - JobTest("com.twitter.scalding.TinyJoinJob") + var idx = 0 + JobTest(new TinyJoinJob(_)) .arg("input1", "fakeInput1") .arg("input2", "fakeInput2") .arg("output", "fakeOutput") @@ -370,9 +365,10 @@ class TinyJoinTest extends Specification { case (k: String, v1: Int, v2: Int) => (k, (v1, v2)) }.toMap - "join tuples with the same key" in { - correctOutput must be_==(actualOutput) + (idx + ": join tuples with the same key") in { + actualOutput shouldBe correctOutput } + idx += 1 } .run .runHadoop @@ -391,14 +387,13 @@ class TinyCollisionJoinJob(args: Args) extends Job(args) { .write(Tsv(args("output"))) } -class TinyCollisionJoinTest extends Specification { - noDetailedDiffs() //Fixes an issue with scala 2.9 +class TinyCollisionJoinTest extends WordSpec with Matchers { "A TinyCollisionJoinJob" should { val input1 = List("a" -> 1, "b" -> 2, "c" -> 3) val input2 = List("b" -> -1, "c" -> 5, "d" -> 4) val correctOutput = Map("b" -> (2, -1), "c" -> (3, 5)) - JobTest("com.twitter.scalding.TinyCollisionJoinJob") + JobTest(new TinyCollisionJoinJob(_)) .arg("input1", "fakeInput1") .arg("input2", "fakeInput2") .arg("output", "fakeOutput") @@ -410,7 +405,7 @@ class TinyCollisionJoinTest extends Specification { (k, (v1, v2)) }.toMap "join tuples with the same key" in { - correctOutput must be_==(actualOutput) + correctOutput shouldBe actualOutput } } .run @@ -434,25 +429,23 @@ class TinyThenSmallJoin(args: Args) extends Job(args) { case class TC(val n: Int) -class TinyThenSmallJoinTest extends Specification with FieldConversions { - noDetailedDiffs() //Fixes an issue with scala 2.9 +class TinyThenSmallJoinTest extends WordSpec with Matchers with FieldConversions { "A TinyThenSmallJoin" should { val input0 = List((1, TC(2)), (2, TC(3)), (3, TC(4))) val input1 = List((1, TC(20)), (2, TC(30)), (3, TC(40))) val input2 = List((1, TC(200)), (2, TC(300)), (3, TC(400))) val correct = List((1, 2, 1, 20, 1, 200), (2, 3, 2, 30, 2, 300), (3, 4, 3, 40, 3, 400)) - - JobTest("com.twitter.scalding.TinyThenSmallJoin") + var idx = 0 + JobTest(new TinyThenSmallJoin(_)) .source(Tsv("in0", ('x0, 'y0)), input0) .source(Tsv("in1", ('x1, 'y1)), input1) .source(Tsv("in2", ('x2, 'y2)), input2) .sink[(Int, Int, Int, Int, Int, Int)](Tsv("out")) { outBuf => - val actualOutput = outBuf.toList.sorted - println(actualOutput) - "join tuples with the same key" in { - correct must be_==(actualOutput) + (idx + ": join tuples with the same key") in { + outBuf.toList.sorted shouldBe correct } + idx += 1 } .run .runHadoop @@ -472,15 +465,14 @@ class LeftJoinJob(args: Args) extends Job(args) { .write(Tsv(args("output"))) } -class LeftJoinTest extends Specification { - noDetailedDiffs() //Fixes an issue with scala 2.9 +class LeftJoinTest extends WordSpec with Matchers { "A LeftJoinJob" should { val input1 = List("a" -> 1, "b" -> 2, "c" -> 3) val input2 = List("b" -> -1, "c" -> 5, "d" -> 4) val correctOutput = Map[String, (Int, AnyRef)]("a" -> (1, "NULL"), "b" -> (2, "-1"), "c" -> (3, "5")) - - JobTest("com.twitter.scalding.LeftJoinJob") + var idx = 0 + JobTest(new LeftJoinJob(_)) .arg("input1", "fakeInput1") .arg("input2", "fakeInput2") .arg("output", "fakeOutput") @@ -492,9 +484,10 @@ class LeftJoinTest extends Specification { val (k, v1, v2) = input (k, (v1, v2)) }.toMap - "join tuples with the same key" in { - correctOutput must be_==(actualOutput) + (idx + ": join tuples with the same key") in { + correctOutput shouldBe actualOutput } + idx += 1 } .run .runHadoop @@ -515,15 +508,14 @@ class LeftJoinWithLargerJob(args: Args) extends Job(args) { .write(Tsv(args("output"))) } -class LeftJoinWithLargerTest extends Specification { - noDetailedDiffs() //Fixes an issue with scala 2.9 +class LeftJoinWithLargerTest extends WordSpec with Matchers { "A LeftJoinWithLargerJob" should { val input1 = List("a" -> 1, "b" -> 2, "c" -> 3) val input2 = List("b" -> -1, "c" -> 5, "d" -> 4) val correctOutput = Map[String, (Int, AnyRef)]("a" -> (1, "NULL"), "b" -> (2, "-1"), "c" -> (3, "5")) - - JobTest("com.twitter.scalding.LeftJoinWithLargerJob") + var idx = 0 + JobTest(new LeftJoinWithLargerJob(_)) .arg("input1", "fakeInput1") .arg("input2", "fakeInput2") .arg("output", "fakeOutput") @@ -535,9 +527,10 @@ class LeftJoinWithLargerTest extends Specification { val (k, v1, v2) = input (k, (v1, v2)) }.toMap - "join tuples with the same key" in { - correctOutput must be_==(actualOutput) + s"$idx: join tuples with the same key" in { + correctOutput shouldBe actualOutput } + idx += 1 } .run .runHadoop @@ -559,8 +552,7 @@ class MergeTestJob(args: Args) extends Job(args) { .write(Tsv("out2")) } -class MergeTest extends Specification { - noDetailedDiffs() //Fixes an issue with scala 2.9 +class MergeTest extends WordSpec with Matchers { "A MergeTest" should { val r = new java.util.Random //Here is our input data: @@ -575,18 +567,18 @@ class MergeTest extends Specification { val small = parsed.filter(_._1 <= 0.5) val golden = (big ++ small).groupBy{ _._1 }.mapValues { itup => (itup.map{ _._2 }.max) } //Now we have the expected input and output: - JobTest("com.twitter.scalding.MergeTestJob"). - arg("in", "fakeInput"). - arg("out", "fakeOutput"). - source(TextLine("fakeInput"), input). - sink[(Double, Double)](Tsv("fakeOutput")) { outBuf => + JobTest(new MergeTestJob(_)) + .arg("in", "fakeInput") + .arg("out", "fakeOutput") + .source(TextLine("fakeInput"), input) + .sink[(Double, Double)](Tsv("fakeOutput")) { outBuf => "correctly merge two pipes" in { - golden must be_==(outBuf.toMap) + golden shouldBe outBuf.toMap } }. sink[(Double, Double)](Tsv("out2")) { outBuf => "correctly self merge" in { - outBuf.toMap must be_==(big.groupBy(_._1).mapValues{ iter => iter.map(_._2).max }) + outBuf.toMap shouldBe (big.groupBy(_._1).mapValues{ iter => iter.map(_._2).max }) } }. run. @@ -609,50 +601,48 @@ class SizeAveStdJob(args: Args) extends Job(args) { .write(Tsv(args("output"))) } -class SizeAveStdSpec extends Specification { +class SizeAveStdSpec extends WordSpec with Matchers { "A sizeAveStd job" should { - "correctly compute aves and standard deviations" in { - val r = new java.util.Random - def powerLawRand = { - // Generates a 1/x powerlaw with a max value or 1e40 - scala.math.pow(1e40, r.nextDouble) - } - //Here is our input data: - val input = (0 to 10000).map { i => (i.toString, r.nextDouble.toString + " " + powerLawRand.toString) } - val output = input.map { numline => numline._2.split(" ").map { _.toDouble } } - .map { vec => ((vec(0) * 4).toInt, vec(1)) } - .groupBy { tup => tup._1 } - .mapValues { tups => - val all = tups.map { tup => tup._2.toDouble }.toList - val size = all.size.toLong - val ave = all.sum / size - //Compute the standard deviation: - val vari = all.map { x => (x - ave) * (x - ave) }.sum / (size) - val stdev = scala.math.sqrt(vari) - (size, ave, stdev) - } - JobTest(new SizeAveStdJob(_)). - arg("input", "fakeInput"). - arg("output", "fakeOutput"). - source(TextLine("fakeInput"), input). - sink[(Int, Long, Double, Double, Double)](Tsv("fakeOutput")) { outBuf => - "correctly compute size, ave, stdev" in { - outBuf.foreach { computed => - val correctTup = output(computed._1) - //Size - computed._2 must be_== (correctTup._1) - //Ave - computed._3 / correctTup._2 must beCloseTo(1.0, 1e-6) - //Stdev - computed._4 / correctTup._3 must beCloseTo(1.0, 1e-6) - //Explicitly calculated Average: - computed._5 / computed._3 must beCloseTo(1.0, 1e-6) - } - } - }. - run. - finish + val r = new java.util.Random + def powerLawRand = { + // Generates a 1/x powerlaw with a max value or 1e40 + scala.math.pow(1e40, r.nextDouble) } + //Here is our input data: + val input = (0 to 10000).map { i => (i.toString, r.nextDouble.toString + " " + powerLawRand.toString) } + val output = input.map { numline => numline._2.split(" ").map { _.toDouble } } + .map { vec => ((vec(0) * 4).toInt, vec(1)) } + .groupBy { tup => tup._1 } + .mapValues { tups => + val all = tups.map { tup => tup._2.toDouble }.toList + val size = all.size.toLong + val ave = all.sum / size + //Compute the standard deviation: + val vari = all.map { x => (x - ave) * (x - ave) }.sum / (size) + val stdev = scala.math.sqrt(vari) + (size, ave, stdev) + } + JobTest(new SizeAveStdJob(_)) + .arg("input", "fakeInput") + .arg("output", "fakeOutput") + .source(TextLine("fakeInput"), input) + .sink[(Int, Long, Double, Double, Double)](Tsv("fakeOutput")) { outBuf => + "correctly compute size, ave, stdev" in { + outBuf.foreach { computed => + val correctTup = output(computed._1) + //Size + computed._2 shouldBe (correctTup._1) + //Ave + computed._3 / correctTup._2 shouldBe 1.0 +- 1e-6 + //Stdev + computed._4 / correctTup._3 shouldBe 1.0 +- 1e-6 + //Explicitly calculated Average: + computed._5 / computed._3 shouldBe 1.0 +- 1e-6 + } + } + } + .run + .finish } } @@ -666,30 +656,28 @@ class DoubleGroupJob(args: Args) extends Job(args) { .write(Tsv(args("out"))) } -class DoubleGroupSpec extends Specification { +class DoubleGroupSpec extends WordSpec with Matchers { "A DoubleGroupJob" should { - "correctly generate output" in { - JobTest("com.twitter.scalding.DoubleGroupJob"). - arg("in", "fakeIn"). - arg("out", "fakeOut"). - source(TextLine("fakeIn"), List("0" -> "one 1", - "1" -> "two 1", - "2" -> "two 2", - "3" -> "three 3", - "4" -> "three 4", - "5" -> "three 5", - "6" -> "just one")). - sink[(Long, Long)](Tsv("fakeOut")) { outBuf => - "correctly build histogram" in { - val outM = outBuf.toMap - outM(1) must be_== (2) //both one and just keys occur only once - outM(2) must be_== (1) - outM(3) must be_== (1) - } - }. - run. - finish - } + JobTest(new DoubleGroupJob(_)) + .arg("in", "fakeIn") + .arg("out", "fakeOut") + .source(TextLine("fakeIn"), List("0" -> "one 1", + "1" -> "two 1", + "2" -> "two 2", + "3" -> "three 3", + "4" -> "three 4", + "5" -> "three 5", + "6" -> "just one")) + .sink[(Long, Long)](Tsv("fakeOut")) { outBuf => + "correctly build histogram" in { + val outM = outBuf.toMap + outM(1) shouldBe 2 //both one and just keys occur only once + outM(2) shouldBe 1 + outM(3) shouldBe 1 + } + } + .run + .finish } } @@ -703,26 +691,25 @@ class GroupUniqueJob(args: Args) extends Job(args) { .write(Tsv(args("out"))) } -class GroupUniqueSpec extends Specification { +class GroupUniqueSpec extends WordSpec with Matchers { "A GroupUniqueJob" should { - JobTest("com.twitter.scalding.GroupUniqueJob"). - arg("in", "fakeIn"). - arg("out", "fakeOut"). - source(TextLine("fakeIn"), List("0" -> "one 1", + JobTest(new GroupUniqueJob(_)) + .arg("in", "fakeIn") + .arg("out", "fakeOut") + .source(TextLine("fakeIn"), List("0" -> "one 1", "1" -> "two 1", "2" -> "two 2", "3" -> "three 3", "4" -> "three 4", "5" -> "three 5", - "6" -> "just one")). - sink[(Long)](Tsv("fakeOut")) { outBuf => + "6" -> "just one")) + .sink[(Long)](Tsv("fakeOut")) { outBuf => "correctly count unique sizes" in { - val outSet = outBuf.toSet - outSet.size must_== 3 + outBuf.toSet should have size 3 } - }. - run. - finish + } + .run + .finish } } @@ -735,18 +722,18 @@ class DiscardTestJob(args: Args) extends Job(args) { .write(Tsv(args("out"))) } -class DiscardTest extends Specification { +class DiscardTest extends WordSpec with Matchers { "A DiscardTestJob" should { - JobTest("com.twitter.scalding.DiscardTestJob") + JobTest(new DiscardTestJob(_)) .arg("in", "fakeIn") .arg("out", "fakeOut") .source(TextLine("fakeIn"), List("0" -> "hello world", "1" -> "foo", "2" -> "bar")) .sink[Boolean](Tsv("fakeOut")) { outBuf => "must reduce down to one line" in { - outBuf.size must_== 1 + outBuf should have size 1 } "must correctly discard word column" in { - outBuf(0) must beTrue + outBuf(0) shouldBe true } } .run @@ -761,18 +748,18 @@ class HistogramJob(args: Args) extends Job(args) { .write(Tsv(args("out"))) } -class HistogramTest extends Specification { +class HistogramTest extends WordSpec with Matchers { "A HistogramJob" should { - JobTest("com.twitter.scalding.HistogramJob") + JobTest(new HistogramJob(_)) .arg("in", "fakeIn") .arg("out", "fakeOut") .source(TextLine("fakeIn"), List("0" -> "single", "1" -> "single")) .sink[(Long, Long)](Tsv("fakeOut")) { outBuf => "must reduce down to a single line for a trivial input" in { - outBuf.size must_== 1 + outBuf should have size 1 } "must get the result right" in { - outBuf(0) must_== (2L, 1L) + outBuf(0) shouldBe (2L, 1L) } } .run @@ -790,16 +777,18 @@ class ForceReducersJob(args: Args) extends Job(args) { .write(Tsv("out")) } -class ForceReducersTest extends Specification { +class ForceReducersTest extends WordSpec with Matchers { "A ForceReducersJob" should { - JobTest("com.twitter.scalding.ForceReducersJob") + var idx = 0 + JobTest(new ForceReducersJob(_)) .source(TextLine("in"), List("0" -> "single test", "1" -> "single result")) .sink[(Int, String)](Tsv("out")) { outBuf => - "must get the result right" in { + (idx + ": must get the result right") in { //need to convert to sets because order - outBuf(0)._2.split(" ").toSet must_== Set("single", "test") - outBuf(1)._2.split(" ").toSet must_== Set("single", "result") + outBuf(0)._2.split(" ").toSet shouldBe Set("single", "test") + outBuf(1)._2.split(" ").toSet shouldBe Set("single", "result") } + idx += 1 } .run .runHadoop @@ -823,20 +812,20 @@ class NullListJob(args: Args) extends Job(args) { .write(Tsv(args("out"))) } -class ToListTest extends Specification { +class ToListTest extends WordSpec with Matchers { "A ToListJob" should { - JobTest("com.twitter.scalding.ToListJob") + JobTest(new ToListJob(_)) .arg("in", "fakeIn") .arg("out", "fakeOut") .source(TextLine("fakeIn"), List("0" -> "single test", "1" -> "single result")) .sink[(Int, String)](Tsv("fakeOut")) { outBuf => "must have the right number of lines" in { - outBuf.size must_== 2 + outBuf should have size 2 } "must get the result right" in { //need to convert to sets because order - outBuf(0)._2.split(" ").toSet must_== Set("single", "test") - outBuf(1)._2.split(" ").toSet must_== Set("single", "result") + outBuf(0)._2.split(" ").toSet shouldBe Set("single", "test") + outBuf(1)._2.split(" ").toSet shouldBe Set("single", "result") } } .run @@ -844,17 +833,17 @@ class ToListTest extends Specification { } "A NullListJob" should { - JobTest("com.twitter.scalding.NullListJob") + JobTest(new NullListJob(_)) .arg("in", "fakeIn") .arg("out", "fakeOut") .source(TextLine("fakeIn"), List("0" -> null, "0" -> "a", "0" -> null, "0" -> "b")) .sink[(Int, String)](Tsv("fakeOut")) { outBuf => "must have the right number of lines" in { - outBuf.size must_== 1 + outBuf should have size 1 } "must return an empty list for null key" in { val sSet = outBuf(0)._2.split(" ").toSet - sSet must_== Set("a", "b") + sSet shouldBe Set("a", "b") } } .run @@ -870,21 +859,21 @@ class CrossJob(args: Args) extends Job(args) { p1.crossWithTiny(p2).write(Tsv(args("out"))) } -class CrossTest extends Specification { - noDetailedDiffs() - +class CrossTest extends WordSpec with Matchers { "A CrossJob" should { - JobTest("com.twitter.scalding.CrossJob") + var idx = 0 + JobTest(new com.twitter.scalding.CrossJob(_)) .arg("in1", "fakeIn1") .arg("in2", "fakeIn2") .arg("out", "fakeOut") .source(Tsv("fakeIn1"), List(("0", "1"), ("1", "2"), ("2", "3"))) .source(Tsv("fakeIn2"), List("4", "5").map { Tuple1(_) }) .sink[(Int, Int, Int)](Tsv("fakeOut")) { outBuf => - "must look exactly right" in { - outBuf.size must_== 6 - outBuf.toSet must_== (Set((0, 1, 4), (0, 1, 5), (1, 2, 4), (1, 2, 5), (2, 3, 4), (2, 3, 5))) + (idx + ": must look exactly right") in { + outBuf should have size 6 + outBuf.toSet shouldBe (Set((0, 1, 4), (0, 1, 5), (1, 2, 4), (1, 2, 5), (2, 3, 4), (2, 3, 5))) } + idx += 1 } .run .runHadoop @@ -906,10 +895,9 @@ class GroupAllCrossJob(args: Args) extends Job(args) { .write(Tsv(args("out"))) } -class GroupAllCrossTest extends Specification { - noDetailedDiffs() - +class GroupAllCrossTest extends WordSpec with Matchers { "A GroupAllCrossJob" should { + var idx = 0 JobTest(new GroupAllCrossJob(_)) .arg("in1", "fakeIn1") .arg("in2", "fakeIn2") @@ -917,10 +905,11 @@ class GroupAllCrossTest extends Specification { .source(Tsv("fakeIn1"), List(("0", "1"), ("1", "2"), ("2", "3"))) .source(Tsv("fakeIn2"), List("4", "5").map { Tuple1(_) }) .sink[(Int, Int)](Tsv("fakeOut")) { outBuf => - "must look exactly right" in { - outBuf.size must_== 2 - outBuf.toSet must_== (Set((1, 4), (1, 5))) + (idx + ": must look exactly right") in { + outBuf should have size 2 + outBuf.toSet shouldBe Set((1, 4), (1, 5)) } + idx += 1 } .run .runHadoop @@ -936,21 +925,21 @@ class SmallCrossJob(args: Args) extends Job(args) { p1.crossWithSmaller(p2).write(Tsv(args("out"))) } -class SmallCrossTest extends Specification { - noDetailedDiffs() - +class SmallCrossTest extends WordSpec with Matchers { "A SmallCrossJob" should { - JobTest("com.twitter.scalding.SmallCrossJob") + var idx = 0 + JobTest(new SmallCrossJob(_)) .arg("in1", "fakeIn1") .arg("in2", "fakeIn2") .arg("out", "fakeOut") .source(Tsv("fakeIn1"), List(("0", "1"), ("1", "2"), ("2", "3"))) .source(Tsv("fakeIn2"), List("4", "5").map { Tuple1(_) }) .sink[(Int, Int, Int)](Tsv("fakeOut")) { outBuf => - "must look exactly right" in { - outBuf.size must_== 6 - outBuf.toSet must_== (Set((0, 1, 4), (0, 1, 5), (1, 2, 4), (1, 2, 5), (2, 3, 4), (2, 3, 5))) + (idx + ": must look exactly right") in { + outBuf should have size 6 + outBuf.toSet shouldBe Set((0, 1, 4), (0, 1, 5), (1, 2, 4), (1, 2, 5), (2, 3, 4), (2, 3, 5)) } + idx += 1 } .run .runHadoop @@ -966,16 +955,16 @@ class TopKJob(args: Args) extends Job(args) { .write(Tsv(args("out"))) } -class TopKTest extends Specification { +class TopKTest extends WordSpec with Matchers { "A TopKJob" should { - JobTest("com.twitter.scalding.TopKJob") + JobTest(new TopKJob(_)) .arg("in", "fakeIn") .arg("out", "fakeOut") .source(Tsv("fakeIn"), List(3, 24, 1, 4, 5).map { Tuple1(_) }) .sink[List[Int]](Tsv("fakeOut")) { outBuf => "must look exactly right" in { - outBuf.size must_== 1 - outBuf(0) must be_==(List(1, 3, 4)) + outBuf should have size 1 + outBuf(0) shouldBe List(1, 3, 4) } } .run @@ -994,18 +983,19 @@ class ScanJob(args: Args) extends Job(args) { .write(Tsv("out")) } -class ScanTest extends Specification { +class ScanTest extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() + "A ScanJob" should { - JobTest("com.twitter.scalding.ScanJob") + var idx = 0 + JobTest(new ScanJob(_)) .source(Tsv("in", ('x, 'y, 'z)), List((3, 0, 1), (3, 1, 10), (3, 5, 100))) .sink[(Int, Int, Int)](Tsv("out")) { outBuf => - () val correct = List((3, 0, 0), (3, 0, 1), (3, 1, 10), (3, 6, 100)) - "have a working scanLeft" in { - outBuf.toList must be_== (correct) + (idx + ": have a working scanLeft") in { + outBuf.toList shouldBe correct } + idx += 1 } .run .runHadoop @@ -1021,22 +1011,20 @@ class TakeJob(args: Args) extends Job(args) { input.groupAll.write(Tsv("outall")) } -class TakeTest extends Specification { - noDetailedDiffs() +class TakeTest extends WordSpec with Matchers { "A TakeJob" should { - JobTest("com.twitter.scalding.TakeJob") + JobTest(new TakeJob(_)) .source(Tsv("in"), List((3, 0, 1), (3, 1, 10), (3, 5, 100))) .sink[(Int, Int, Int)](Tsv("outall")) { outBuf => - () "groupAll must see everything in same order" in { - outBuf.size must_== 3 - outBuf.toList must be_== (List((3, 0, 1), (3, 1, 10), (3, 5, 100))) + outBuf should have size 3 + outBuf.toList shouldBe List((3, 0, 1), (3, 1, 10), (3, 5, 100)) } } .sink[(Int, Int, Int)](Tsv("out2")) { outBuf => "take(2) must only get 2" in { - outBuf.size must_== 2 - outBuf.toList must be_== (List((3, 0, 1), (3, 1, 10))) + outBuf should have size 2 + outBuf.toList shouldBe List((3, 0, 1), (3, 1, 10)) } } .run @@ -1052,21 +1040,19 @@ class DropJob(args: Args) extends Job(args) { input.groupAll.write(Tsv("outall")) } -class DropTest extends Specification { - noDetailedDiffs() +class DropTest extends WordSpec with Matchers { "A DropJob" should { - JobTest("com.twitter.scalding.DropJob") + JobTest(new DropJob(_)) .source(Tsv("in"), List((3, 0, 1), (3, 1, 10), (3, 5, 100))) .sink[(Int, Int, Int)](Tsv("outall")) { outBuf => - () "groupAll must see everything in same order" in { - outBuf.size must_== 3 - outBuf.toList must be_== (List((3, 0, 1), (3, 1, 10), (3, 5, 100))) + outBuf should have size 3 + outBuf.toList shouldBe List((3, 0, 1), (3, 1, 10), (3, 5, 100)) } } .sink[(Int, Int, Int)](Tsv("out2")) { outBuf => "drop(2) must only get 1" in { - outBuf.toList must be_== (List((3, 5, 100))) + outBuf.toList shouldBe List((3, 5, 100)) } } .run @@ -1087,29 +1073,28 @@ class PivotJob(args: Args) extends Job(args) { }.write(Tsv("pivot_with_default")) } -class PivotTest extends Specification with FieldConversions { - noDetailedDiffs() +class PivotTest extends WordSpec with Matchers with FieldConversions { val input = List(("1", "a", "b", "c"), ("2", "d", "e", "f")) "A PivotJob" should { JobTest("com.twitter.scalding.PivotJob") .source(Tsv("in", ('k, 'w, 'y, 'z)), input) .sink[(String, String, String)](Tsv("unpivot")) { outBuf => "unpivot columns correctly" in { - outBuf.size must_== 6 - outBuf.toList.sorted must be_== (List(("1", "w", "a"), ("1", "y", "b"), ("1", "z", "c"), + outBuf should have size 6 + outBuf.toList.sorted shouldBe (List(("1", "w", "a"), ("1", "y", "b"), ("1", "z", "c"), ("2", "w", "d"), ("2", "y", "e"), ("2", "z", "f")).sorted) } } .sink[(String, String, String, String)](Tsv("pivot")) { outBuf => "pivot back to the original" in { - outBuf.size must_== 2 - outBuf.toList.sorted must be_== (input.sorted) + outBuf should have size 2 + outBuf.toList.sorted shouldBe (input.sorted) } } .sink[(String, String, String, String, Double)](Tsv("pivot_with_default")) { outBuf => "pivot back to the original with the missing column replace by the specified default" in { - outBuf.size must_== 2 - outBuf.toList.sorted must be_== (List(("1", "a", "b", "c", 2.0), ("2", "d", "e", "f", 2.0)).sorted) + outBuf should have size 2 + outBuf.toList.sorted shouldBe (List(("1", "a", "b", "c", 2.0), ("2", "d", "e", "f", 2.0)).sorted) } } .run @@ -1132,26 +1117,29 @@ class IterableSourceJob(args: Args) extends Job(args) { .joinWithTiny('x -> 0, list).write(Tsv("imp")) } -class IterableSourceTest extends Specification with FieldConversions { - noDetailedDiffs() +class IterableSourceTest extends WordSpec with Matchers with FieldConversions { val input = List((1, 10), (2, 20), (3, 30)) "A IterableSourceJob" should { - JobTest("com.twitter.scalding.IterableSourceJob") + var idx = 0 + JobTest(new IterableSourceJob(_)) .source(Tsv("in", ('x, 'w)), input) .sink[(Int, Int, Int, Int)](Tsv("out")) { outBuf => - "Correctly joinWithSmaller" in { - outBuf.toList.sorted must be_== (List((1, 10, 2, 3), (3, 30, 8, 9))) + s"$idx: Correctly joinWithSmaller" in { + outBuf.toList.sorted shouldBe List((1, 10, 2, 3), (3, 30, 8, 9)) } + idx += 1 } .sink[(Int, Int, Int, Int)](Tsv("tiny")) { outBuf => - "Correctly joinWithTiny" in { - outBuf.toList.sorted must be_== (List((1, 10, 2, 3), (3, 30, 8, 9))) + s"$idx: correctly joinWithTiny" in { + outBuf.toList.sorted shouldBe List((1, 10, 2, 3), (3, 30, 8, 9)) } + idx += 1 } .sink[(Int, Int, Int, Int, Int)](Tsv("imp")) { outBuf => - "Correctly implicitly joinWithTiny" in { - outBuf.toList.sorted must be_== (List((1, 10, 1, 2, 3), (3, 30, 3, 8, 9))) + s"$idx: correctly implicitly joinWithTiny" in { + outBuf.toList.sorted shouldBe List((1, 10, 1, 2, 3), (3, 30, 3, 8, 9)) } + idx += 1 } .run .runHadoop @@ -1166,16 +1154,15 @@ class HeadLastJob(args: Args) extends Job(args) { }.write(Tsv("output")) } -class HeadLastTest extends Specification { +class HeadLastTest extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() val input = List((1, 10), (1, 20), (1, 30), (2, 0)) "A HeadLastJob" should { - JobTest("com.twitter.scalding.HeadLastJob") + JobTest(new HeadLastJob(_)) .source(Tsv("input", ('x, 'y)), input) .sink[(Int, Int, Int)](Tsv("output")) { outBuf => "Correctly do head/last" in { - outBuf.toList must be_==(List((1, 10, 30), (2, 0, 0))) + outBuf.toList shouldBe List((1, 10, 30), (2, 0, 0)) } } .run @@ -1189,16 +1176,15 @@ class HeadLastUnsortedJob(args: Args) extends Job(args) { }.write(Tsv("output")) } -class HeadLastUnsortedTest extends Specification { +class HeadLastUnsortedTest extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() val input = List((1, 10), (1, 20), (1, 30), (2, 0)) "A HeadLastUnsortedTest" should { - JobTest("com.twitter.scalding.HeadLastUnsortedJob") + JobTest(new HeadLastUnsortedJob(_)) .source(Tsv("input", ('x, 'y)), input) .sink[(Int, Int, Int)](Tsv("output")) { outBuf => "Correctly do head/last" in { - outBuf.toList must be_==(List((1, 10, 30), (2, 0, 0))) + outBuf.toList shouldBe List((1, 10, 30), (2, 0, 0)) } } .run @@ -1214,15 +1200,14 @@ class MkStringToListJob(args: Args) extends Job(args) { }.write(Tsv("output")) } -class MkStringToListTest extends Specification with FieldConversions { - noDetailedDiffs() +class MkStringToListTest extends WordSpec with Matchers with FieldConversions { val input = List((1, 30), (1, 10), (1, 20), (2, 0)) "A IterableSourceJob" should { - JobTest("com.twitter.scalding.MkStringToListJob") + JobTest(new MkStringToListJob(_)) .source(Tsv("input", ('x, 'y)), input) .sink[(Int, String, List[Int])](Tsv("output")) { outBuf => "Correctly do mkString/toList" in { - outBuf.toSet must be_==(Set((1, "10,20,30", List(10, 20, 30)), (2, "0", List(0)))) + outBuf.toSet shouldBe Set((1, "10,20,30", List(10, 20, 30)), (2, "0", List(0))) } } .run @@ -1235,18 +1220,17 @@ class InsertJob(args: Args) extends Job(args) { Tsv("input", ('x, 'y)).insert(('z, 'w), (1, 2)).write(Tsv("output")) } -class InsertJobTest extends Specification { +class InsertJobTest extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() val input = List((2, 2), (3, 3)) "An InsertJob" should { - JobTest(new com.twitter.scalding.InsertJob(_)) + JobTest(new InsertJob(_)) .source(Tsv("input", ('x, 'y)), input) .sink[(Int, Int, Int, Int)](Tsv("output")) { outBuf => "Correctly insert a constant" in { - outBuf.toSet must be_==(Set((2, 2, 1, 2), (3, 3, 1, 2))) + outBuf.toSet shouldBe Set((2, 2, 1, 2), (3, 3, 1, 2)) } } .run @@ -1265,18 +1249,17 @@ class FoldJob(args: Args) extends Job(args) { }.write(Tsv("output")) } -class FoldJobTest extends Specification { +class FoldJobTest extends WordSpec with Matchers { import Dsl._ import scala.collection.mutable.{ Set => MSet } - noDetailedDiffs() val input = List((1, 30), (1, 10), (1, 20), (2, 0)) "A FoldTestJob" should { - JobTest("com.twitter.scalding.FoldJob") + JobTest(new FoldJob(_)) .source(Tsv("input", ('x, 'y)), input) .sink[(Int, MSet[Int])](Tsv("output")) { outBuf => "Correctly do a fold with MutableSet" in { - outBuf.toSet must be_==(Set((1, MSet(10, 20, 30)), (2, MSet(0)))) + outBuf.toSet shouldBe Set((1, MSet(10, 20, 30)), (2, MSet(0))) } } .run @@ -1296,17 +1279,16 @@ class InnerCaseJob(args: Args) extends Job(args) { .write(Tsv("output")) } -class InnerCaseTest extends Specification { +class InnerCaseTest extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() val input = List(Tuple1(1), Tuple1(2), Tuple1(2), Tuple1(4)) "An InnerCaseJob" should { - JobTest(new com.twitter.scalding.InnerCaseJob(_)) + JobTest(new InnerCaseJob(_)) .source(TypedTsv[Int]("input"), input) .sink[(Int, Int)](Tsv("output")) { outBuf => "Correctly handle inner case classes" in { - outBuf.toSet must be_==(Set((1, 1), (2, 4), (4, 16))) + outBuf.toSet shouldBe Set((1, 1), (2, 4), (4, 16)) } } .runHadoop @@ -1323,17 +1305,14 @@ class NormalizeJob(args: Args) extends Job(args) { .write(Tsv("out")) } -class NormalizeTest extends Specification { - noDetailedDiffs() - +class NormalizeTest extends WordSpec with Matchers { "A NormalizeJob" should { - JobTest("com.twitter.scalding.NormalizeJob") - .source(Tsv("in"), List(("0.3", "1"), ("0.3", "1"), ("0.3", - "1"), ("0.3", "1"))) + JobTest(new NormalizeJob(_)) + .source(Tsv("in"), List(("0.3", "1"), ("0.3", "1"), ("0.3", "1"), ("0.3", "1"))) .sink[(Double, Int)](Tsv("out")) { outBuf => "must be normalized" in { - outBuf.size must_== 4 - outBuf.toSet must_== (Set((0.25, 1), (0.25, 1), (0.25, 1), (0.25, 1))) + outBuf should have size 4 + outBuf.toSet shouldBe Set((0.25, 1), (0.25, 1), (0.25, 1), (0.25, 1)) } } .run @@ -1348,20 +1327,19 @@ class ApproxUniqJob(args: Args) extends Job(args) { .write(Tsv("out")) } -class ApproxUniqTest extends Specification { +class ApproxUniqTest extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() "A ApproxUniqJob" should { val input = (1 to 1000).flatMap { i => List(("x0", i), ("x1", i)) }.toList - JobTest("com.twitter.scalding.ApproxUniqJob") + JobTest(new ApproxUniqJob(_)) .source(Tsv("in", ('x, 'y)), input) .sink[(String, Double)](Tsv("out")) { outBuf => "must approximately count" in { - outBuf.size must_== 2 + outBuf should have size 2 val kvresult = outBuf.groupBy { _._1 }.mapValues { _.head._2 } - kvresult("x0") must beCloseTo(1000.0, 30.0) //We should be 1%, but this is on average, so - kvresult("x1") must beCloseTo(1000.0, 30.0) //We should be 1%, but this is on average, so + 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 @@ -1381,20 +1359,21 @@ class ForceToDiskJob(args: Args) extends Job(args) { .write(Tsv("out")) } -class ForceToDiskTest extends Specification { +class ForceToDiskTest extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() "A ForceToDiskJob" should { + var idx = 0 val input = (1 to 1000).flatMap { i => List((-1, i), (1, i)) }.toList JobTest(new ForceToDiskJob(_)) .source(Tsv("in", ('x, 'y)), input) .sink[(Int, Int, Int)](Tsv("out")) { outBuf => - "run correctly when combined with joinWithTiny" in { - outBuf.size must_== 2000 + (idx + ": run correctly when combined with joinWithTiny") in { + outBuf should have size 2000 val correct = (1 to 1000).flatMap { y => List((1, 1, y), (-1, 1, y)) }.sorted - outBuf.toList.sorted must_== correct + outBuf.toList.sorted shouldBe correct } + idx += 1 } .run .runHadoop @@ -1413,10 +1392,9 @@ class ThrowsErrorsJob(args: Args) extends Job(args) { .write(Tsv("output")) } -class ItsATrapTest extends Specification { +class ItsATrapTest extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() //Fixes an issue with scala 2.9 "An AddTrap" should { val input = List(("a", 1), ("b", 2), ("c", 3), ("d", 1), ("e", 2)) @@ -1424,12 +1402,12 @@ class ItsATrapTest extends Specification { .source(Tsv("input", ('letter, 'x)), input) .sink[(String, Int)](Tsv("output")) { outBuf => "must contain all numbers in input except for 1" in { - outBuf.toList.sorted must be_==(List(("b", 2), ("c", 3), ("e", 2))) + outBuf.toList.sorted shouldBe List(("b", 2), ("c", 3), ("e", 2)) } } .sink[(String, Int)](Tsv("trapped")) { outBuf => "must contain all 1s and fields in input" in { - outBuf.toList.sorted must be_==(List(("a", 1), ("d", 1))) + outBuf.toList.sorted shouldBe List(("a", 1), ("d", 1)) } } .run @@ -1483,10 +1461,9 @@ class TypedThrowsErrorsJob2(args: Args) extends Job(args) { .write(output) } -class TypedItsATrapTest extends Specification { +class TypedItsATrapTest extends WordSpec with Matchers { import TDsl._ - noDetailedDiffs() //Fixes an issue with scala 2.9 "A Typed AddTrap with many traps" should { import TypedThrowsErrorsJob._ @@ -1496,17 +1473,17 @@ class TypedItsATrapTest extends Specification { .source(input, data) .typedSink(output) { outBuf => "output must contain all odd except first" in { - outBuf.toList.sorted must be_==(List(("c", 3), ("e", 5))) + outBuf.toList.sorted shouldBe List(("c", 3), ("e", 5)) } } .typedSink(trap1) { outBuf => "trap1 must contain only the first" in { - outBuf.toList.sorted must be_==(List(("a", 1, 1))) + outBuf.toList.sorted shouldBe List(("a", 1, 1)) } } .typedSink(trap2) { outBuf => "trap2 must contain the even numbered" in { - outBuf.toList.sorted must be_==(List(("b", 2, 4, "b"), ("d", 4, 16, "d"))) + outBuf.toList.sorted shouldBe List(("b", 2, 4, "b"), ("d", 4, 16, "d")) } } .run @@ -1522,12 +1499,12 @@ class TypedItsATrapTest extends Specification { .source(input, data) .typedSink(output) { outBuf => "output must contain all odd except first" in { - outBuf.toList.sorted must be_==(List(("c", 3), ("e", 5))) + outBuf.toList.sorted shouldBe List(("c", 3), ("e", 5)) } } - .typedSink(trap) { outBuf => + .typedSink(TypedThrowsErrorsJob2.trap) { outBuf => "trap must contain the first and the evens" in { - outBuf.toList.sorted must be_==(List(("a", 1, 1), ("b", 2, 2), ("d", 4, 4))) + outBuf.toList.sorted shouldBe List(("a", 1, 1), ("b", 2, 2), ("d", 4, 4)) } } .run @@ -1550,11 +1527,9 @@ class GroupAllToListTestJob(args: Args) extends Job(args) { .write(Tsv("output")) } -class GroupAllToListTest extends Specification { +class GroupAllToListTest extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() - "A GroupAllToListTestJob" should { val input = List((1L, "a", 1.0), (1L, "b", 2.0), (2L, "a", 1.0), (2L, "b", 2.0)) val output = Map(2L -> Map("a" -> 1.0, "b" -> 2.0), 1L -> Map("a" -> 1.0, "b" -> 2.0)) @@ -1562,8 +1537,8 @@ class GroupAllToListTest extends Specification { .source(TypedTsv[(Long, String, Double)]("input"), input) .sink[String](Tsv("output")) { outBuf => "must properly aggregate stuff into a single map" in { - outBuf.size must_== 1 - outBuf(0) must be_==(output.toString) + outBuf should have size 1 + outBuf(0) shouldBe output.toString } } .runHadoop @@ -1582,11 +1557,9 @@ class ToListGroupAllToListTestJob(args: Args) extends Job(args) { .write(Tsv("output")) } -class ToListGroupAllToListSpec extends Specification { +class ToListGroupAllToListSpec extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() - val expected = List(("us", List(1)), ("jp", List(3, 2)), ("gb", List(3, 1))) "A ToListGroupAllToListTestJob" should { @@ -1594,8 +1567,8 @@ class ToListGroupAllToListSpec extends Specification { .source(TypedTsv[(Long, String)]("input"), List((1L, "us"), (1L, "gb"), (2L, "jp"), (3L, "jp"), (3L, "gb"))) .sink[String](Tsv("output")) { outBuf => "must properly aggregate stuff in hadoop mode" in { - outBuf.size must_== 1 - outBuf.head must_== expected.toString + outBuf should have size 1 + outBuf.head shouldBe (expected.toString) println(outBuf.head) } } @@ -1606,8 +1579,8 @@ class ToListGroupAllToListSpec extends Specification { .source(TypedTsv[(Long, String)]("input"), List((1L, "us"), (1L, "gb"), (2L, "jp"), (3L, "jp"), (3L, "gb"))) .sink[List[(String, List[Long])]](Tsv("output")) { outBuf => "must properly aggregate stuff in local model" in { - outBuf.size must_== 1 - outBuf.head must_== expected + outBuf should have size 1 + outBuf.head shouldBe expected println(outBuf.head) } } @@ -1660,17 +1633,16 @@ class Function2Job(args: Args) extends Job(args) { Tsv("in", ('x, 'y)).mapTo(('x, 'y) -> 'xy) { (x: String, y: String) => x + y }.write(Tsv("output")) } -class Function2Test extends Specification { +class Function2Test extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() //Fixes an issue with scala 2.9 "A Function2Job" should { val input = List(("a", "b")) - JobTest("com.twitter.scalding.Function2Job") + JobTest(new Function2Job(_)) .source(Tsv("in", ('x, 'y)), input) .sink[String](Tsv("output")) { outBuf => "convert a function2 to tupled function1" in { - outBuf must be_==(List("ab")) + outBuf shouldBe List("ab") } } .run @@ -1684,7 +1656,7 @@ class SampleWithReplacementJob(args: Args) extends Job(args) { .write(Tsv("output")) } -class SampleWithReplacementTest extends Specification { +class SampleWithReplacementTest extends WordSpec with Matchers { import com.twitter.scalding.mathematics.Poisson val p = new Poisson(1.0, 0) @@ -1692,16 +1664,14 @@ class SampleWithReplacementTest extends Specification { i => i -> p.nextInt }.filterNot(_._2 == 0).toSet - noDetailedDiffs() "A SampleWithReplacementJob" should { - JobTest("com.twitter.scalding.SampleWithReplacementJob") + JobTest(new SampleWithReplacementJob(_)) .source(Tsv("in"), (1 to 100).map(i => i)) .sink[Int](Tsv("output")) { outBuf => - () "sampleWithReplacement must sample items according to a poisson distribution" in { outBuf.toList.groupBy(i => i) .map(p => p._1 -> p._2.size) - .filterNot(_._2 == 0).toSet must_== simulated + .filterNot(_._2 == 0).toSet shouldBe simulated } } .run @@ -1717,19 +1687,19 @@ class VerifyTypesJob(args: Args) extends Job(args) { .write(Tsv("output")) } -class VerifyTypesJobTest extends Specification { +class VerifyTypesJobTest extends WordSpec with Matchers { "Verify types operation" should { "put bad records in a trap" in { val input = List((3, "aaa"), (23, 154), (15, "123"), (53, 143), (7, 85), (19, 195), (42, 187), (35, 165), (68, 121), (13, "34"), (17, 173), (2, 13), (2, "break")) - JobTest(new com.twitter.scalding.VerifyTypesJob(_)) + JobTest(new VerifyTypesJob(_)) .source(Tsv("input", new Fields("age", "weight")), input) .sink[(Int, Int)](Tsv("output")) { outBuf => - outBuf.toList.size must_== input.size - 2 + outBuf.toList should have size (input.size - 2) } .sink[(Any, Any)](Tsv("trap")) { outBuf => - outBuf.toList.size must_== 2 + outBuf.toList should have size 2 } .run .finish @@ -1745,16 +1715,15 @@ class SortingJob(args: Args) extends Job(args) { .write(Tsv("output")) } -class SortingJobTest extends Specification { +class SortingJobTest extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() "A SortingJob" should { JobTest(new SortingJob(_)) .source(Tsv("in", ('x, 'y, 'z)), (1 to 100).map(i => (i, i * i % 5, i * i * i))) .sink[(Int, Int, Int)](Tsv("output")) { outBuf => "keep all the columns" in { val correct = (1 to 100).map(i => (i, i * i % 5, i * i * i)).toList.sortBy(_._2) - outBuf.toList must_== (correct) + outBuf.toList shouldBe correct } } .run @@ -1768,18 +1737,18 @@ class CollectJob(args: Args) extends Job(args) { .write(Tsv("output")) } -class CollectJobTest extends Specification { - noDetailedDiffs() +class CollectJobTest extends WordSpec with Matchers { "A CollectJob" should { val input = List(("steve m", 21), ("john f", 89), ("s smith", 12), ("jill q", 55), ("some child", 8)) val expectedOutput = input.collect{ case (name, age) if age > 18 => name.split(" ").head } - JobTest(new com.twitter.scalding.CollectJob(_)) + JobTest(new CollectJob(_)) .source(Tsv("input", new Fields("name", "age")), input) .sink[String](Tsv("output")) { outBuf => - outBuf.toList must be_==(expectedOutput) + outBuf.toList shouldBe expectedOutput } - .run.finish + .run + .finish } } @@ -1789,8 +1758,7 @@ class FilterJob(args: Args) extends Job(args) { .write(Tsv("output")) } -class FilterJobTest extends Specification { - noDetailedDiffs() +class FilterJobTest extends WordSpec with Matchers { "A FilterJob" should { val input = List(("steve m", 21), ("john f", 89), ("s smith", 12), ("jill q", 55), ("some child", 8)) val expectedOutput = input.filter(_._2 > 18) @@ -1798,9 +1766,10 @@ class FilterJobTest extends Specification { JobTest(new com.twitter.scalding.FilterJob(_)) .source(Tsv("input", new Fields("name", "age")), input) .sink[(String, Int)](Tsv("output")) { outBuf => - outBuf.toList must be_==(expectedOutput) + outBuf.toList shouldBe expectedOutput } - .run.finish + .run + .finish } } @@ -1810,8 +1779,7 @@ class FilterNotJob(args: Args) extends Job(args) { .write(Tsv("output")) } -class FilterNotJobTest extends Specification { - noDetailedDiffs() +class FilterNotJobTest extends WordSpec with Matchers { "A FilterNotJob" should { val input = List(("steve m", 21), ("john f", 89), ("s smith", 12), ("jill q", 55), ("some child", 8)) val expectedOutput = input.filterNot(_._2 > 18) @@ -1819,9 +1787,10 @@ class FilterNotJobTest extends Specification { JobTest(new com.twitter.scalding.FilterNotJob(_)) .source(Tsv("input", new Fields("name", "age")), input) .sink[(String, Int)](Tsv("output")) { outBuf => - outBuf.toList must be_==(expectedOutput) + outBuf.toList shouldBe expectedOutput } - .run.finish + .run + .finish } } @@ -1829,7 +1798,6 @@ class CounterJob(args: Args) extends Job(args) { val foo_bar = Stat("foo_bar") val age_group_older_than_18 = Stat("age_group_older_than_18") val reduce_hit = Stat("reduce_hit") - age_group_older_than_18 Tsv("input", new Fields("name", "age")) .filter('age){ age: Int => foo_bar.incBy(2) @@ -1850,23 +1818,22 @@ class CounterJob(args: Args) extends Job(args) { .write(Tsv("output")) } -class CounterJobTest extends Specification { - noDetailedDiffs() +class CounterJobTest extends WordSpec with Matchers { "A CounterJob" should { val input = List(("steve m", 21), ("john f", 89), ("s smith", 12), ("jill q", 55), ("some child", 8)) val expectedOutput = input.collect{ case (name, age) if age > 18 => age }.sum.toString "have the right counter and output values" in { - JobTest(new com.twitter.scalding.CounterJob(_)) + JobTest(new CounterJob(_)) .source(Tsv("input", new Fields("name", "age")), input) - .sink[String](Tsv("output")) { outBuf => outBuf(0) must be_==(expectedOutput) } - .counter("foo_bar") { _ must_== 10 } - .counter("age_group_older_than_18") { _ must_== 3 } - .counter("reduce_hit") { _ must_== 2 } - .counter("bad_group_bad_counter") { _ must_== 0 } + .sink[String](Tsv("output")) { outBuf => outBuf(0) shouldBe expectedOutput } + .counter("foo_bar") { _ shouldBe 10 } + .counter("age_group_older_than_18") { _ shouldBe 3 } + .counter("reduce_hit") { _ shouldBe 2 } + .counter("bad_group_bad_counter") { _ shouldBe 0 } // This is redundant but just added here to show both methods for counter tests .counters { - _ must_== Map( + _ shouldBe Map( "foo_bar" -> 10, "age_group_older_than_18" -> 3, "reduce_hit" -> 2) @@ -1892,9 +1859,7 @@ class DailySuffixTsvJob(args: Args) extends Job(args) with UtcDateRangeJob { DailySuffixTsvJob.source("input0").read.toTypedPipe[(String, Int)]((0, 1)).write(TypedTsv[(String, Int)]("output0")) } -class DailySuffixTsvTest extends Specification { - noDetailedDiffs() - +class DailySuffixTsvTest extends WordSpec with Matchers { val data = List(("aaa", 1), ("bbb", 2)) "A DailySuffixTsv Source" should { @@ -1904,7 +1869,7 @@ class DailySuffixTsvTest extends Specification { .source(source("input0"), data) .sink[(String, Int)](TypedTsv[(String, Int)]("output0")) { buf => "read and write data" in { - buf must be_==(data) + buf shouldBe data } } .run 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 df76ab3fe9..7f8fb5faf7 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/CumulitiveSumTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/CumulitiveSumTest.scala @@ -1,6 +1,7 @@ package com.twitter.scalding -import org.specs._ +import org.scalatest.WordSpec + import com.twitter.scalding._ import com.twitter.scalding.typed.CumulativeSum._ @@ -33,7 +34,7 @@ class AddRankingWithPartitionedCumulativeSum(args: Args) extends Job(args) { .write(TypedTsv("result1")) } -class CumulativeSumTest1 extends Specification { +class CumulativeSumTest1 extends WordSpec { // --- A simple ranking job val sampleInput1 = List( @@ -66,10 +67,10 @@ class CumulativeSumTest1 extends Specification { .source(TypedTsv[(String, Double)]("input1"), sampleInput1) .sink[(String, Double, Long)](TypedTsv[(String, Double, Long)]("result1")) { outBuf1 => "produce correct number of records when filtering out null values" in { - outBuf1.size must_== 10 + assert(outBuf1.size === 10) } "create correct ranking per group, 1st being the heighest person of that group" in { - outBuf1.toSet must_== expectedOutput1 + assert(outBuf1.toSet === expectedOutput1) } } .run @@ -81,10 +82,10 @@ class CumulativeSumTest1 extends Specification { .source(TypedTsv[(String, Double)]("input1"), sampleInput1) .sink[(String, Double, Long)](TypedTsv[(String, Double, Long)]("result1")) { outBuf1 => "produce correct number of records when filtering out null values" in { - outBuf1.size must_== 10 + assert(outBuf1.size === 10) } "create correct ranking per group, 1st being the heighest person of that group" in { - outBuf1.toSet must_== expectedOutput1 + assert(outBuf1.toSet === expectedOutput1) } } .run diff --git a/scalding-core/src/test/scala/com/twitter/scalding/FieldImpsTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/FieldImpsTest.scala index 0eb8c00407..2b1bcfae1d 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/FieldImpsTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/FieldImpsTest.scala @@ -17,25 +17,20 @@ package com.twitter.scalding import cascading.tuple.Fields -import org.specs._ +import org.scalatest.{ Matchers, WordSpec } -class FieldImpsTest extends Specification with FieldConversions { - noDetailedDiffs() //Fixes issue for scala 2.9 +class FieldImpsTest extends WordSpec with Matchers with FieldConversions { def setAndCheck[T <: Comparable[_]](v: T)(implicit conv: (T) => Fields) { - val vF = conv(v) - vF.equals(new Fields(v)) must beTrue + conv(v) shouldBe (new Fields(v)) } def setAndCheckS[T <: Comparable[_]](v: Seq[T])(implicit conv: (Seq[T]) => Fields) { - val vF = conv(v) - vF.equals(new Fields(v: _*)) must beTrue + conv(v) shouldBe (new Fields(v: _*)) } def setAndCheckSym(v: Symbol) { - val vF: Fields = v - vF.equals(new Fields(v.toString.tail)) must beTrue + (v: Fields) shouldBe (new Fields(v.toString.tail)) } def setAndCheckSymS(v: Seq[Symbol]) { - val vF: Fields = v - vF.equals(new Fields(v.map(_.toString.tail): _*)) must beTrue + (v: Fields) shouldBe (new Fields(v.map(_.toString.tail): _*)) } def setAndCheckField(v: Field[_]) { val vF: Fields = v @@ -50,24 +45,22 @@ class FieldImpsTest extends Specification with FieldConversions { checkFieldsWithComparators(vF, fields) } def setAndCheckEnumValue(v: Enumeration#Value) { - val vF: Fields = v - vF.equals(new Fields(v.toString)) must beTrue + (v: Fields) shouldBe (new Fields(v.toString)) } def setAndCheckEnumValueS(v: Seq[Enumeration#Value]) { - val vF: Fields = v - vF.equals(new Fields(v.map(_.toString): _*)) must beTrue + (v: Fields) shouldBe (new Fields(v.map(_.toString): _*)) } def checkFieldsWithComparators(actual: Fields, expected: Fields) { // sometimes one or the other is actually a RichFields, so rather than test for // actual.equals(expected), we just check that all the field names and comparators line up - actual.size must_== expected.size - (asList(actual), asList(expected)).zipped.forall(_.equals(_)) must beTrue - actual.getComparators.toSeq.equals(expected.getComparators.toSeq) must beTrue + actual should have size (expected.size) + asList(actual) shouldBe asList(expected) + actual.getComparators.toSeq shouldBe (expected.getComparators.toSeq) } "Field" should { "contain manifest" in { val field = Field[Long]("foo") - field.mf mustEqual Some(implicitly[Manifest[Long]]) + field.mf should contain (implicitly[Manifest[Long]]) } } "RichFields" should { @@ -76,24 +69,24 @@ class FieldImpsTest extends Specification with FieldConversions { val f2 = Field[String]('bar) val rf = RichFields(f1, f2) val fields: Fields = rf - fields.size mustEqual 2 - f1.id mustEqual fields.get(0) - f2.id mustEqual fields.get(1) - f1.ord mustEqual fields.getComparators()(0) - f2.ord mustEqual fields.getComparators()(1) + fields should have size 2 + f1.id shouldBe (fields.get(0)) + f2.id shouldBe (fields.get(1)) + f1.ord shouldBe (fields.getComparators()(0)) + f2.ord shouldBe (fields.getComparators()(1)) } "convert from Fields" in { val fields = new Fields("foo", "bar") val comparator = implicitly[Ordering[String]] fields.setComparators(comparator, comparator) val fieldList: List[Field[_]] = fields.toFieldList - fieldList mustEqual List(new StringField[String]("foo")(comparator, None), new StringField[String]("bar")(comparator, None)) + fieldList shouldBe List(new StringField[String]("foo")(comparator, None), new StringField[String]("bar")(comparator, None)) } "throw an exception on when converting a virtual Fields instance" in { import Fields._ List(ALL, ARGS, FIRST, GROUP, LAST, NONE, REPLACE, RESULTS, SWAP, UNKNOWN, VALUES).foreach { fields => - fields.toFieldList must throwA[Exception] + an[Exception] should be thrownBy fields.toFieldList } } } @@ -140,34 +133,33 @@ class FieldImpsTest extends Specification with FieldConversions { object Schema extends Enumeration { val one, two, three = Value } - var vf: Fields = Schema - vf must be_==(new Fields("one", "two", "three")) + (Schema: Fields) shouldBe (new Fields("one", "two", "three")) } "convert from general int tuples" in { var vf: Fields = Tuple1(1) - vf must be_==(new Fields(int2Integer(1))) + vf shouldBe (new Fields(int2Integer(1))) vf = (1, 2) - vf must be_==(new Fields(int2Integer(1), int2Integer(2))) + vf shouldBe (new Fields(int2Integer(1), int2Integer(2))) vf = (1, 2, 3) - vf must be_==(new Fields(int2Integer(1), int2Integer(2), int2Integer(3))) + vf shouldBe (new Fields(int2Integer(1), int2Integer(2), int2Integer(3))) vf = (1, 2, 3, 4) - vf must be_==(new Fields(int2Integer(1), int2Integer(2), int2Integer(3), int2Integer(4))) + vf shouldBe (new Fields(int2Integer(1), int2Integer(2), int2Integer(3), int2Integer(4))) } "convert from general string tuples" in { var vf: Fields = Tuple1("hey") - vf must be_==(new Fields("hey")) + vf shouldBe (new Fields("hey")) vf = ("hey", "world") - vf must be_==(new Fields("hey", "world")) + vf shouldBe (new Fields("hey", "world")) vf = ("foo", "bar", "baz") - vf must be_==(new Fields("foo", "bar", "baz")) + vf shouldBe (new Fields("foo", "bar", "baz")) } "convert from general symbol tuples" in { var vf: Fields = Tuple1('hey) - vf must be_==(new Fields("hey")) + vf shouldBe (new Fields("hey")) vf = ('hey, 'world) - vf must be_==(new Fields("hey", "world")) + vf shouldBe (new Fields("hey", "world")) vf = ('foo, 'bar, 'baz) - vf must be_==(new Fields("foo", "bar", "baz")) + vf shouldBe (new Fields("foo", "bar", "baz")) } "convert from general com.twitter.scalding.Field tuples" in { val foo = Field[java.math.BigInteger]("foo") @@ -195,52 +187,52 @@ class FieldImpsTest extends Specification with FieldConversions { } import Schema._ var vf: Fields = Tuple1(one) - vf must be_==(new Fields("one")) + vf shouldBe (new Fields("one")) vf = (one, two) - vf must be_==(new Fields("one", "two")) + vf shouldBe (new Fields("one", "two")) vf = (one, two, three) - vf must be_==(new Fields("one", "two", "three")) + vf shouldBe (new Fields("one", "two", "three")) } "convert to a pair of Fields from a pair of values" in { var f2: (Fields, Fields) = "hey" -> "you" - f2 must be_==((new Fields("hey"), new Fields("you"))) + f2 shouldBe (new Fields("hey"), new Fields("you")) f2 = 'hey -> 'you - f2 must be_==((new Fields("hey"), new Fields("you"))) + f2 shouldBe (new Fields("hey"), new Fields("you")) f2 = (0 until 10) -> 'you - f2 must be_==((new Fields((0 until 10).map(int2Integer): _*), new Fields("you"))) + f2 shouldBe (new Fields((0 until 10).map(int2Integer): _*), new Fields("you")) f2 = (('hey, 'world) -> 'other) - f2 must be_==((new Fields("hey", "world"), new Fields("other"))) + f2 shouldBe (new Fields("hey", "world"), new Fields("other")) f2 = 0 -> 2 - f2 must be_==((new Fields(int2Integer(0)), new Fields(int2Integer(2)))) + f2 shouldBe (new Fields(int2Integer(0)), new Fields(int2Integer(2))) f2 = (0, (1, "you")) - f2 must be_==((new Fields(int2Integer(0)), new Fields(int2Integer(1), "you"))) + f2 shouldBe (new Fields(int2Integer(0)), new Fields(int2Integer(1), "you")) val foo = Field[java.math.BigInteger]("foo") val bar = Field[java.math.BigDecimal]("bar") f2 = ((foo, bar) -> 'bell) var fields = new Fields("foo", "bar") fields.setComparators(foo.ord, bar.ord) - f2 must be_==((fields, new Fields("bell"))) + f2 shouldBe (fields, new Fields("bell")) f2 = (foo -> ('bar, 'bell)) fields = RichFields(foo) fields.setComparators(foo.ord) - f2 must be_==((fields, new Fields("bar", "bell"))) + f2 shouldBe (fields, new Fields("bar", "bell")) f2 = Seq("one", "two", "three") -> Seq("1", "2", "3") - f2 must be_==((new Fields("one", "two", "three"), new Fields("1", "2", "3"))) + f2 shouldBe (new Fields("one", "two", "three"), new Fields("1", "2", "3")) f2 = List("one", "two", "three") -> List("1", "2", "3") - f2 must be_==((new Fields("one", "two", "three"), new Fields("1", "2", "3"))) + f2 shouldBe (new Fields("one", "two", "three"), new Fields("1", "2", "3")) f2 = List('one, 'two, 'three) -> List('n1, 'n2, 'n3) - f2 must be_==((new Fields("one", "two", "three"), new Fields("n1", "n2", "n3"))) + f2 shouldBe (new Fields("one", "two", "three"), new Fields("n1", "n2", "n3")) f2 = List(4, 5, 6) -> List(1, 2, 3) - f2 must be_==((new Fields(int2Integer(4), int2Integer(5), int2Integer(6)), - new Fields(int2Integer(1), int2Integer(2), int2Integer(3)))) + f2 shouldBe (new Fields(int2Integer(4), int2Integer(5), int2Integer(6)), + new Fields(int2Integer(1), int2Integer(2), int2Integer(3))) object Schema extends Enumeration { val one, two, three = Value @@ -248,40 +240,40 @@ class FieldImpsTest extends Specification with FieldConversions { import Schema._ f2 = one -> two - f2 must be_==((new Fields("one"), new Fields("two"))) + f2 shouldBe (new Fields("one"), new Fields("two")) f2 = (one, two) -> three - f2 must be_==((new Fields("one", "two"), new Fields("three"))) + f2 shouldBe (new Fields("one", "two"), new Fields("three")) f2 = one -> (two, three) - f2 must be_==((new Fields("one"), new Fields("two", "three"))) + f2 shouldBe (new Fields("one"), new Fields("two", "three")) } "correctly see if there are ints" in { - hasInts(0) must beTrue - hasInts((0, 1)) must beTrue - hasInts('hey) must beFalse - hasInts((0, 'hey)) must beTrue - hasInts(('hey, 9)) must beTrue - hasInts(('a, 'b)) must beFalse + hasInts(0) shouldBe true + hasInts((0, 1)) shouldBe true + hasInts('hey) shouldBe false + hasInts((0, 'hey)) shouldBe true + hasInts(('hey, 9)) shouldBe true + hasInts(('a, 'b)) shouldBe false def i(xi: Int) = new java.lang.Integer(xi) - asSet(0) must be_==(Set(i(0))) - asSet((0, 1, 2)) must be_==(Set(i(0), i(1), i(2))) - asSet((0, 1, 'hey)) must be_==(Set(i(0), i(1), "hey")) + asSet(0) shouldBe Set(i(0)) + asSet((0, 1, 2)) shouldBe Set(i(0), i(1), i(2)) + asSet((0, 1, 'hey)) shouldBe Set(i(0), i(1), "hey") } "correctly determine default modes" in { //Default case: - defaultMode(0, 'hey) must be_==(Fields.ALL) - defaultMode((0, 't), 'x) must be_==(Fields.ALL) - defaultMode(('hey, 'x), 'y) must be_==(Fields.ALL) + defaultMode(0, 'hey) shouldBe Fields.ALL + defaultMode((0, 't), 'x) shouldBe Fields.ALL + defaultMode(('hey, 'x), 'y) shouldBe Fields.ALL //Equal: - defaultMode('hey, 'hey) must be_==(Fields.REPLACE) - defaultMode(('hey, 'x), ('hey, 'x)) must be_==(Fields.REPLACE) - defaultMode(0, 0) must be_==(Fields.REPLACE) + defaultMode('hey, 'hey) shouldBe Fields.REPLACE + defaultMode(('hey, 'x), ('hey, 'x)) shouldBe Fields.REPLACE + defaultMode(0, 0) shouldBe Fields.REPLACE //Subset/superset: - defaultMode(('hey, 'x), 'x) must be_==(Fields.SWAP) - defaultMode('x, ('hey, 'x)) must be_==(Fields.SWAP) - defaultMode(0, ('hey, 0)) must be_==(Fields.SWAP) - defaultMode(('hey, 0), 0) must be_==(Fields.SWAP) + defaultMode(('hey, 'x), 'x) shouldBe Fields.SWAP + defaultMode('x, ('hey, 'x)) shouldBe Fields.SWAP + defaultMode(0, ('hey, 0)) shouldBe Fields.SWAP + defaultMode(('hey, 0), 0) shouldBe Fields.SWAP } } 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 5720a590a1..f6872aea7e 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/FileSourceTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/FileSourceTest.scala @@ -15,7 +15,7 @@ limitations under the License. */ package com.twitter.scalding -import org.specs._ +import org.scalatest.{ Matchers, WordSpec } import org.apache.hadoop.conf.Configuration class MultiTsvInputJob(args: Args) extends Job(args) { @@ -36,8 +36,7 @@ class SequenceFileInputJob(args: Args) extends Job(args) { } } -class FileSourceTest extends Specification { - noDetailedDiffs() +class FileSourceTest extends WordSpec with Matchers { import Dsl._ "A MultipleTsvFile Source" should { @@ -47,8 +46,8 @@ class FileSourceTest extends Specification { sink[(String, Int)](Tsv("output0")) { outBuf => "take multiple Tsv files as input sources" in { - outBuf.length must be_==(2) - outBuf.toList must be_==(List(("foobar", 1), ("helloworld", 2))) + outBuf should have length 2 + outBuf.toList shouldBe List(("foobar", 1), ("helloworld", 2)) } } .run @@ -64,15 +63,15 @@ class FileSourceTest extends Specification { sink[(String, Int)](SequenceFile("output0")) { outBuf => "sequence file input" in { - outBuf.length must be_==(2) - outBuf.toList must be_==(List(("foobar0", 1), ("helloworld0", 2))) + outBuf should have length 2 + outBuf.toList shouldBe List(("foobar0", 1), ("helloworld0", 2)) } } .sink[(String, Int)](WritableSequenceFile("output1", ('query, 'queryStats))) { outBuf => "writable sequence file input" in { - outBuf.length must be_==(2) - outBuf.toList must be_==(List(("foobar1", 1), ("helloworld1", 2))) + outBuf should have length 2 + outBuf.toList shouldBe List(("foobar1", 1), ("helloworld1", 2)) } } .run @@ -80,8 +79,7 @@ class FileSourceTest extends Specification { } "TextLine.toIterator" should { "correctly read strings" in { - TextLine("../tutorial/data/hello.txt").toIterator(Config.default, Local(true)).toList must be_==( - List("Hello world", "Goodbye world")) + TextLine("../tutorial/data/hello.txt").toIterator(Config.default, Local(true)).toList shouldBe List("Hello world", "Goodbye world") } } @@ -104,26 +102,26 @@ class FileSourceTest extends Specification { import TestFileSource.pathIsGood "accept a directory with data in it" in { - pathIsGood("test_data/2013/03/") must be_==(true) - pathIsGood("test_data/2013/03/*") must be_==(true) + pathIsGood("test_data/2013/03/") shouldBe true + pathIsGood("test_data/2013/03/*") shouldBe true } "accept a directory with data and _SUCCESS in it" in { - pathIsGood("test_data/2013/04/") must be_==(true) - pathIsGood("test_data/2013/04/*") must be_==(true) + pathIsGood("test_data/2013/04/") shouldBe true + pathIsGood("test_data/2013/04/*") shouldBe true } "reject an empty directory" in { - pathIsGood("test_data/2013/05/") must be_==(false) - pathIsGood("test_data/2013/05/*") must be_==(false) + pathIsGood("test_data/2013/05/") shouldBe false + pathIsGood("test_data/2013/05/*") shouldBe false } "reject a directory with only _SUCCESS when specified as a glob" in { - pathIsGood("test_data/2013/06/*") must be_==(false) + pathIsGood("test_data/2013/06/*") shouldBe false } "accept a directory with only _SUCCESS when specified without a glob" in { - pathIsGood("test_data/2013/06/") must be_==(true) + pathIsGood("test_data/2013/06/") shouldBe true } } @@ -131,29 +129,29 @@ class FileSourceTest extends Specification { import TestSuccessFileSource.pathIsGood "reject a directory with data in it but no _SUCCESS file" in { - pathIsGood("test_data/2013/03/") must be_==(false) - pathIsGood("test_data/2013/03/*") must be_==(false) + pathIsGood("test_data/2013/03/") shouldBe false + pathIsGood("test_data/2013/03/*") shouldBe false } "accept a directory with data and _SUCCESS in it when specified as a glob" in { - pathIsGood("test_data/2013/04/*") must be_==(true) + pathIsGood("test_data/2013/04/*") shouldBe true } "reject a directory with data and _SUCCESS in it when specified without a glob" in { - pathIsGood("test_data/2013/04/") must be_==(false) + pathIsGood("test_data/2013/04/") shouldBe false } "reject an empty directory" in { - pathIsGood("test_data/2013/05/") must be_==(false) - pathIsGood("test_data/2013/05/*") must be_==(false) + pathIsGood("test_data/2013/05/") shouldBe false + pathIsGood("test_data/2013/05/*") shouldBe false } "reject a directory with only _SUCCESS when specified as a glob" in { - pathIsGood("test_data/2013/06/*") must be_==(false) + pathIsGood("test_data/2013/06/*") shouldBe false } "reject a directory with only _SUCCESS when specified without a glob" in { - pathIsGood("test_data/2013/06/") must be_==(false) + pathIsGood("test_data/2013/06/") shouldBe false } } diff --git a/scalding-core/src/test/scala/com/twitter/scalding/IntegralCompTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/IntegralCompTest.scala index 390903a36e..6299d08c3b 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/IntegralCompTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/IntegralCompTest.scala @@ -14,54 +14,55 @@ See the License for the specific language governing permissions and limitations under the License. */ package com.twitter.scalding -import org.specs._ -class IntegralCompTest extends Specification { +import org.scalatest.{ Matchers, WordSpec } + +class IntegralCompTest extends WordSpec with Matchers { def box[T](t: T) = t.asInstanceOf[AnyRef] "IntegralComparator" should { val intComp = new IntegralComparator "recognize integral types" in { - intComp.isIntegral(box(1)) must beTrue - intComp.isIntegral(box(1L)) must beTrue - intComp.isIntegral(box(1.asInstanceOf[Short])) must beTrue + intComp.isIntegral(box(1)) shouldBe true + intComp.isIntegral(box(1L)) shouldBe true + intComp.isIntegral(box(1.asInstanceOf[Short])) shouldBe true //Boxed - intComp.isIntegral(new java.lang.Long(2)) must beTrue - intComp.isIntegral(new java.lang.Integer(2)) must beTrue - intComp.isIntegral(new java.lang.Short(2.asInstanceOf[Short])) must beTrue - intComp.isIntegral(new java.lang.Long(2)) must beTrue - intComp.isIntegral(new java.lang.Long(2)) must beTrue + intComp.isIntegral(new java.lang.Long(2)) shouldBe true + intComp.isIntegral(new java.lang.Integer(2)) shouldBe true + intComp.isIntegral(new java.lang.Short(2.asInstanceOf[Short])) shouldBe true + intComp.isIntegral(new java.lang.Long(2)) shouldBe true + intComp.isIntegral(new java.lang.Long(2)) shouldBe true //These are not integrals - intComp.isIntegral(box(0.0)) must beFalse - intComp.isIntegral(box("hey")) must beFalse - intComp.isIntegral(box(Nil)) must beFalse - intComp.isIntegral(box(None)) must beFalse + intComp.isIntegral(box(0.0)) shouldBe false + intComp.isIntegral(box("hey")) shouldBe false + intComp.isIntegral(box(Nil)) shouldBe false + intComp.isIntegral(box(None)) shouldBe false } "handle null inputs" in { - intComp.hashCode(null) must be_==(0) + intComp.hashCode(null) shouldBe 0 List(box(1), box("hey"), box(2L), box(0.0)).foreach { x => - intComp.compare(null, x) must be_<(0) - intComp.compare(x, null) must be_>(0) - intComp.compare(x, x) must be_==(0) + intComp.compare(null, x) should be < (0) + intComp.compare(x, null) should be > (0) + intComp.compare(x, x) shouldBe 0 } - intComp.compare(null, null) must be_==(0) + intComp.compare(null, null) shouldBe 0 } "have consistent hashcode" in { List((box(1), box(1L)), (box(2), box(2L)), (box(3), box(3L))) .foreach { pair => - intComp.compare(pair._1, pair._2) must be_==(0) - intComp.hashCode(pair._1) must be_==(intComp.hashCode(pair._2)) + intComp.compare(pair._1, pair._2) shouldBe 0 + intComp.hashCode(pair._1) shouldBe (intComp.hashCode(pair._2)) } List((box(1), box(2L)), (box(2), box(3L)), (box(3), box(4L))) .foreach { pair => - intComp.compare(pair._1, pair._2) must be_<(0) - intComp.compare(pair._2, pair._1) must be_>(0) + intComp.compare(pair._1, pair._2) should be < (0) + intComp.compare(pair._2, pair._1) should be > (0) } } "Compare strings properly" in { - intComp.compare("hey", "you") must be_==("hey".compareTo("you")) - intComp.compare("hey", "hey") must be_==("hey".compareTo("hey")) - intComp.compare("you", "hey") must be_==("you".compareTo("hey")) + intComp.compare("hey", "you") shouldBe ("hey".compareTo("you")) + intComp.compare("hey", "hey") shouldBe ("hey".compareTo("hey")) + intComp.compare("you", "hey") shouldBe ("you".compareTo("hey")) } } } diff --git a/scalding-core/src/test/scala/com/twitter/scalding/JobTestTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/JobTestTest.scala index 212925f184..1179bb956b 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/JobTestTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/JobTestTest.scala @@ -1,6 +1,6 @@ package com.twitter.scalding -import org.specs.Specification +import org.scalatest.{ Matchers, WordSpec } /** * Simple identity job that reads from a Tsv and writes to a Tsv with no change. @@ -11,7 +11,7 @@ class SimpleTestJob(args: Args) extends Job(args) { Tsv(args("input")).read.write(Tsv(args("output"))) } -class JobTestTest extends Specification { +class JobTestTest extends WordSpec with Matchers { "A JobTest" should { "error helpfully when a source in the job doesn't have a corresponding .source call" in { val testInput: List[(String, Int)] = List(("a", 1), ("b", 2)) @@ -28,14 +28,12 @@ class JobTestTest extends Specification { .arg("input", "input") .arg("output", "output") .source(incorrectSource, testInput) - .sink[(String, Int)](Tsv("output")){ outBuf => { assert(outBuf == testInput) } } + .sink[(String, Int)](Tsv("output")){ outBuf => { outBuf shouldBe testInput } } .run - runJobTest() must throwA[IllegalArgumentException].like { - case iae: IllegalArgumentException => - iae.getMessage mustVerify ( - _.contains(TestTapFactory.sourceNotFoundError.format(requiredSource))) - } + the[IllegalArgumentException] thrownBy { + runJobTest() + } should have message (s"Failed to create tap for: ${requiredSource}, with error: requirement failed: " + TestTapFactory.sourceNotFoundError.format(requiredSource)) } } } 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 485bc4c9f9..aabae9197a 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/KryoTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/KryoTest.scala @@ -17,7 +17,7 @@ package com.twitter.scalding import com.twitter.scalding.serialization._ -import org.specs._ +import org.scalatest.{ Matchers, WordSpec } import java.io.{ ByteArrayOutputStream => BOS } import java.io.{ ByteArrayInputStream => BIS } @@ -49,12 +49,10 @@ case class TestCaseClassForSerialization(x: String, y: Int) case class TestValMap(val map: Map[String, Double]) case class TestValHashMap(val map: HashMap[String, Double]) -class KryoTest extends Specification { +class KryoTest extends WordSpec with Matchers { implicit def dateParser: DateParser = DateParser.default - noDetailedDiffs() //Fixes issue for scala 2.9 - def getSerialization = { val conf = new Configuration val chillConf = new HadoopConfig(conf) @@ -124,14 +122,14 @@ class KryoTest extends Specification { Monoid.sum(List(1, 2, 3, 4).map { hllmon(_) }), 'hai) .asInstanceOf[List[AnyRef]] - serializationRT(test) must be_==(test) + serializationRT(test) shouldBe test // HyperLogLogMonoid doesn't have a good equals. :( - singleRT(new HyperLogLogMonoid(5)).bits must be_==(5) + singleRT(new HyperLogLogMonoid(5)).bits shouldBe 5 } "handle arrays" in { def arrayRT[T](arr: Array[T]) { serializationRT(List(arr))(0) - .asInstanceOf[Array[T]].toList must be_==(arr.toList) + .asInstanceOf[Array[T]].toList shouldBe (arr.toList) } arrayRT(Array(0)) arrayRT(Array(0.1)) @@ -142,9 +140,9 @@ class KryoTest extends Specification { "handle scala singletons" in { val test = List(Nil, None) //Serialize each: - serializationRT(test) must be_==(test) + serializationRT(test) shouldBe test //Together in a list: - singleRT(test) must be_==(test) + singleRT(test) shouldBe test } "handle Date, RichDate and DateRange" in { import DateOps._ @@ -152,17 +150,15 @@ class KryoTest extends Specification { val myDate: RichDate = "1999-12-30T14" val simpleDate: java.util.Date = myDate.value val myDateRange = DateRange("2012-01-02", "2012-06-09") - singleRT(myDate) must be_==(myDate) - singleRT(simpleDate) must be_==(simpleDate) - singleRT(myDateRange) must be_==(myDateRange) + singleRT(myDate) shouldBe myDate + singleRT(simpleDate) shouldBe simpleDate + singleRT(myDateRange) shouldBe myDateRange } "Serialize a giant list" in { val bigList = (1 to 100000).toList val list2 = deserObj[List[Int]](bigList.getClass, serObj(bigList)) //Specs, it turns out, also doesn't deal with giant lists well: - list2.zip(bigList).foreach { tup => - tup._1 must be_==(tup._2) - } + list2.zip(bigList).foreach { case (l, r) => l shouldBe r } } } } 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 2a0cfb3d1d..bb616a9fc0 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/LookupJoinTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/LookupJoinTest.scala @@ -16,7 +16,7 @@ limitations under the License. package com.twitter.scalding import com.twitter.scalding.typed.LookupJoin -import org.specs._ +import org.scalatest.{ Matchers, WordSpec } class LookupJoinerJob(args: Args) extends Job(args) { import TDsl._ @@ -33,8 +33,7 @@ class LookupJoinerJob(args: Args) extends Job(args) { .write(TypedTsv[(String, String, String, String)]("output")) } -class LookupJoinedTest extends Specification { - noDetailedDiffs() +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: @@ -62,8 +61,8 @@ class LookupJoinedTest extends Specification { .source(TypedTsv[(Int, Int, Int)]("input1"), in1) .sink[(String, String, String, String)]( TypedTsv[(String, String, String, String)]("output")) { outBuf => - outBuf.toSet must be_==(lookupJoin(in0, in1).toSet) - in0.size must be_==(outBuf.size) + outBuf.toSet shouldBe (lookupJoin(in0, in1).toSet) + in0 should have size (outBuf.size) } .run .runHadoop diff --git a/scalding-core/src/test/scala/com/twitter/scalding/PackTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/PackTest.scala index c1c4a23106..8c86980fe8 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/PackTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/PackTest.scala @@ -17,8 +17,8 @@ package com.twitter.scalding import cascading.tuple.TupleEntry -import org.specs._ -import scala.reflect.BeanProperty +import org.scalatest.{ Matchers, WordSpec } +import scala.beans.BeanProperty import scala.collection.mutable.Buffer @@ -139,21 +139,19 @@ class FatContainerToPopulationJob(args: Args) extends Job(args) { .write(Tsv("output")) } -class PackTest extends Specification { - noDetailedDiffs() - +class PackTest extends WordSpec with Matchers { val inputData = List( (1, 2), (2, 2), (3, 2)) "A ContainerPopulationJob" should { - JobTest("com.twitter.scalding.ContainerPopulationJob") + JobTest(new ContainerPopulationJob(_)) .source(Tsv("input"), inputData) .sink[(Int, Int)](Tsv("output")) { buf => "correctly populate container objects" in { - buf.size must_== 3 - buf.toSet must_== inputData.toSet + buf should have size 3 + buf.toSet shouldBe inputData.toSet } } .run @@ -165,14 +163,14 @@ class PackTest extends Specification { .source(Tsv("input"), inputData) .sink[(Int, Int)](Tsv("output")) { buf => "correctly populate container objects" in { - buf.size must_== 3 - buf.toSet must_== inputData.toSet + buf should have size 3 + buf.toSet shouldBe inputData.toSet } } .sink[(Int, Int)](Tsv("output-cc")) { buf => "correctly populate container case class objects" in { - buf.size must_== 3 - buf.toSet must_== inputData.toSet + buf should have size 3 + buf.toSet shouldBe inputData.toSet } } .run @@ -183,13 +181,13 @@ class PackTest extends Specification { val fatCorrect = List(8, 13, 21, 34, 55, 89, 144, 233, 377, 610, 987, 1597, 2584, 4181, 6765, 10946, 17711, 28657, 46368, 75025, 121393, 196418, 317811) "A FatContainerPopulationJob" should { - JobTest("com.twitter.scalding.FatContainerPopulationJob") + JobTest(new FatContainerPopulationJob(_)) .source(Tsv("input"), fatInputData) .sink[TupleEntry](Tsv("output")) { buf: Buffer[TupleEntry] => "correctly populate a fat container object" in { val te = buf.head for (idx <- fatCorrect.indices) { - te.getInteger(idx) must_== fatCorrect(idx) + te.getInteger(idx) shouldBe fatCorrect(idx) } } } @@ -198,13 +196,13 @@ class PackTest extends Specification { } "A FatContainerToPopulationJob" should { - JobTest("com.twitter.scalding.FatContainerPopulationJob") + JobTest(new FatContainerPopulationJob(_)) .source(Tsv("input"), fatInputData) .sink[TupleEntry](Tsv("output")) { buf: Buffer[TupleEntry] => "correctly populate a fat container object" in { val te = buf.head for (idx <- fatCorrect.indices) { - te.getInteger(idx) must_== fatCorrect(idx) + te.getInteger(idx) shouldBe fatCorrect(idx) } } } diff --git a/scalding-core/src/test/scala/com/twitter/scalding/PageRankTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/PageRankTest.scala index 0275b8dcbc..abb61e8de6 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/PageRankTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/PageRankTest.scala @@ -15,40 +15,40 @@ limitations under the License. */ package com.twitter.scalding -import org.specs._ +import org.scalatest.{ Matchers, WordSpec } -class PageRankTest extends Specification { +class PageRankTest extends WordSpec with Matchers { "A PageRank job" should { - JobTest(new com.twitter.scalding.examples.PageRank(_)). - arg("input", "inputFile"). - arg("output", "outputFile"). - arg("errorOut", "error"). - arg("temp", "tempBuffer"). + JobTest(new com.twitter.scalding.examples.PageRank(_)) + .arg("input", "inputFile") + .arg("output", "outputFile") + .arg("errorOut", "error") + .arg("temp", "tempBuffer") //How many iterations to do each time: - arg("iterations", "6"). - arg("convergence", "0.05"). - source(Tsv("inputFile"), List((1L, "2", 1.0), (2L, "1,3", 1.0), (3L, "2", 1.0))). + .arg("iterations", "6") + .arg("convergence", "0.05") + .source(Tsv("inputFile"), List((1L, "2", 1.0), (2L, "1,3", 1.0), (3L, "2", 1.0))) //Don't check the tempBuffer: - sink[(Long, String, Double)](Tsv("tempBuffer")) { ob => () }. - sink[Double](TypedTsv[Double]("error")) { ob => + .sink[(Long, String, Double)](Tsv("tempBuffer")) { ob => () } + .sink[Double](TypedTsv[Double]("error")) { ob => "have low error" in { - ob.head must be_<=(0.05) + ob.head should be <= 0.05 } - }. - sink[(Long, String, Double)](Tsv("outputFile")){ outputBuffer => + } + .sink[(Long, String, Double)](Tsv("outputFile")){ outputBuffer => val pageRank = outputBuffer.map { res => (res._1, res._3) }.toMap "correctly compute pagerank" in { val d = 0.85 val twoPR = (1.0 + 2 * d) / (1.0 + d) val otherPR = (1.0 + d / 2.0) / (1.0 + d) println(pageRank) - (pageRank(1L) + pageRank(2L) + pageRank(3L)) must beCloseTo(3.0, 0.1) - pageRank(1L) must beCloseTo(otherPR, 0.1) - pageRank(2L) must beCloseTo(twoPR, 0.1) - pageRank(3L) must beCloseTo(otherPR, 0.1) + (pageRank(1L) + pageRank(2L) + pageRank(3L)) shouldBe 3.0 +- 0.1 + pageRank(1L) shouldBe otherPR +- 0.1 + pageRank(2L) shouldBe twoPR +- 0.1 + pageRank(3L) shouldBe otherPR +- 0.1 } - }. - run. - finish + } + .run + .finish } } 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 a7a5a51105..204f0f96fa 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/PartitionSourceTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/PartitionSourceTest.scala @@ -19,7 +19,7 @@ package com.twitter.scalding import java.io.File import scala.io.{ Source => ScalaSource } -import org.specs._ +import org.scalatest.{ Matchers, WordSpec } import cascading.tap.SinkMode import cascading.tuple.Fields @@ -79,8 +79,7 @@ class PartialPartitionTestJob(args: Args) extends Job(args) { } } -class DelimitedPartitionSourceTest extends Specification { - noDetailedDiffs() +class DelimitedPartitionSourceTest extends WordSpec with Matchers { import Dsl._ import PartitionSourceTestHelpers._ "PartitionedTsv fed a DelimitedPartition" should { @@ -103,19 +102,18 @@ class DelimitedPartitionSourceTest extends Specification { val directory = new File(testMode.getWritePathFor(DelimitedPartitionedTsv)) - directory.listFiles().map({ _.getName() }).toSet mustEqual Set("A", "B") + directory.listFiles().map({ _.getName() }).toSet shouldBe Set("A", "B") val aSource = ScalaSource.fromFile(new File(directory, "A/part-00000-00000")) val bSource = ScalaSource.fromFile(new File(directory, "B/part-00000-00001")) - aSource.getLines.toList mustEqual Seq("A\t1", "A\t2") - bSource.getLines.toList mustEqual Seq("B\t3") + aSource.getLines.toSeq shouldBe Seq("A\t1", "A\t2") + bSource.getLines.toSeq shouldBe Seq("B\t3") } } } -class CustomPartitionSourceTest extends Specification { - noDetailedDiffs() +class CustomPartitionSourceTest extends WordSpec with Matchers { import Dsl._ import PartitionSourceTestHelpers._ "PartitionedTsv fed a CustomPartition" should { @@ -138,19 +136,18 @@ class CustomPartitionSourceTest extends Specification { val directory = new File(testMode.getWritePathFor(CustomPartitionedTsv)) - directory.listFiles().map({ _.getName() }).toSet mustEqual Set("{A}->{x}", "{B}->{y}") + directory.listFiles().map({ _.getName() }).toSet shouldBe Set("{A}->{x}", "{B}->{y}") val aSource = ScalaSource.fromFile(new File(directory, "{A}->{x}/part-00000-00000")) val bSource = ScalaSource.fromFile(new File(directory, "{B}->{y}/part-00000-00001")) - aSource.getLines.toList mustEqual Seq("A\tx\t1", "A\tx\t2") - bSource.getLines.toList mustEqual Seq("B\ty\t3") + aSource.getLines.toSeq shouldBe Seq("A\tx\t1", "A\tx\t2") + bSource.getLines.toSeq shouldBe Seq("B\ty\t3") } } } -class PartialPartitionSourceTest extends Specification { - noDetailedDiffs() +class PartialPartitionSourceTest extends WordSpec with Matchers { import Dsl._ import PartitionSourceTestHelpers._ "PartitionedTsv fed a DelimitedPartition and only a subset of fields" should { @@ -174,13 +171,13 @@ class PartialPartitionSourceTest extends Specification { val directory = new File(testMode.getWritePathFor(PartialPartitionedTsv)) - directory.listFiles().map({ _.getName() }).toSet mustEqual Set("A", "B") + directory.listFiles().map({ _.getName() }).toSet shouldBe Set("A", "B") val aSource = ScalaSource.fromFile(new File(directory, "A/x/part-00000-00000")) val bSource = ScalaSource.fromFile(new File(directory, "B/y/part-00000-00001")) - aSource.getLines.toList mustEqual Seq("A\t1", "A\t2") - bSource.getLines.toList mustEqual Seq("B\t3") + aSource.getLines.toSeq shouldBe Seq("A\t1", "A\t2") + bSource.getLines.toSeq shouldBe Seq("B\t3") } } } diff --git a/scalding-core/src/test/scala/com/twitter/scalding/PathFilterTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/PathFilterTest.scala index 68e3105261..698206cea6 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/PathFilterTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/PathFilterTest.scala @@ -1,37 +1,37 @@ package com.twitter.scalding -import org.specs.Specification +import org.scalatest.{ Matchers, WordSpec } import org.apache.hadoop.fs.{ Path => HadoopPath, PathFilter } -class PathFilterTest extends Specification { +class PathFilterTest extends WordSpec with Matchers { "RichPathFilter" should { import RichPathFilter.toRichPathFilter val p = new HadoopPath("/nowhere") "compose ands" in { - AlwaysTrue.and(AlwaysTrue).accept(p) must be_==(true) - AlwaysTrue.and(AlwaysFalse).accept(p) must be_==(false) - AlwaysFalse.and(AlwaysTrue).accept(p) must be_==(false) - AlwaysFalse.and(AlwaysFalse).accept(p) must be_==(false) + AlwaysTrue.and(AlwaysTrue).accept(p) shouldBe true + AlwaysTrue.and(AlwaysFalse).accept(p) shouldBe false + AlwaysFalse.and(AlwaysTrue).accept(p) shouldBe false + AlwaysFalse.and(AlwaysFalse).accept(p) shouldBe false - AlwaysTrue.and(AlwaysTrue, AlwaysTrue).accept(p) must be_==(true) - AlwaysTrue.and(AlwaysTrue, AlwaysFalse).accept(p) must be_==(false) + AlwaysTrue.and(AlwaysTrue, AlwaysTrue).accept(p) shouldBe true + AlwaysTrue.and(AlwaysTrue, AlwaysFalse).accept(p) shouldBe false } "compose ors" in { - AlwaysTrue.or(AlwaysTrue).accept(p) must be_==(true) - AlwaysTrue.or(AlwaysFalse).accept(p) must be_==(true) - AlwaysFalse.or(AlwaysTrue).accept(p) must be_==(true) - AlwaysFalse.or(AlwaysFalse).accept(p) must be_==(false) + AlwaysTrue.or(AlwaysTrue).accept(p) shouldBe true + AlwaysTrue.or(AlwaysFalse).accept(p) shouldBe true + AlwaysFalse.or(AlwaysTrue).accept(p) shouldBe true + AlwaysFalse.or(AlwaysFalse).accept(p) shouldBe false - AlwaysFalse.or(AlwaysTrue, AlwaysTrue).accept(p) must be_==(true) - AlwaysTrue.or(AlwaysFalse, AlwaysFalse).accept(p) must be_==(true) + AlwaysFalse.or(AlwaysTrue, AlwaysTrue).accept(p) shouldBe true + AlwaysTrue.or(AlwaysFalse, AlwaysFalse).accept(p) shouldBe true } "negate nots" in { - AlwaysTrue.not.accept(p) must be_==(false) - AlwaysFalse.not.accept(p) must be_==(true) - AlwaysTrue.not.not.accept(p) must be_==(true) + AlwaysTrue.not.accept(p) shouldBe false + AlwaysFalse.not.accept(p) shouldBe true + AlwaysTrue.not.not.accept(p) shouldBe true } } @@ -43,4 +43,4 @@ object AlwaysTrue extends PathFilter { object AlwaysFalse extends PathFilter { override def accept(p: HadoopPath): Boolean = false -} \ No newline at end of file +} 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 a0b23474e2..2a1c484b01 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/ReduceOperationsTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/ReduceOperationsTest.scala @@ -15,7 +15,7 @@ limitations under the License. */ package com.twitter.scalding -import org.specs._ +import org.scalatest.{ Matchers, WordSpec } import com.twitter.scalding._ class SortWithTakeJob(args: Args) extends Job(args) { @@ -85,13 +85,12 @@ class ApproximateUniqueCountJob(args: Args) extends Job(args) { } } -class ReduceOperationsTest extends Specification { - noDetailedDiffs() +class ReduceOperationsTest extends WordSpec with Matchers { import Dsl._ val inputData = List(("a", 2L, 3.0), ("a", 3L, 3.0), ("a", 1L, 3.5), ("b", 1L, 6.0), ("b", 2L, 5.0), ("b", 3L, 4.0), ("b", 4L, 3.0), ("b", 5L, 2.0), ("b", 6L, 1.0)) "A sortWithTake job" should { - JobTest("com.twitter.scalding.SortWithTakeJob") + JobTest(new SortWithTakeJob(_)) .source(Tsv("input0", ('key, 'item_id, 'score)), inputData) .sink[(String, List[(Long, Double)])](Tsv("output0")) { buf => "grouped list" in { @@ -99,15 +98,15 @@ class ReduceOperationsTest extends Specification { "a" -> List((1L, 3.5), (3L, 3.0), (2L, 3.0)).toString, "b" -> List((1L, 6.0), (2L, 5.0), (3L, 4.0), (4L, 3.0), (5L, 2.0)).toString) val whatWeGet: Map[String, List[(Long, Double)]] = buf.toMap - whatWeGet.get("a").getOrElse("apples") must be_==(whatWeWant.get("a").getOrElse("oranges")) - whatWeGet.get("b").getOrElse("apples") must be_==(whatWeWant.get("b").getOrElse("oranges")) + whatWeGet.get("a").getOrElse("apples") shouldBe (whatWeWant.get("a").getOrElse("oranges")) + whatWeGet.get("b").getOrElse("apples") shouldBe (whatWeWant.get("b").getOrElse("oranges")) } } .runHadoop .finish } "A sortedTake job" should { - JobTest("com.twitter.scalding.SortedTakeJob") + JobTest(new SortedTakeJob(_)) .source(Tsv("input0", ('key, 'item_id, 'score)), inputData) .sink[(String, List[(Long, Double)])](Tsv("output0")) { buf => "grouped list" in { @@ -115,8 +114,8 @@ class ReduceOperationsTest extends Specification { "a" -> List((1L, 3.5), (2L, 3.0), (3L, 3.0)).toString, "b" -> List((1L, 6.0), (2L, 5.0), (3L, 4.0), (4L, 3.0), (5L, 2.0)).toString) val whatWeGet: Map[String, List[(Long, Double)]] = buf.toMap - whatWeGet.get("a").getOrElse("apples") must be_==(whatWeWant.get("a").getOrElse("oranges")) - whatWeGet.get("b").getOrElse("apples") must be_==(whatWeWant.get("b").getOrElse("oranges")) + whatWeGet.get("a").getOrElse("apples") shouldBe (whatWeWant.get("a").getOrElse("oranges")) + whatWeGet.get("b").getOrElse("apples") shouldBe (whatWeWant.get("b").getOrElse("oranges")) } } .runHadoop @@ -124,7 +123,7 @@ class ReduceOperationsTest extends Specification { } "A sortedReverseTake job" should { - JobTest("com.twitter.scalding.SortedReverseTakeJob") + JobTest(new SortedReverseTakeJob(_)) .source(Tsv("input0", ('key, 'item_id, 'score)), inputData) .sink[(String, List[(Long, Double)])](Tsv("output0")) { buf => "grouped list" in { @@ -132,8 +131,8 @@ class ReduceOperationsTest extends Specification { "a" -> List((3L, 3.0), (2L, 3.0), (1L, 3.5)).toString, "b" -> List((6L, 1.0), (5L, 2.0), (4L, 3.0), (3L, 4.0), (2L, 5.0)).toString) val whatWeGet: Map[String, List[(Long, Double)]] = buf.toMap - whatWeGet.get("a").getOrElse("apples") must be_==(whatWeWant.get("a").getOrElse("oranges")) - whatWeGet.get("b").getOrElse("apples") must be_==(whatWeWant.get("b").getOrElse("oranges")) + whatWeGet.get("a").getOrElse("apples") shouldBe (whatWeWant.get("a").getOrElse("oranges")) + whatWeGet.get("b").getOrElse("apples") shouldBe (whatWeWant.get("b").getOrElse("oranges")) } } .runHadoop @@ -146,7 +145,7 @@ class ReduceOperationsTest extends Specification { ("mobile", "iphone5", "ios"), ("mobile", "droid x", "android")) - JobTest("com.twitter.scalding.ApproximateUniqueCountJob") + JobTest(new ApproximateUniqueCountJob(_)) .source(Tsv("input0", ('category, 'model, 'os)), inputData) .sink[(String, Double)](Tsv("output0")) { buf => "grouped OS count" in { @@ -154,9 +153,9 @@ class ReduceOperationsTest extends Specification { "laptop" -> 1.0, "mobile" -> 2.0) val whatWeGet: Map[String, Double] = buf.toMap - whatWeGet.size must be_==(2) - whatWeGet.get("laptop").getOrElse("apples") must be_==(whatWeWant.get("laptop").getOrElse("oranges")) - whatWeGet.get("mobile").getOrElse("apples") must be_==(whatWeWant.get("mobile").getOrElse("oranges")) + 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")) } } .runHadoop 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 7a54118931..21970aa1bf 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/ScanLeftTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/ScanLeftTest.scala @@ -1,6 +1,6 @@ package com.twitter.scalding -import org.specs._ +import org.scalatest.{ Matchers, WordSpec } import com.twitter.scalding._ /** @@ -8,7 +8,6 @@ import com.twitter.scalding._ * Then add another column for each group which is the rank order of the height. */ class AddRankingWithScanLeft(args: Args) extends Job(args) { - Tsv("input1", ('gender, 'height)) .read .groupBy('gender) { group => @@ -20,49 +19,13 @@ class AddRankingWithScanLeft(args: Args) extends Job(args) { } } } - // scanLeft generates an extra line per group, thus remove it + // scanLeft generates an extra line per group, thus remove it .filter('height) { x: String => x != null } .debug .write(Tsv("result1")) - } -/** - * Advanced example: Count seconds each user spent reading a blog article (using scanLeft) - * For the sake of simplicity we assume that you have converted date-time into epoch - */ -//class ScanLeftTimeExample(args: Args) extends Job(args) { -// -// Tsv("input2", ('epoch, 'user, 'event)) -// // Create a helper symbol first -// .insert('temp, 0L) -// // Group by user and sort by epoch in reverse, so that most recent event comes first -// .groupBy('user) { group => -// group.sortBy('epoch).reverse -// .scanLeft(('epoch, 'temp) -> ('originalEpoch, 'duration))((0L, 0L)) { -// (firstLine: (Long, Long), secondLine: (Long, Long)) => -// var delta = firstLine._1 - secondLine._1 -// // scanLeft is initialised with (0L,0L) so first subtraction -// // will result into a negative number! -// if (delta < 0L) delta = -delta -// (secondLine._1, delta) -// } -// } -// .project('epoch, 'user, 'event, 'duration) -// // Remove lines introduced by scanLeft and discard helping symbols -// .filter('epoch) { x: Any => x != null } -// // Order in ascending time -// .groupBy('user) { group => -// group.sortBy('epoch) -// } -// // You can now remove most recent events where we are uncertain of time spent -// .filter('duration) { x: Long => x < 10000L } -// .debug -// .write(Tsv("result2")) -// -//} - -class ScanLeftTest extends Specification { +class ScanLeftTest extends WordSpec with Matchers { import Dsl._ // --- A simple ranking job @@ -82,51 +45,17 @@ class ScanLeftTest extends Specification { ("female", 128.6, 2)) "A simple ranking scanleft job" should { - JobTest("com.twitter.scalding.AddRankingWithScanLeft") + JobTest(new AddRankingWithScanLeft(_)) .source(Tsv("input1", ('gender, 'height)), sampleInput1) .sink[(String, Double, Long)](Tsv("result1")) { outBuf1 => "produce correct number of records when filtering out null values" in { - outBuf1.size must_== 5 + outBuf1 should have size 5 } "create correct ranking per group, 1st being the heighest person of that group" in { - outBuf1.toSet must_== expectedOutput1 + outBuf1.toSet shouldBe expectedOutput1 } } .run .finish } - - // // --- A trickier duration counting job - // var sampleInput2 = List( - // (1370737000L, "userA", "/read/blog/123"), - // (1370737002L, "userB", "/read/blog/781"), - // (1370737028L, "userA", "/read/blog/621"), - // (1370737067L, "userB", "/add/comment/"), - // (1370737097L, "userA", "/read/blog/888"), - // (1370737103L, "userB", "/read/blog/999")) - // - // // Each group sorted and ranking added highest person to shortest - // val expectedOutput2 = Set( - // (1370737000L, "userA", "/read/blog/123", 28), // userA was reading blog/123 for 28 seconds - // (1370737028L, "userA", "/read/blog/621", 69), // userA was reading blog/621 for 69 seconds - // (1370737002L, "userB", "/read/blog/781", 65), // userB was reading blog/781 for 65 seconds - // (1370737067L, "userB", "/add/comment/", 36)) // userB was posting a comment for 36 seconds - // // Note that the blog/999 is not recorded as we can't tell how long userB spend on it based on the input - // - // "A more advanced time extraction scanleft job" should { - // JobTest("com.twitter.scalding.ScanLeftTimeExample") - // .source(Tsv("input2", ('epoch, 'user, 'event)), sampleInput2) - // .sink[(Long, String, String, Long)](Tsv("result2")) { outBuf2 => - // "produce correct number of records when filtering out null values" in { - // outBuf2.size must_== 4 - // } - // "create correct output per user" in { - // outBuf2.toSet must_== expectedOutput2 - // } - // } - // .run - // .finish - // } - } - 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 ac4e9a48a2..83ce7bfa30 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/SideEffectTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/SideEffectTest.scala @@ -17,7 +17,7 @@ package com.twitter.scalding import scala.annotation.tailrec import cascading.pipe._ -import org.specs._ +import org.scalatest.{ Matchers, WordSpec } /* * Zip uses side effect construct to create zipped list. @@ -48,19 +48,15 @@ class Zip(args: Args) extends Job(args) { zipped.write(Tsv("zipped")) } -class SideEffectTest extends Specification with FieldConversions { +class SideEffectTest extends WordSpec with Matchers with FieldConversions { "Zipper should do create zipped sequence. Coded with side effect" should { - JobTest("com.twitter.scalding.Zip") + JobTest(new Zip(_)) .source(Tsv("line", ('line)), List(Tuple1("line1"), Tuple1("line2"), Tuple1("line3"), Tuple1("line4"))) .sink[(String, String)](Tsv("zipped")) { ob => "correctly compute zipped sequence" in { val res = ob.toList val expected = List(("line1", "line2"), ("line2", "line3"), ("line3", "line4")) - res.zip(expected) foreach { - case ((a, b), (c, d)) => - a must be_== (c) - b must be_== (d) - } + res shouldBe expected } } .run @@ -104,7 +100,7 @@ class ZipBuffer(args: Args) extends Job(args) { zipped.write(Tsv("zipped")) } -class SideEffectBufferTest extends Specification with FieldConversions { +class SideEffectBufferTest extends WordSpec with Matchers with FieldConversions { "ZipBuffer should do create two zipped sequences, one for even lines and one for odd lines. Coded with side effect" should { JobTest("com.twitter.scalding.ZipBuffer") .source(Tsv("line", ('line)), List(Tuple1("line1"), Tuple1("line2"), Tuple1("line3"), Tuple1("line4"), Tuple1("line5"), Tuple1("line6"))) @@ -112,11 +108,7 @@ class SideEffectBufferTest extends Specification with FieldConversions { "correctly compute zipped sequence" in { val res = ob.toList.sorted val expected = List(("line1", "line3"), ("line3", "line5"), ("line2", "line4"), ("line4", "line6")).sorted - res.zip(expected) foreach { - case ((a, b), (c, d)) => - a must be_== (c) - b must be_== (d) - } + res shouldBe expected } } .run 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 8e774e7e8a..eb6e0b6560 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/SkewJoinTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/SkewJoinTest.scala @@ -15,7 +15,7 @@ limitations under the License. */ package com.twitter.scalding -import org.specs._ +import org.scalatest.{ Matchers, WordSpec } import cascading.pipe.joiner._ @@ -78,56 +78,53 @@ object JoinTestHelper { } } -class SkewJoinPipeTest extends Specification { - noDetailedDiffs() - +class SkewJoinPipeTest extends WordSpec with Matchers { import JoinTestHelper._ "A SkewInnerProductJob" should { - "compute skew join with sampleRate = 0.001, using strategy A" in { val (sk, inner) = runJobWithArguments(new SkewJoinJob(_), sampleRate = 0.001, replicator = "a") - sk must_== inner + sk shouldBe inner } "compute skew join with sampleRate = 0.001, using strategy B" in { val (sk, inner) = runJobWithArguments(new SkewJoinJob(_), sampleRate = 0.001, replicator = "b") - sk must_== inner + sk shouldBe inner } "compute skew join with sampleRate = 0.1, using strategy A" in { val (sk, inner) = runJobWithArguments(new SkewJoinJob(_), sampleRate = 0.1, replicator = "a") - sk must_== inner + sk shouldBe inner } "compute skew join with sampleRate = 0.1, using strategy B" in { val (sk, inner) = runJobWithArguments(new SkewJoinJob(_), sampleRate = 0.1, replicator = "b") - sk must_== inner + sk shouldBe inner } "compute skew join with sampleRate = 0.9, using strategy A" in { val (sk, inner) = runJobWithArguments(new SkewJoinJob(_), sampleRate = 0.9, replicator = "a") - sk must_== inner + sk shouldBe inner } "compute skew join with sampleRate = 0.9, using strategy B" in { val (sk, inner) = runJobWithArguments(new SkewJoinJob(_), sampleRate = 0.9, replicator = "b") - sk must_== inner + sk shouldBe inner } "compute skew join with replication factor 5, using strategy A" in { val (sk, inner) = runJobWithArguments(new SkewJoinJob(_), replicationFactor = 5, replicator = "a") - sk must_== inner + sk shouldBe inner } "compute skew join with reducers = 10, using strategy A" in { val (sk, inner) = runJobWithArguments(new SkewJoinJob(_), reducers = 10, replicator = "a") - sk must_== inner + sk shouldBe inner } "compute skew join with reducers = 10, using strategy B" in { val (sk, inner) = runJobWithArguments(new SkewJoinJob(_), reducers = 10, replicator = "b") - sk must_== inner + sk shouldBe inner } } } @@ -157,20 +154,18 @@ class CollidingKeySkewJoinJob(args: Args) extends Job(args) { .write(Tsv("jws-output")) } -class CollidingKeySkewJoinTest extends Specification { - noDetailedDiffs() +class CollidingKeySkewJoinTest extends WordSpec with Matchers { import JoinTestHelper._ "A CollidingSkewInnerProductJob" should { - "compute skew join with colliding fields, using strategy A" in { val (sk, inn) = runJobWithArguments(new CollidingKeySkewJoinJob(_), replicator = "a") - sk must_== inn + sk shouldBe inn } "compute skew join with colliding fields, using strategy B" in { val (sk, inn) = runJobWithArguments(new CollidingKeySkewJoinJob(_), replicator = "b") - sk must_== inn + sk shouldBe inn } } } diff --git a/scalding-core/src/test/scala/com/twitter/scalding/SourceSpec.scala b/scalding-core/src/test/scala/com/twitter/scalding/SourceSpec.scala index 8a3df0e050..851765a750 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/SourceSpec.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/SourceSpec.scala @@ -15,13 +15,13 @@ limitations under the License. */ package com.twitter.scalding -import org.specs._ +import org.scalatest.{ Matchers, WordSpec } import cascading.pipe.Pipe import cascading.tuple.Fields import com.twitter.scalding.source._ -class SourceSpec extends Specification { +class SourceSpec extends WordSpec with Matchers { import Dsl._ "A case class Source" should { @@ -41,10 +41,10 @@ class SourceSpec extends Specification { val d = new DailySuffixTsvSecond("/testNew")(dr1) val e = DailySuffixTsv("/test")(dr1) - (a == b) must beFalse - (b == c) must beFalse - (a == d) must beFalse - (a == e) must beTrue + a should not be b + b should not be c + a should not be d + a shouldBe e } } @@ -58,7 +58,7 @@ class SourceSpec extends Specification { JobTest(new AddRemoveOneJob(_)) .source(AddOneTsv("input"), List((0, "0"), (1, "1"))) .sink[(String, String)](RemoveOneTsv("output")) { buf => - buf.toSet must_== Set(("0", "0"), ("1", "1")) + buf.toSet shouldBe Set(("0", "0"), ("1", "1")) } .run .finish 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 84b09f8555..74809e6b4a 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/TemplateSourceTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/TemplateSourceTest.scala @@ -19,7 +19,7 @@ package com.twitter.scalding import java.io.File import scala.io.{ Source => ScalaSource } -import org.specs._ +import org.scalatest.{ Matchers, WordSpec } import cascading.tap.SinkMode import cascading.tuple.Fields @@ -32,8 +32,7 @@ class TemplateTestJob(args: Args) extends Job(args) { } } -class TemplateSourceTest extends Specification { - noDetailedDiffs() +class TemplateSourceTest extends WordSpec with Matchers { import Dsl._ "TemplatedTsv" should { "split output by template" in { @@ -55,13 +54,13 @@ class TemplateSourceTest extends Specification { val directory = new File(testMode.getWritePathFor(TemplatedTsv("base", "%s", 'col1))) - directory.listFiles().map({ _.getName() }).toSet mustEqual Set("A", "B") + directory.listFiles().map({ _.getName() }).toSet shouldBe Set("A", "B") val aSource = ScalaSource.fromFile(new File(directory, "A/part-00000")) val bSource = ScalaSource.fromFile(new File(directory, "B/part-00000")) - aSource.getLines.toList mustEqual Seq("A\t1", "A\t2") - bSource.getLines.toList mustEqual Seq("B\t3") + aSource.getLines.toList shouldBe Seq("A\t1", "A\t2") + bSource.getLines.toList shouldBe Seq("B\t3") } } } 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 54c375f30d..152d822a2c 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/TestTapFactoryTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/TestTapFactoryTest.scala @@ -4,39 +4,26 @@ import cascading.tap.Tap import cascading.tuple.{ Fields, Tuple } import java.lang.IllegalArgumentException import scala.collection.mutable.Buffer -import org.specs.Specification +import org.scalatest.{ Matchers, WordSpec } -class TestTapFactoryTest extends Specification { +class TestTapFactoryTest extends WordSpec with Matchers { "A test tap created by TestTapFactory" should { - "error helpfully when a source is not in the map for test buffers" >> { + "error helpfully when a source is not in the map for test buffers" in { // Source to use for this test. - val testSource = new Tsv("path") + val testSource = Tsv("path") // Map of sources to use when creating the tap-- does not contain testSource val emptySourceMap = Map[Source, Buffer[Tuple]]() - def buffers(s: Source): Option[Buffer[Tuple]] = { - if (emptySourceMap.contains(s)) { - Some(emptySourceMap(s)) - } else { - None - } - } - val testFields = new Fields() - - val testMode = Test(buffers) - val testTapFactory = TestTapFactory(testSource, testFields) + val testMode = Test { emptySourceMap.get(_) } + val testTapFactory = TestTapFactory(testSource, new Fields()) def createIllegalTap(): Tap[Any, Any, Any] = testTapFactory.createTap(Read)(testMode).asInstanceOf[Tap[Any, Any, Any]] - createIllegalTap() must throwA[IllegalArgumentException].like { - case iae: IllegalArgumentException => - iae.getMessage mustVerify ( - _.contains(TestTapFactory.sourceNotFoundError.format(testSource))) - } + the[IllegalArgumentException] thrownBy { + createIllegalTap() + } should have message ("requirement failed: " + TestTapFactory.sourceNotFoundError.format(testSource)) } - } - } diff --git a/scalding-core/src/test/scala/com/twitter/scalding/TupleTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/TupleTest.scala index e35d3615fb..7d3b2f2ac1 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/TupleTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/TupleTest.scala @@ -17,11 +17,9 @@ package com.twitter.scalding import cascading.tuple.{ TupleEntry, Tuple => CTuple } -import org.specs._ - -class TupleTest extends Specification { - noDetailedDiffs() //Fixes issue for scala 2.9 +import org.scalatest.{ Matchers, WordSpec } +class TupleTest extends WordSpec with Matchers { def get[T](ctup: CTuple)(implicit tc: TupleConverter[T]) = tc(new TupleEntry(ctup)) def set[T](t: T)(implicit ts: TupleSetter[T]): CTuple = ts(t) @@ -41,51 +39,51 @@ class TupleTest extends Specification { "TupleGetter should work as a type-class" in { val emptyTup = new CTuple val ctup = new CTuple("hey", new java.lang.Long(2), new java.lang.Integer(3), emptyTup) - TupleGetter.get[String](ctup, 0) must be_==("hey") - TupleGetter.get[Long](ctup, 1) must be_==(2L) - TupleGetter.get[Int](ctup, 2) must be_==(3) - TupleGetter.get[CTuple](ctup, 3) must be_==(emptyTup) + TupleGetter.get[String](ctup, 0) shouldBe "hey" + TupleGetter.get[Long](ctup, 1) shouldBe 2L + TupleGetter.get[Int](ctup, 2) shouldBe 3 + TupleGetter.get[CTuple](ctup, 3) shouldBe emptyTup } "get primitives out of cascading tuples" in { val ctup = new CTuple("hey", new java.lang.Long(2), new java.lang.Integer(3)) - get[(String, Long, Int)](ctup) must be_==(("hey", 2L, 3)) - - roundTrip[Int](3) must beTrue - arityConvMatches(3, 1) must beTrue - aritySetMatches(3, 1) must beTrue - roundTrip[Long](42L) must beTrue - arityConvMatches(42L, 1) must beTrue - aritySetMatches(42L, 1) must beTrue - roundTrip[String]("hey") must beTrue - arityConvMatches("hey", 1) must beTrue - aritySetMatches("hey", 1) must beTrue - roundTrip[(Int, Int)]((4, 2)) must beTrue - arityConvMatches((2, 3), 2) must beTrue - aritySetMatches((2, 3), 2) must beTrue + get[(String, Long, Int)](ctup) shouldBe ("hey", 2L, 3) + + roundTrip[Int](3) shouldBe true + arityConvMatches(3, 1) shouldBe true + aritySetMatches(3, 1) shouldBe true + roundTrip[Long](42L) shouldBe true + arityConvMatches(42L, 1) shouldBe true + aritySetMatches(42L, 1) shouldBe true + roundTrip[String]("hey") shouldBe true + arityConvMatches("hey", 1) shouldBe true + aritySetMatches("hey", 1) shouldBe true + roundTrip[(Int, Int)]((4, 2)) shouldBe true + arityConvMatches((2, 3), 2) shouldBe true + aritySetMatches((2, 3), 2) shouldBe true } "get non-primitives out of cascading tuples" in { val ctup = new CTuple(None, List(1, 2, 3), 1 -> 2) - get[(Option[Int], List[Int], (Int, Int))](ctup) must be_==((None, List(1, 2, 3), 1 -> 2)) + get[(Option[Int], List[Int], (Int, Int))](ctup) shouldBe (None, List(1, 2, 3), 1 -> 2) - roundTrip[(Option[Int], List[Int])]((Some(1), List())) must beTrue - arityConvMatches((None, Nil), 2) must beTrue - aritySetMatches((None, Nil), 2) must beTrue + roundTrip[(Option[Int], List[Int])]((Some(1), List())) shouldBe true + arityConvMatches((None, Nil), 2) shouldBe true + aritySetMatches((None, Nil), 2) shouldBe true - arityConvMatches(None, 1) must beTrue - aritySetMatches(None, 1) must beTrue - arityConvMatches(List(1, 2, 3), 1) must beTrue - aritySetMatches(List(1, 2, 3), 1) must beTrue + arityConvMatches(None, 1) shouldBe true + aritySetMatches(None, 1) shouldBe true + arityConvMatches(List(1, 2, 3), 1) shouldBe true + aritySetMatches(List(1, 2, 3), 1) shouldBe true } "deal with AnyRef" in { val ctup = new CTuple(None, List(1, 2, 3), 1 -> 2) - get[(AnyRef, AnyRef, AnyRef)](ctup) must be_==((None, List(1, 2, 3), 1 -> 2)) - get[AnyRef](new CTuple("you")) must be_==("you") + get[(AnyRef, AnyRef, AnyRef)](ctup) shouldBe (None, List(1, 2, 3), 1 -> 2) + get[AnyRef](new CTuple("you")) shouldBe "you" - roundTrip[AnyRef]("hey") must beTrue - roundTrip[(AnyRef, AnyRef)]((Nil, Nil)) must beTrue - arityConvMatches[(AnyRef, AnyRef)](("hey", "you"), 2) must beTrue - aritySetMatches[(AnyRef, AnyRef)](("hey", "you"), 2) must beTrue + roundTrip[AnyRef]("hey") shouldBe true + roundTrip[(AnyRef, AnyRef)]((Nil, Nil)) shouldBe true + arityConvMatches[(AnyRef, AnyRef)](("hey", "you"), 2) shouldBe true + aritySetMatches[(AnyRef, AnyRef)](("hey", "you"), 2) shouldBe true } } } 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 6b54f0e857..4197e18e3f 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/TypedDelimitedTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/TypedDelimitedTest.scala @@ -15,7 +15,7 @@ limitations under the License. */ package com.twitter.scalding -import org.specs._ +import org.scalatest.{ Matchers, WordSpec } import com.twitter.scalding._ import com.twitter.scalding.source.DailySuffixTypedTsv @@ -70,8 +70,7 @@ class DailySuffixTypedTsvJob(args: Args) extends Job(args) with UtcDateRangeJob } } -class TypedDelimitedTest extends Specification { - noDetailedDiffs() +class TypedDelimitedTest extends WordSpec with Matchers { import Dsl._ val data = List(("aaa", 1), ("bbb", 2)) @@ -79,9 +78,9 @@ class TypedDelimitedTest extends Specification { "A TypedTsv Source" should { JobTest(new TypedTsvJob(_)) .source(TypedTsv[(String, Int)]("input0"), data) - .sink[(String, Int)](TypedTsv[(String, Int)]("output0")) { buf => + .typedSink(TypedTsv[(String, Int)]("output0")) { buf => "read and write data" in { - buf must be_==(data) + buf shouldBe data } } .run @@ -91,9 +90,9 @@ class TypedDelimitedTest extends Specification { "A TypedCsv Source" should { JobTest(new TypedCsvJob(_)) .source(TypedCsv[(String, Int)]("input0"), data) - .sink[(String, Int)](TypedCsv[(String, Int)]("output0")) { buf => + .typedSink(TypedCsv[(String, Int)]("output0")) { buf => "read and write data" in { - buf must be_==(data) + buf shouldBe data } } .run @@ -103,21 +102,9 @@ class TypedDelimitedTest extends Specification { "A TypedPsv Source" should { JobTest(new TypedPsvJob(_)) .source(TypedPsv[(String, Int)]("input0"), data) - .sink[(String, Int)](TypedPsv[(String, Int)]("output0")) { buf => + .typedSink(TypedPsv[(String, Int)]("output0")) { buf => "read and write data" in { - buf must be_==(data) - } - } - .run - .finish - } - - "A TypedTsv Source" should { - JobTest(new TypedTsvJob(_)) - .source(TypedTsv[(String, Int)]("input0"), data) - .sink[(String, Int)](TypedTsv[(String, Int)]("output0")) { buf => - "read and write data" in { - buf must be_==(data) + buf shouldBe data } } .run @@ -127,9 +114,9 @@ class TypedDelimitedTest extends Specification { "A TypedOsv Source" should { JobTest(new TypedOsvJob(_)) .source(TypedOsv[(String, Int)]("input0"), data) - .sink[(String, Int)](TypedOsv[(String, Int)]("output0")) { buf => + .typedSink(TypedOsv[(String, Int)]("output0")) { buf => "read and write data" in { - buf must be_==(data) + buf shouldBe data } } .run @@ -141,9 +128,9 @@ class TypedDelimitedTest extends Specification { JobTest(new DailySuffixTypedTsvJob(_)) .arg("date", strd1 + " " + strd2) .source(source("input0"), data) - .sink[(String, Int)](TypedTsv[(String, Int)]("output0")) { buf => + .typedSink(TypedTsv[(String, Int)]("output0")) { buf => "read and write data" in { - buf must be_==(data) + buf shouldBe data } } .run diff --git a/scalding-core/src/test/scala/com/twitter/scalding/TypedFieldsTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/TypedFieldsTest.scala index bcc2b2bdf9..65e0768335 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/TypedFieldsTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/TypedFieldsTest.scala @@ -16,9 +16,9 @@ limitations under the License. package com.twitter.scalding import cascading.flow.FlowException -import org.specs._ +import org.scalatest.{ Matchers, WordSpec } -class TypedFieldsTest extends Specification { +class TypedFieldsTest extends WordSpec with Matchers { "A fields API job" should { @@ -26,40 +26,38 @@ class TypedFieldsTest extends Specification { // the Opaque class has no comparator "throw an exception if a field is not comparable" in { - - untypedJob must throwA(new FlowException("local step failed")) - + val thrown = the[FlowException] thrownBy untypedJob + thrown.getMessage shouldBe "local step failed" } // Now run the typed fields version "group by custom comparator correctly" in { - - JobTest("com.twitter.scalding.TypedFieldsJob"). - arg("input", "inputFile"). - arg("output", "outputFile"). - source(TextLine("inputFile"), List("0" -> "5,foo", "1" -> "6,bar", "2" -> "9,foo")). - sink[(Opaque, Int)](Tsv("outputFile")){ outputBuffer => + JobTest(new TypedFieldsJob(_)) + .arg("input", "inputFile") + .arg("output", "outputFile") + .source(TextLine("inputFile"), List("0" -> "5,foo", "1" -> "6,bar", "2" -> "9,foo")) + .sink[(Opaque, Int)](Tsv("outputFile")){ outputBuffer => val outMap = outputBuffer.map { case (opaque: Opaque, i: Int) => (opaque.str, i) }.toMap - outMap.size must_== 2 - outMap("foo") must be_==(14) - outMap("bar") must be_==(6) - }. - run. - finish + outMap should have size 2 + outMap("foo") shouldBe 14 + outMap("bar") shouldBe 6 + } + .run + .finish } } def untypedJob { - JobTest("com.twitter.scalding.UntypedFieldsJob"). - arg("input", "inputFile"). - arg("output", "outputFile"). - source(TextLine("inputFile"), List("0" -> "5,foo", "1" -> "6,bar", "2" -> "9,foo")). - sink[(Opaque, Int)](Tsv("outputFile")){ _ => }. - run. - finish + JobTest(new UntypedFieldsJob(_)) + .arg("input", "inputFile") + .arg("output", "outputFile") + .source(TextLine("inputFile"), List("0" -> "5,foo", "1" -> "6,bar", "2" -> "9,foo")) + .sink[(Opaque, Int)](Tsv("outputFile")){ _ => } + .run + .finish } } 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 51b919a798..3a345c051c 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/TypedPipeTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/TypedPipeTest.scala @@ -15,7 +15,7 @@ limitations under the License. */ package com.twitter.scalding -import org.specs._ +import org.scalatest.{ Matchers, WordSpec } // Use the scalacheck generators import org.scalacheck.Gen @@ -40,16 +40,15 @@ class TupleAdderJob(args: Args) extends Job(args) { .write(TypedTsv[(Int, String, String, Int, Int)]("output")) } -class TupleAdderTest extends Specification { +class TupleAdderTest extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() "A TupleAdderJob" should { JobTest(new TupleAdderJob(_)) .source(TypedTsv[(String, String)]("input", ('a, 'b)), List(("a", "a"), ("b", "b"))) .sink[(Int, String, String, Int, Int)](TypedTsv[(Int, String, String, Int, Int)]("output")) { outBuf => "be able to use generated tuple adders" in { - outBuf.size must_== 2 - outBuf.toSet must_== Set((1, "a", "a", 2, 3), (1, "b", "b", 2, 3)) + outBuf should have size 2 + outBuf.toSet shouldBe Set((1, "a", "a", 2, 3), (1, "b", "b", 2, 3)) } } .run @@ -70,23 +69,24 @@ class TypedPipeJob(args: Args) extends Job(args) { .write(TypedTsv[(String, Long)]("outputFile")) } -class TypedPipeTest extends Specification { +class TypedPipeTest extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() //Fixes an issue with scala 2.9 "A TypedPipe" should { + var idx = 0 TUtil.printStack { - JobTest(new com.twitter.scalding.TypedPipeJob(_)). - source(TextLine("inputFile"), List("0" -> "hack hack hack and hack")). - sink[(String, Long)](TypedTsv[(String, Long)]("outputFile")){ outputBuffer => + JobTest(new TypedPipeJob(_)) + .source(TextLine("inputFile"), List("0" -> "hack hack hack and hack")) + .sink[(String, Long)](TypedTsv[(String, Long)]("outputFile")){ outputBuffer => val outMap = outputBuffer.toMap - "count words correctly" in { - outMap("hack") must be_==(4) - outMap("and") must be_==(1) + (idx + ": count words correctly") in { + outMap("hack") shouldBe 4 + outMap("and") shouldBe 1 } - }. - run. - runHadoop. - finish + idx += 1 + } + .run + .runHadoop + .finish } } } @@ -99,22 +99,23 @@ class TypedSumByKeyJob(args: Args) extends Job(args) { .write(TypedTsv[(String, Long)]("outputFile")) } -class TypedSumByKeyTest extends Specification { - noDetailedDiffs() //Fixes an issue with scala 2.9 +class TypedSumByKeyTest extends WordSpec with Matchers { "A TypedSumByKeyPipe" should { + var idx = 0 TUtil.printStack { - JobTest(new com.twitter.scalding.TypedSumByKeyJob(_)). - source(TextLine("inputFile"), List("0" -> "hack hack hack and hack")). - sink[(String, Long)](TypedTsv[(String, Long)]("outputFile")){ outputBuffer => + JobTest(new TypedSumByKeyJob(_)) + .source(TextLine("inputFile"), List("0" -> "hack hack hack and hack")) + .sink[(String, Long)](TypedTsv[(String, Long)]("outputFile")){ outputBuffer => val outMap = outputBuffer.toMap - "count words correctly" in { - outMap("hack") must be_==(4) - outMap("and") must be_==(1) + (idx + ": count words correctly") in { + outMap("hack") shouldBe 4 + outMap("and") shouldBe 1 } - }. - run. - runHadoop. - finish + idx += 1 + } + .run + .runHadoop + .finish } } } @@ -126,8 +127,7 @@ class TypedPipeJoinJob(args: Args) extends Job(args) { .write(TypedTsv[(Int, (Int, Option[Int]))]("outputFile")) } -class TypedPipeJoinTest extends Specification { - noDetailedDiffs() //Fixes an issue with scala 2.9 +class TypedPipeJoinTest extends WordSpec with Matchers { import Dsl._ "A TypedPipeJoin" should { JobTest(new com.twitter.scalding.TypedPipeJoinJob(_)) @@ -136,16 +136,16 @@ class TypedPipeJoinTest extends Specification { .sink[(Int, (Int, Option[Int]))](TypedTsv[(Int, (Int, Option[Int]))]("outputFile")){ outputBuffer => val outMap = outputBuffer.toMap "correctly join" in { - outMap(0) must be_==((0, Some(1))) - outMap(1) must be_==((1, Some(2))) - outMap(2) must be_==((2, Some(3))) - outMap(3) must be_==((3, Some(4))) - outMap(4) must be_==((5, None)) - outMap.size must be_==(5) + outMap should have size 5 + outMap(0) shouldBe (0, Some(1)) + outMap(1) shouldBe (1, Some(2)) + outMap(2) shouldBe (2, Some(3)) + outMap(3) shouldBe (3, Some(4)) + outMap(4) shouldBe (5, None) } - }. - run. - finish + } + .run + .finish } } @@ -155,21 +155,19 @@ class TypedPipeDistinctJob(args: Args) extends Job(args) { .write(TypedTsv[(Int, Int)]("outputFile")) } -class TypedPipeDistinctTest extends Specification { - noDetailedDiffs() //Fixes an issue with scala 2.9 +class TypedPipeDistinctTest extends WordSpec with Matchers { import Dsl._ "A TypedPipeDistinctJob" should { - JobTest(new com.twitter.scalding.TypedPipeDistinctJob(_)) + JobTest(new TypedPipeDistinctJob(_)) .source(Tsv("inputFile"), List((0, 0), (1, 1), (2, 2), (2, 2), (2, 5))) .sink[(Int, Int)](TypedTsv[(Int, Int)]("outputFile")){ outputBuffer => val outMap = outputBuffer.toMap "correctly count unique item sizes" in { - val outSet = outputBuffer.toSet - outSet.size must_== 4 + outputBuffer.toSet should have size 4 } - }. - run. - finish + } + .run + .finish } } @@ -179,22 +177,20 @@ class TypedPipeDistinctByJob(args: Args) extends Job(args) { .write(TypedTsv[(Int, Int)]("outputFile")) } -class TypedPipeDistinctByTest extends Specification { - noDetailedDiffs() //Fixes an issue with scala 2.9 +class TypedPipeDistinctByTest extends WordSpec with Matchers { import Dsl._ "A TypedPipeDistinctByJob" should { - JobTest(new com.twitter.scalding.TypedPipeDistinctByJob(_)) + JobTest(new TypedPipeDistinctByJob(_)) .source(Tsv("inputFile"), List((0, 1), (1, 1), (2, 2), (2, 2), (2, 5))) - .sink[(Int, Int)](TypedTsv[(Int, Int)]("outputFile")){ outputBuffer => - val outMap = outputBuffer.toMap + .typedSink(TypedTsv[(Int, Int)]("outputFile")){ outputBuffer => "correctly count unique item sizes" in { val outSet = outputBuffer.toSet - outSet.size must_== 3 - outSet must beOneOf (Set((0, 1), (2, 2), (2, 5)), Set((1, 1), (2, 2), (2, 5))) + outSet should have size 3 + List(outSet) should contain oneOf (Set((0, 1), (2, 2), (2, 5)), Set((1, 1), (2, 2), (2, 5))) } - }. - run. - finish + } + .run + .finish } } @@ -205,26 +201,25 @@ class TypedPipeHashJoinJob(args: Args) extends Job(args) { .write(TypedTsv[(Int, (Int, Option[Int]))]("outputFile")) } -class TypedPipeHashJoinTest extends Specification { - noDetailedDiffs() //Fixes an issue with scala 2.9 +class TypedPipeHashJoinTest extends WordSpec with Matchers { import Dsl._ "A TypedPipeHashJoinJob" should { - JobTest(new com.twitter.scalding.TypedPipeHashJoinJob(_)) + JobTest(new TypedPipeHashJoinJob(_)) .source(TypedTsv[(Int, Int)]("inputFile0"), List((0, 0), (1, 1), (2, 2), (3, 3), (4, 5))) .source(TypedTsv[(Int, Int)]("inputFile1"), List((0, 1), (1, 2), (2, 3), (3, 4))) - .sink[(Int, (Int, Option[Int]))](TypedTsv[(Int, (Int, Option[Int]))]("outputFile")){ outputBuffer => + .typedSink(TypedTsv[(Int, (Int, Option[Int]))]("outputFile")){ outputBuffer => val outMap = outputBuffer.toMap "correctly join" in { - outMap(0) must be_==((0, Some(1))) - outMap(1) must be_==((1, Some(2))) - outMap(2) must be_==((2, Some(3))) - outMap(3) must be_==((3, Some(4))) - outMap(4) must be_==((5, None)) - outMap.size must be_==(5) + outMap should have size 5 + outMap(0) shouldBe (0, Some(1)) + outMap(1) shouldBe (1, Some(2)) + outMap(2) shouldBe (2, Some(3)) + outMap(3) shouldBe (3, Some(4)) + outMap(4) shouldBe (5, None) } - }. - run. - finish + } + .run + .finish } } @@ -246,17 +241,16 @@ class TypedImplicitJob(args: Args) extends Job(args) { }.write(TypedTsv[(String, Int)]("outputFile")) } -class TypedPipeTypedTest extends Specification { - noDetailedDiffs() //Fixes an issue with scala 2.9 +class TypedPipeTypedTest extends WordSpec with Matchers { import Dsl._ "A TypedImplicitJob" should { - JobTest(new com.twitter.scalding.TypedImplicitJob(_)) + JobTest(new TypedImplicitJob(_)) .source(TextLine("inputFile"), List("0" -> "hack hack hack and hack")) - .sink[(String, Int)](TypedTsv[(String, Int)]("outputFile")){ outputBuffer => + .typedSink(TypedTsv[(String, Int)]("outputFile")){ outputBuffer => val outMap = outputBuffer.toMap "find max word" in { - outMap("hack") must be_==(4) - outMap.size must be_==(1) + outMap should have size 1 + outMap("hack") shouldBe 4 } } .run @@ -281,20 +275,19 @@ class TypedWithOnCompleteJob(args: Args) extends Job(args) { .write(TypedTsv[String]("output")) } -class TypedPipeWithOnCompleteTest extends Specification { +class TypedPipeWithOnCompleteTest extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() val inputText = "the quick brown fox jumps over the lazy LAZY dog" "A TypedWithOnCompleteJob" should { JobTest(new TypedWithOnCompleteJob(_)) .source(TypedTsv[String]("input"), inputText.split("\\s+").map(Tuple1(_))) - .counter("onCompleteMapper") { cnt => "have onComplete called on mapper" in { cnt must_== 1 } } - .counter("onCompleteReducer") { cnt => "have onComplete called on reducer" in { cnt must_== 1 } } + .counter("onCompleteMapper") { cnt => "have onComplete called on mapper" in { assert(cnt == 1) } } + .counter("onCompleteReducer") { cnt => "have onComplete called on reducer" in { assert(cnt == 1) } } .sink[String](TypedTsv[String]("output")) { outbuf => "have the correct output" in { val correct = inputText.split("\\s+").map(_.toUpperCase).groupBy(x => x).filter(_._2.size > 1).keys.toList.sorted val sortedL = outbuf.toList.sorted - sortedL must_== (correct) + assert(sortedL == correct) } } .runHadoop @@ -314,9 +307,9 @@ class TypedPipeWithOuterAndLeftJoin(args: Args) extends Job(args) { .write(TypedTsv[Int]("output")) } -class TypedPipeWithOuterAndLeftJoinTest extends Specification { +class TypedPipeWithOuterAndLeftJoinTest extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() + "A TypedPipeWithOuterAndLeftJoin" should { JobTest(new TypedPipeWithOuterAndLeftJoin(_)) .source(TypedTsv[(Int, String)]("inputNames"), List((1, "Jimmy Foursquare"))) @@ -324,13 +317,13 @@ class TypedPipeWithOuterAndLeftJoinTest extends Specification { .source(TypedTsv[(Int, Boolean)]("inputOptionalData"), List((1, true), (99, false))) .sink[Long](TypedTsv[Int]("output")) { outbuf => "have output for user 1" in { - outbuf.toList.contains(1) must_== true + assert(outbuf.toList.contains(1) == true) } "have output for user 5" in { - outbuf.toList.contains(5) must_== true + assert(outbuf.toList.contains(5) == true) } "not have output for user 99" in { - outbuf.toList.contains(99) must_== false + assert(outbuf.toList.contains(99) == false) } } .run @@ -419,41 +412,44 @@ class TNiceJoinByCountJob(args: Args) extends Job(args) { .write(TypedTsv[(Int, Int, Int)]("out3")) } -class TypedPipeJoinCountTest extends Specification { - noDetailedDiffs() //Fixes an issue with scala 2.9 +class TypedPipeJoinCountTest extends WordSpec with Matchers { import Dsl._ val joinTests = List("com.twitter.scalding.TJoinCountJob", "com.twitter.scalding.TNiceJoinCountJob", "com.twitter.scalding.TNiceJoinByCountJob") joinTests.foreach{ jobName => "A " + jobName should { + var idx = 0 JobTest(jobName) .source(Tsv("in0", (0, 1)), List((0, 1), (0, 2), (1, 1), (1, 5), (2, 10))) .source(Tsv("in1", (0, 1)), List((0, 10), (1, 20), (1, 10), (1, 30))) - .sink[(Int, Long)](TypedTsv[(Int, Long)]("out")) { outbuf => + .typedSink(TypedTsv[(Int, Long)]("out")) { outbuf => val outMap = outbuf.toMap - "correctly reduce after cogroup" in { - outMap(0) must be_==(2) - outMap(1) must be_==(6) - outMap.size must be_==(2) + (idx + ": correctly reduce after cogroup") in { + outMap should have size 2 + outMap(0) shouldBe 2 + outMap(1) shouldBe 6 } + idx += 1 } - .sink[(Int, Int, Int)](TypedTsv[(Int, Int, Int)]("out2")) { outbuf2 => + .typedSink(TypedTsv[(Int, Int, Int)]("out2")) { outbuf2 => val outMap = outbuf2.groupBy { _._1 } - "correctly do a simple join" in { - outMap.size must be_==(2) - outMap(0).toList.sorted must be_==(List((0, 1, 10), (0, 2, 10))) - outMap(1).toList.sorted must be_==(List((1, 1, 10), (1, 1, 20), (1, 1, 30), (1, 5, 10), (1, 5, 20), (1, 5, 30))) + (idx + ": correctly do a simple join") in { + outMap should have size 2 + outMap(0).toList.sorted shouldBe List((0, 1, 10), (0, 2, 10)) + outMap(1).toList.sorted shouldBe List((1, 1, 10), (1, 1, 20), (1, 1, 30), (1, 5, 10), (1, 5, 20), (1, 5, 30)) } + idx += 1 } - .sink[(Int, Int, Int)](TypedTsv[(Int, Int, Int)]("out3")) { outbuf => + .typedSink(TypedTsv[(Int, Int, Int)]("out3")) { outbuf => val outMap = outbuf.groupBy { _._1 } - "correctly do a simple leftJoin" in { - outMap.size must be_==(3) - outMap(0).toList.sorted must be_==(List((0, 1, 10), (0, 2, 10))) - outMap(1).toList.sorted must be_==(List((1, 1, 10), (1, 1, 20), (1, 1, 30), (1, 5, 10), (1, 5, 20), (1, 5, 30))) - outMap(2).toList.sorted must be_==(List((2, 10, -1))) + (idx + ": correctly do a simple leftJoin") in { + outMap should have size 3 + outMap(0).toList.sorted shouldBe List((0, 1, 10), (0, 2, 10)) + outMap(1).toList.sorted shouldBe List((1, 1, 10), (1, 1, 20), (1, 1, 30), (1, 5, 10), (1, 5, 20), (1, 5, 30)) + outMap(2).toList.sorted shouldBe List((2, 10, -1)) } + idx += 1 } .run .runHadoop @@ -467,22 +463,23 @@ class TCrossJob(args: Args) extends Job(args) { .write(TypedTsv[(String, String)]("crossed")) } -class TypedPipeCrossTest extends Specification { - noDetailedDiffs() //Fixes an issue with scala 2.9 +class TypedPipeCrossTest extends WordSpec with Matchers { import Dsl._ "A TCrossJob" should { + var idx = 0 TUtil.printStack { - JobTest(new com.twitter.scalding.TCrossJob(_)) + JobTest(new TCrossJob(_)) .source(TextLine("in0"), List((0, "you"), (1, "all"))) .source(TextLine("in1"), List((0, "every"), (1, "body"))) - .sink[(String, String)](TypedTsv[(String, String)]("crossed")) { outbuf => + .typedSink(TypedTsv[(String, String)]("crossed")) { outbuf => val sortedL = outbuf.toList.sorted - "create a cross-product" in { - sortedL must be_==(List(("all", "body"), + (idx + ": create a cross-product") in { + sortedL shouldBe List(("all", "body"), ("all", "every"), ("you", "body"), - ("you", "every"))) + ("you", "every")) } + idx += 1 } .run .runHadoop @@ -501,20 +498,20 @@ class TJoinTakeJob(args: Args) extends Job(args) { .write(TypedTsv[(Int, String)]("joined")) } -class TypedJoinTakeTest extends Specification { - noDetailedDiffs() //Fixes an issue with scala 2.9 +class TypedJoinTakeTest extends WordSpec with Matchers { import Dsl._ "A TJoinTakeJob" should { + var idx = 0 TUtil.printStack { JobTest(new TJoinTakeJob(_)) .source(TextLine("in0"), List((0, "you"), (1, "all"))) .source(TextLine("in1"), List((0, "3"), (1, "2"), (0, "3"))) - .sink[(Int, String)](TypedTsv[(Int, String)]("joined")) { outbuf => + .typedSink(TypedTsv[(Int, String)]("joined")) { outbuf => val sortedL = outbuf.toList.sorted - "dedup keys by using take" in { - sortedL must be_==( - List((3, "you"), (3, "all"), (2, "you"), (2, "all")).sorted) + (idx + ": dedup keys by using take") in { + sortedL shouldBe (List((3, "you"), (3, "all"), (2, "you"), (2, "all")).sorted) } + idx += 1 } .run .runHadoop @@ -531,20 +528,21 @@ class TGroupAllJob(args: Args) extends Job(args) { .write(TypedTsv[String]("out")) } -class TypedGroupAllTest extends Specification { - noDetailedDiffs() //Fixes an issue with scala 2.9 +class TypedGroupAllTest extends WordSpec with Matchers { import Dsl._ "A TGroupAllJob" should { + var idx = 0 TUtil.printStack { val input = List((0, "you"), (1, "all"), (2, "everybody")) JobTest(new TGroupAllJob(_)) .source(TextLine("in"), input) - .sink[String](TypedTsv[String]("out")) { outbuf => + .typedSink(TypedTsv[String]("out")) { outbuf => val sortedL = outbuf.toList val correct = input.map { _._2 }.sorted - "create sorted output" in { - sortedL must_== (correct) + (idx + ": create sorted output") in { + sortedL shouldBe correct } + idx += 1 } .run .runHadoop @@ -558,14 +556,13 @@ class TSelfJoin(args: Args) extends Job(args) { g.join(g).values.write(TypedTsv[(Int, Int)]("out")) } -class TSelfJoinTest extends Specification { - noDetailedDiffs() //Fixes an issue with scala 2.9 +class TSelfJoinTest extends WordSpec with Matchers { import Dsl._ "A TSelfJoin" should { JobTest(new TSelfJoin(_)) .source(TypedTsv[(Int, Int)]("in"), List((1, 2), (1, 3), (2, 1))) - .sink[(Int, Int)](TypedTsv[(Int, Int)]("out")) { outbuf => - outbuf.toList.sorted must be_==(List((1, 1), (2, 2), (2, 3), (3, 2), (3, 3))) + .typedSink(TypedTsv[(Int, Int)]("out")) { outbuf => + outbuf.toList.sorted shouldBe List((1, 1), (2, 2), (2, 3), (3, 2), (3, 3)) } .run .runHadoop @@ -595,8 +592,7 @@ class TJoinWordCount(args: Args) extends Job(args) { .write(TypedTsv[(String, Int, Int)]("out")) } -class TypedJoinWCTest extends Specification { - noDetailedDiffs() //Fixes an issue with scala 2.9 +class TypedJoinWCTest extends WordSpec with Matchers { import Dsl._ "A TJoinWordCount" should { TUtil.printStack { @@ -616,10 +612,10 @@ class TypedJoinWCTest extends Specification { JobTest(new TJoinWordCount(_)) .source(TextLine("in0"), in0) .source(TextLine("in1"), in1) - .sink[(String, Int, Int)](TypedTsv[(String, Int, Int)]("out")) { outbuf => + .typedSink(TypedTsv[(String, Int, Int)]("out")) { outbuf => val sortedL = outbuf.toList "create sorted output" in { - sortedL must_== (correct) + sortedL shouldBe correct } } .run @@ -633,15 +629,14 @@ class TypedLimitJob(args: Args) extends Job(args) { p.write(TypedTsv[String]("output")) } -class TypedLimitTest extends Specification { +class TypedLimitTest extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() "A TypedLimitJob" should { JobTest(new TypedLimitJob(_)) .source(TypedTsv[String]("input"), (0 to 100).map { i => Tuple1(i.toString) }) - .sink[String](TypedTsv[String]("output")) { outBuf => + .typedSink(TypedTsv[String]("output")) { outBuf => "not have more than the limited outputs" in { - outBuf.size must be_<=(10) + outBuf.size should be <= 10 } } .runHadoop @@ -655,15 +650,14 @@ class TypedFlattenJob(args: Args) extends Job(args) { .write(TypedTsv[String]("output")) } -class TypedFlattenTest extends Specification { +class TypedFlattenTest extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() "A TypedLimitJob" should { JobTest(new TypedFlattenJob(_)) .source(TypedTsv[String]("input"), List(Tuple1("you all"), Tuple1("every body"))) - .sink[String](TypedTsv[String]("output")) { outBuf => + .typedSink(TypedTsv[String]("output")) { outBuf => "correctly flatten" in { - outBuf.toSet must be_==(Set("you", "all", "every", "body")) + outBuf.toSet shouldBe Set("you", "all", "every", "body") } } .runHadoop @@ -680,22 +674,24 @@ class TypedMergeJob(args: Args) extends Job(args) { .write(TypedTsv[String]("output2")) } -class TypedMergeTest extends Specification { +class TypedMergeTest extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() "A TypedMergeJob" should { + var idx = 0 JobTest(new TypedMergeJob(_)) .source(TypedTsv[String]("input"), List(Tuple1("you all"), Tuple1("every body"))) - .sink[String](TypedTsv[String]("output")) { outBuf => - "correctly flatten" in { - outBuf.toSet must be_==(Set("you all", "every body")) + .typedSink(TypedTsv[String]("output")) { outBuf => + (idx + ": correctly flatten") in { + outBuf.toSet shouldBe Set("you all", "every body") } + idx += 1 } - .sink[String](TypedTsv[String]("output2")) { outBuf => - "correctly flatten" in { + .typedSink(TypedTsv[String]("output2")) { outBuf => + (idx + ": correctly flatten") in { val correct = Set("you all", "every body") - outBuf.toSet must be_==(correct ++ correct.map(_.reverse)) + outBuf.toSet shouldBe (correct ++ correct.map(_.reverse)) } + idx += 1 } .runHadoop .finish @@ -710,19 +706,18 @@ class TypedShardJob(args: Args) extends Job(args) { .write(TypedTsv[String]("output")) } -class TypedShardTest extends Specification { +class TypedShardTest extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() "A TypedShardJob" should { val genList = Gen.listOf(Gen.identifier) // Take one random sample lazy val mk: List[String] = genList.sample.getOrElse(mk) JobTest(new TypedShardJob(_)) .source(TypedTsv[String]("input"), mk) - .sink[String](TypedTsv[String]("output")) { outBuf => + .typedSink(TypedTsv[String]("output")) { outBuf => "correctly flatten" in { - outBuf.size must be_==(mk.size + 1) - outBuf.toSet must be_==(mk.toSet + "item") + outBuf should have size (mk.size + 1) + outBuf.toSet shouldBe (mk.toSet + "item") } } .run @@ -737,23 +732,24 @@ class TypedLocalSumJob(args: Args) extends Job(args) { .write(TypedTsv[(String, Long)]("output")) } -class TypedLocalSumTest extends Specification { +class TypedLocalSumTest extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() "A TypedLocalSumJob" should { + var idx = 0 val genList = Gen.listOf(Gen.identifier) // Take one random sample lazy val mk: List[String] = genList.sample.getOrElse(mk) JobTest(new TypedLocalSumJob(_)) .source(TypedTsv[String]("input"), mk) - .sink[(String, Long)](TypedTsv[(String, Long)]("output")) { outBuf => - "not expand and have correct total sum" in { + .typedSink(TypedTsv[(String, Long)]("output")) { outBuf => + s"$idx: not expand and have correct total sum" in { import com.twitter.algebird.MapAlgebra.sumByKey val lres = outBuf.toList val fmapped = mk.flatMap { s => s.split(" ").map((_, 1L)) } - lres.size must be_<=(fmapped.size) - sumByKey(lres) must be_==(sumByKey(fmapped)) + lres.size should be <= (fmapped.size) + sumByKey(lres) shouldBe (sumByKey(fmapped)) } + idx += 1 } .run .runHadoop @@ -768,9 +764,8 @@ class TypedHeadJob(args: Args) extends Job(args) { .write(TypedTsv[(Int, Int)]("output")) } -class TypedHeadTest extends Specification { +class TypedHeadTest extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() "A TypedHeadJob" should { val rng = new java.util.Random val COUNT = 10000 @@ -778,11 +773,11 @@ class TypedHeadTest extends Specification { val mk = (1 to COUNT).map { _ => (rng.nextInt % KEYS, rng.nextInt) } JobTest(new TypedHeadJob(_)) .source(TypedTsv[(Int, Int)]("input"), mk) - .sink[(Int, Int)](TypedTsv[(Int, Int)]("output")) { outBuf => + .typedSink(TypedTsv[(Int, Int)]("output")) { outBuf => "correctly take the first" in { val correct = mk.groupBy(_._1).mapValues(_.head._2) - outBuf.size must be_==(correct.size) - outBuf.toMap must be_==(correct) + outBuf should have size (correct.size) + outBuf.toMap shouldBe correct } } .run @@ -807,9 +802,8 @@ class TypedSortWithTakeJob(args: Args) extends Job(args) { .write(TypedTsv[(Int, Int)]("output2")) } -class TypedSortWithTakeTest extends Specification { +class TypedSortWithTakeTest extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() "A TypedSortWithTakeJob" should { val rng = new java.util.Random val COUNT = 10000 @@ -820,13 +814,13 @@ class TypedSortWithTakeTest extends Specification { .sink[(Int, Int)](TypedTsv[(Int, Int)]("output")) { outBuf => "correctly take the first" in { val correct = mk.groupBy(_._1).mapValues(_.map(i => i._2).sorted.reverse.take(5).toSet) - outBuf.groupBy(_._1).mapValues(_.map { case (k, v) => v }.toSet) must be_==(correct) + outBuf.groupBy(_._1).mapValues(_.map { case (k, v) => v }.toSet) shouldBe correct } } .sink[(Int, Int)](TypedTsv[(Int, Int)]("output2")) { outBuf => "correctly take the first using sorted.reverse.take" in { val correct = mk.groupBy(_._1).mapValues(_.map(i => i._2).sorted.reverse.take(5).toSet) - outBuf.groupBy(_._1).mapValues(_.map { case (k, v) => v }.toSet) must be_==(correct) + outBuf.groupBy(_._1).mapValues(_.map { case (k, v) => v }.toSet) shouldBe correct } } .run @@ -840,9 +834,8 @@ class TypedLookupJob(args: Args) extends Job(args) { .write(TypedTsv[(Int, Option[String])]("output")) } -class TypedLookupJobTest extends Specification { +class TypedLookupJobTest extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() "A TypedLookupJob" should { val rng = new java.util.Random val COUNT = 10000 @@ -851,15 +844,15 @@ class TypedLookupJobTest extends Specification { JobTest(new TypedLookupJob(_)) .source(TypedTsv[Int]("input0"), (-1 to 100)) .source(TypedTsv[(Int, String)]("input1"), mk) - .sink[(Int, Option[String])](TypedTsv[(Int, Option[String])]("output")) { outBuf => + .typedSink(TypedTsv[(Int, Option[String])]("output")) { outBuf => "correctly TypedPipe.hashLookup" in { val data = mk.groupBy(_._1) .mapValues(kvs => kvs.map { case (k, v) => (k, Some(v)) }) val correct = (-1 to 100).flatMap { k => data.get(k).getOrElse(List((k, None))) }.toList.sorted - outBuf.size must be_==(correct.size) - outBuf.toList.sorted must be_==(correct) + outBuf should have size (correct.size) + outBuf.toList.sorted shouldBe correct } } .run @@ -873,9 +866,8 @@ class TypedLookupReduceJob(args: Args) extends Job(args) { .write(TypedTsv[(Int, Option[String])]("output")) } -class TypedLookupReduceJobTest extends Specification { +class TypedLookupReduceJobTest extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() "A TypedLookupJob" should { val rng = new java.util.Random val COUNT = 10000 @@ -884,7 +876,7 @@ class TypedLookupReduceJobTest extends Specification { JobTest(new TypedLookupReduceJob(_)) .source(TypedTsv[Int]("input0"), (-1 to 100)) .source(TypedTsv[(Int, String)]("input1"), mk) - .sink[(Int, Option[String])](TypedTsv[(Int, Option[String])]("output")) { outBuf => + .typedSink(TypedTsv[(Int, Option[String])]("output")) { outBuf => "correctly TypedPipe.hashLookup" in { val data = mk.groupBy(_._1) .mapValues { kvs => @@ -894,8 +886,8 @@ class TypedLookupReduceJobTest extends Specification { val correct = (-1 to 100).map { k => data.get(k).getOrElse((k, None)) }.toList.sorted - outBuf.size must be_==(correct.size) - outBuf.toList.sorted must be_==(correct) + outBuf should have size (correct.size) + outBuf.toList.sorted shouldBe correct } } .run @@ -910,9 +902,8 @@ class TypedFilterJob(args: Args) extends Job(args) { .write(TypedTsv[Int]("output")) } -class TypedFilterTest extends Specification { +class TypedFilterTest extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() //Fixes an issue with scala 2.9 "A TypedPipe" should { "filter and filterNot elements" in { val input = -1 to 100 @@ -920,14 +911,14 @@ class TypedFilterTest extends Specification { val expectedOutput = input filter { _ > 50 } filterNot isEven TUtil.printStack { - JobTest(new com.twitter.scalding.TypedFilterJob(_)). - source(TypedTsv[Int]("input"), input). - sink[Int](TypedTsv[Int]("output")) { outBuf => - outBuf.toList must be_==(expectedOutput) - }. - run. - runHadoop. - finish + JobTest(new com.twitter.scalding.TypedFilterJob(_)) + .source(TypedTsv[Int]("input"), input) + .typedSink(TypedTsv[Int]("output")) { outBuf => + outBuf.toList shouldBe expectedOutput + } + .run + .runHadoop + .finish } } } @@ -939,9 +930,8 @@ class TypedPartitionJob(args: Args) extends Job(args) { p2.write(TypedTsv[Int]("output2")) } -class TypedPartitionTest extends Specification { +class TypedPartitionTest extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() "A TypedPipe" should { "partition elements" in { val input = -1 to 100 @@ -950,11 +940,11 @@ class TypedPartitionTest extends Specification { TUtil.printStack { JobTest(new com.twitter.scalding.TypedPartitionJob(_)) .source(TypedTsv[Int]("input"), input) - .sink[Int](TypedTsv[Int]("output1")) { outBuf => - outBuf.toList must be_==(expected1) + .typedSink(TypedTsv[Int]("output1")) { outBuf => + outBuf.toList shouldBe expected1 } - .sink[Int](TypedTsv[Int]("output2")) { outBuf => - outBuf.toList must be_==(expected2) + .typedSink(TypedTsv[Int]("output2")) { outBuf => + outBuf.toList shouldBe expected2 } .run .runHadoop @@ -981,9 +971,8 @@ class TypedMultiJoinJob(args: Args) extends Job(args) { .write(TypedTsv[(Int, Int, Int, Int)]("output")) } -class TypedMultiJoinJobTest extends Specification { +class TypedMultiJoinJobTest extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() "A TypedMultiJoinJob" should { val rng = new java.util.Random val COUNT = 100 * 100 @@ -996,7 +985,7 @@ class TypedMultiJoinJobTest extends Specification { .source(TypedTsv[(Int, Int)]("input0"), mk0) .source(TypedTsv[(Int, Int)]("input1"), mk1) .source(TypedTsv[(Int, Int)]("input2"), mk2) - .sink[(Int, Int, Int, Int)](TypedTsv[(Int, Int, Int, Int)]("output")) { outBuf => + .typedSink(TypedTsv[(Int, Int, Int, Int)]("output")) { outBuf => "correctly do a multi-join" in { def groupMax(it: Seq[(Int, Int)]): Map[Int, Int] = it.groupBy(_._1).mapValues { kvs => @@ -1022,8 +1011,8 @@ class TypedMultiJoinJobTest extends Specification { } .toList.sorted - outBuf.size must be_==(correct.size) - outBuf.toList.sorted must be_==(correct) + outBuf should have size (correct.size) + outBuf.toList.sorted shouldBe correct } } .runHadoop @@ -1051,9 +1040,8 @@ class TypedMultiSelfJoinJob(args: Args) extends Job(args) { .write(TypedTsv[(Int, Int, Int, Int)]("output")) } -class TypedMultiSelfJoinJobTest extends Specification { +class TypedMultiSelfJoinJobTest extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() "A TypedMultiSelfJoinJob" should { val rng = new java.util.Random val COUNT = 10000 @@ -1064,7 +1052,7 @@ class TypedMultiSelfJoinJobTest extends Specification { JobTest(new TypedMultiSelfJoinJob(_)) .source(TypedTsv[(Int, Int)]("input0"), mk0) .source(TypedTsv[(Int, Int)]("input1"), mk1) - .sink[(Int, Int, Int, Int)](TypedTsv[(Int, Int, Int, Int)]("output")) { outBuf => + .typedSink(TypedTsv[(Int, Int, Int, Int)]("output")) { outBuf => "correctly do a multi-self-join" in { def group(it: Seq[(Int, Int)])(red: (Int, Int) => Int): Map[Int, Int] = it.groupBy(_._1).mapValues { kvs => @@ -1089,8 +1077,8 @@ class TypedMultiSelfJoinJobTest extends Specification { } .toList.sorted - outBuf.size must be_==(correct.size) - outBuf.toList.sorted must be_==(correct) + outBuf should have size (correct.size) + outBuf.toList.sorted shouldBe correct } } .runHadoop @@ -1106,9 +1094,8 @@ class TypedMapGroup(args: Args) extends Job(args) { .write(TypedTsv[(Int, Int)]("output")) } -class TypedMapGroupTest extends Specification { +class TypedMapGroupTest extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() "A TypedMapGroup" should { val rng = new java.util.Random val COUNT = 10000 @@ -1116,15 +1103,15 @@ class TypedMapGroupTest extends Specification { val mk = (1 to COUNT).map { _ => (rng.nextInt % KEYS, rng.nextInt) } JobTest(new TypedMapGroup(_)) .source(TypedTsv[(Int, Int)]("input"), mk) - .sink[(Int, Int)](TypedTsv[(Int, Int)]("output")) { outBuf => + .typedSink(TypedTsv[(Int, Int)]("output")) { outBuf => "correctly do a mapGroup" in { def mapGroup(it: Seq[(Int, Int)]): Map[Int, Int] = it.groupBy(_._1).mapValues { kvs => kvs.map { case (k, v) => k * v }.max }.toMap val correct = mapGroup(mk).toList.sorted - outBuf.size must be_==(correct.size) - outBuf.toList.sorted must be_==(correct) + outBuf should have size (correct.size) + outBuf.toList.sorted shouldBe correct } } .runHadoop @@ -1140,19 +1127,20 @@ class TypedSelfCrossJob(args: Args) extends Job(args) { .write(TypedTsv[(Int, Int)]("output")) } -class TypedSelfCrossTest extends Specification { +class TypedSelfCrossTest extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() val input = (1 to 100).toList "A TypedSelfCrossJob" should { + var idx = 0 JobTest(new TypedSelfCrossJob(_)) .source(TypedTsv[Int]("input"), input) - .sink[(Int, Int)](TypedTsv[(Int, Int)]("output")) { outBuf => - "not change the length of the input" in { - outBuf.size must_== input.size + .typedSink(TypedTsv[(Int, Int)]("output")) { outBuf => + (idx + ": not change the length of the input") in { + outBuf should have size (input.size) } + idx += 1 } .run .runHadoop @@ -1168,22 +1156,23 @@ class TypedSelfLeftCrossJob(args: Args) extends Job(args) { .write(TypedTsv[(Int, Option[Int])]("output")) } -class TypedSelfLeftCrossTest extends Specification { +class TypedSelfLeftCrossTest extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() val input = (1 to 100).toList "A TypedSelfLeftCrossJob" should { + var idx = 0 JobTest(new TypedSelfLeftCrossJob(_)) .source(TypedTsv[Int]("input"), input) - .sink[(Int, Option[Int])](TypedTsv[(Int, Option[Int])]("output")) { outBuf => - "attach the sum of all values correctly" in { - outBuf.size must_== input.size + .typedSink(TypedTsv[(Int, Option[Int])]("output")) { outBuf => + s"$idx: attach the sum of all values correctly" in { + outBuf should have size (input.size) val sum = input.reduceOption(_ + _) // toString to deal with our hadoop testing jank - outBuf.toList.sortBy(_._1).toString must be_== (input.sorted.map((_, sum)).toString) + outBuf.toList.sortBy(_._1).toString shouldBe (input.sorted.map((_, sum)).toString) } + idx += 1 } .run .runHadoop @@ -1198,15 +1187,15 @@ class JoinMapGroupJob(args: Args) extends Job(args) { .mapGroup { case (a, b) => Iterator("a") } .write(TypedTsv("output")) } -class JoinMapGroupJobTest extends Specification { + +class JoinMapGroupJobTest extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() "A JoinMapGroupJob" should { JobTest(new JoinMapGroupJob(_)) - .sink[(Int, String)](TypedTsv[(Int, String)]("output")) { outBuf => + .typedSink(TypedTsv[(Int, String)]("output")) { outBuf => "not duplicate keys" in { - outBuf.toList must be_==(List((1, "a"))) + outBuf.toList shouldBe List((1, "a")) } } .run @@ -1227,15 +1216,14 @@ class MapValueStreamNonEmptyIteratorJob(args: Args) extends Job(args) { .write(TypedTsv[(Int, Int)]("output")) } -class MapValueStreamNonEmptyIteratorTest extends Specification { +class MapValueStreamNonEmptyIteratorTest extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() "A MapValueStreamNonEmptyIteratorJob" should { JobTest(new MapValueStreamNonEmptyIteratorJob(_)) .sink[(Int, Int)](TypedTsv[(Int, Int)]("output")) { outBuf => "not have iterators of size 0" in { - outBuf.toList.filter(_._2 == 0) must be_==(Nil) + assert(outBuf.toList.filter(_._2 == 0) === Nil) } } .run @@ -1302,8 +1290,8 @@ object TypedSketchJoinTestHelper { .arg("reducers", reducers.toString) .source(TypedTsv[(Int, Int)]("input0"), generateInput(1000, 100, dist)) .source(TypedTsv[(Int, Int)]("input1"), generateInput(100, 100, x => 1)) - .sink[(Int, Int, Int)](TypedTsv[(Int, Int, Int)]("output-sketch")) { outBuf => sketchResult ++= outBuf } - .sink[(Int, Int, Int)](TypedTsv[(Int, Int, Int)]("output-join")) { outBuf => innerResult ++= outBuf } + .typedSink(TypedTsv[(Int, Int, Int)]("output-sketch")) { outBuf => sketchResult ++= outBuf } + .typedSink(TypedTsv[(Int, Int, Int)]("output-join")) { outBuf => innerResult ++= outBuf } .run .runHadoop .finish @@ -1312,71 +1300,66 @@ object TypedSketchJoinTestHelper { } } -class TypedSketchJoinJobTest extends Specification { +class TypedSketchJoinJobTest extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() - import TypedSketchJoinTestHelper._ "A TypedSketchJoinJob" should { "get the same result as an inner join" in { val (sk, inner) = runJobWithArguments(new TypedSketchJoinJob(_), 10, x => 1) - sk must_== inner + sk shouldBe inner } "get the same result when half the left keys are missing" in { val (sk, inner) = runJobWithArguments(new TypedSketchJoinJob(_), 10, x => if (x < 50) 0 else 1) - sk must_== inner + sk shouldBe inner } "get the same result with a massive skew to one key" in { val (sk, inner) = runJobWithArguments(new TypedSketchJoinJob(_), 10, x => if (x == 50) 1000 else 1) - sk must_== inner + sk shouldBe inner } "still work with only one reducer" in { val (sk, inner) = runJobWithArguments(new TypedSketchJoinJob(_), 1, x => 1) - sk must_== inner + sk shouldBe inner } "still work with massive skew and only one reducer" in { val (sk, inner) = runJobWithArguments(new TypedSketchJoinJob(_), 1, x => if (x == 50) 1000 else 1) - sk must_== inner + sk shouldBe inner } } } -class TypedSketchLeftJoinJobTest extends Specification { +class TypedSketchLeftJoinJobTest extends WordSpec with Matchers { import Dsl._ - noDetailedDiffs() - import TypedSketchJoinTestHelper._ "A TypedSketchLeftJoinJob" should { "get the same result as a left join" in { val (sk, left) = runJobWithArguments(new TypedSketchLeftJoinJob(_), 10, x => 1) - sk must_== left + sk shouldBe left } "get the same result when half the left keys are missing" in { val (sk, inner) = runJobWithArguments(new TypedSketchJoinJob(_), 10, x => if (x < 50) 0 else 1) - sk must_== inner + sk shouldBe inner } "get the same result with a massive skew to one key" in { val (sk, inner) = runJobWithArguments(new TypedSketchJoinJob(_), 10, x => if (x == 50) 1000 else 1) - sk must_== inner + sk shouldBe inner } "still work with only one reducer" in { val (sk, inner) = runJobWithArguments(new TypedSketchJoinJob(_), 1, x => 1) - sk must_== inner + sk shouldBe inner } "still work with massive skew and only one reducer" in { val (sk, inner) = runJobWithArguments(new TypedSketchJoinJob(_), 1, x => if (x == 50) 1000 else 1) - sk must_== inner + sk shouldBe inner } } } - diff --git a/scalding-core/src/test/scala/com/twitter/scalding/WeightedPageRankTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/WeightedPageRankTest.scala index 621314d020..ea11b98770 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/WeightedPageRankTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/WeightedPageRankTest.scala @@ -15,40 +15,43 @@ limitations under the License. */ package com.twitter.scalding -import org.specs._ +import org.scalatest.{ Matchers, WordSpec } -class WeightedPageRankSpec extends Specification { +class WeightedPageRankSpec extends WordSpec with Matchers { "Weighted PageRank job" should { - JobTest(new com.twitter.scalding.examples.WeightedPageRank(_)). - arg("pwd", "."). - arg("weighted", "true"). - arg("maxiterations", "1"). - arg("jumpprob", "0.1"). - source(Tsv("./nodes"), List((1, "2,3", "1,2", 0.26), (2, "3", "1", 0.54), (3, "", "", 0.2))). - source(Tsv("./numnodes"), List((3))). - source(Tsv("./pagerank_0"), List((1, 0.086), (2, 0.192), (3, 0.722))). - sink[Double](TypedTsv[Double]("./totaldiff")) { ob => - "have low error" in { - ob.head must beCloseTo(0.722 - 0.461 + 0.2964 - 0.192 + 0.2426 - 0.086, 0.001) + var idx = 0 + JobTest(new com.twitter.scalding.examples.WeightedPageRank(_)) + .arg("pwd", ".") + .arg("weighted", "true") + .arg("maxiterations", "1") + .arg("jumpprob", "0.1") + .source(Tsv("./nodes"), List((1, "2,3", "1,2", 0.26), (2, "3", "1", 0.54), (3, "", "", 0.2))) + .source(Tsv("./numnodes"), List((3))) + .source(Tsv("./pagerank_0"), List((1, 0.086), (2, 0.192), (3, 0.722))) + .typedSink(TypedTsv[Double]("./totaldiff")) { ob => + (idx + ": have low error") in { + ob.head shouldBe (0.722 - 0.461 + 0.2964 - 0.192 + 0.2426 - 0.086) +- 0.001 } - }. - sink[(Int, Double)](Tsv("./pagerank_1")){ outputBuffer => + idx += 1 + } + .sink[(Int, Double)](Tsv("./pagerank_1")){ outputBuffer => val pageRank = outputBuffer.map { res => (res._1, res._2) }.toMap - "correctly compute pagerank" in { + (idx + ": correctly compute pagerank") in { val deadMass = 0.722 / 3 * 0.9 val userMass = List(0.26, 0.54, 0.2).map { _ * 0.1 } val massNext = List(0, 0.086 / 3, (0.086 * 2 / 3 + 0.192)).map { _ * 0.9 } val expected = (userMass zip massNext) map { a: (Double, Double) => a._1 + a._2 + deadMass } println(pageRank) - (pageRank(1) + pageRank(2) + pageRank(3)) must beCloseTo(1.0, 0.001) - pageRank(1) must beCloseTo(expected(0), 0.001) - pageRank(2) must beCloseTo(expected(1), 0.001) - pageRank(3) must beCloseTo(expected(2), 0.001) + (pageRank(1) + pageRank(2) + pageRank(3)) shouldBe 1.0 +- 0.001 + pageRank(1) shouldBe (expected(0)) +- 0.001 + pageRank(2) shouldBe (expected(1)) +- 0.001 + pageRank(3) shouldBe (expected(2)) +- 0.001 } - }. - runWithoutNext(useHadoop = false). - runWithoutNext(useHadoop = true). - finish + idx += 1 + } + .runWithoutNext(useHadoop = false) + .runWithoutNext(useHadoop = true) + .finish } } 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 9a8421dd1d..8f0f907c23 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/WordCountTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/WordCountTest.scala @@ -15,22 +15,22 @@ limitations under the License. */ package com.twitter.scalding -import org.specs._ +import org.scalatest.{ Matchers, WordSpec } -class WordCountTest extends Specification { +class WordCountTest extends WordSpec with Matchers { "A WordCount job" should { - JobTest("com.twitter.scalding.examples.WordCountJob"). - arg("input", "inputFile"). - arg("output", "outputFile"). - source(TextLine("inputFile"), List((0, "hack hack hack and hack"))). - sink[(String, Int)](Tsv("outputFile")){ outputBuffer => + JobTest(new com.twitter.scalding.examples.WordCountJob(_)) + .arg("input", "inputFile") + .arg("output", "outputFile") + .source(TextLine("inputFile"), List((0, "hack hack hack and hack"))) + .sink[(String, Int)](Tsv("outputFile")){ outputBuffer => val outMap = outputBuffer.toMap "count words correctly" in { - outMap("hack") must be_==(4) - outMap("and") must be_==(1) + outMap("hack") shouldBe 4 + outMap("and") shouldBe 1 } - }. - run. - finish + } + .run + .finish } } diff --git a/scalding-core/src/test/scala/com/twitter/scalding/XHandlerTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/XHandlerTest.scala index 35e8ce6b8c..ae962cf668 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/XHandlerTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/XHandlerTest.scala @@ -15,38 +15,38 @@ limitations under the License. */ package com.twitter.scalding -import org.specs._ +import org.scalatest.{ Matchers, WordSpec } import cascading.flow.planner.PlannerException -class XHandlerTest extends Specification { +class XHandlerTest extends WordSpec with Matchers { "Throwable classes" should { "be handled if exist in default mapping" in { val rxh = RichXHandler() - rxh.handlers.find(h => h(new PlannerException)).isDefined must beTrue - rxh.handlers.find(h => h(new InvalidSourceException("Invalid Source"))).isDefined must beTrue - rxh.handlers.find(h => h(new NoSuchMethodError)).isDefined must beTrue - rxh.handlers.find(h => h(new AbstractMethodError)).isDefined must beTrue - rxh.handlers.find(h => h(new NoClassDefFoundError)).isDefined must beTrue + rxh.handlers.find(h => h(new PlannerException)) should not be empty + rxh.handlers.find(h => h(new InvalidSourceException("Invalid Source"))) should not be empty + rxh.handlers.find(h => h(new NoSuchMethodError)) should not be empty + rxh.handlers.find(h => h(new AbstractMethodError)) should not be empty + rxh.handlers.find(h => h(new NoClassDefFoundError)) should not be empty } "be handled if exist in custom mapping" in { val cRxh = RichXHandler(RichXHandler.mapping ++ Map(classOf[NullPointerException] -> "NPE")) - cRxh.handlers.find(h => h(new NullPointerException)).isDefined must beTrue - cRxh.mapping(classOf[NullPointerException]) must_== "NPE" + cRxh.handlers.find(h => h(new NullPointerException)) should not be empty + cRxh.mapping(classOf[NullPointerException]) shouldBe "NPE" } "not be handled if missing in mapping" in { val rxh = RichXHandler() - rxh.handlers.find(h => h(new NullPointerException)).isDefined must beFalse - rxh.handlers.find(h => h(new IndexOutOfBoundsException)).isDefined must beFalse + rxh.handlers.find(h => h(new NullPointerException)) shouldBe empty + rxh.handlers.find(h => h(new IndexOutOfBoundsException)) shouldBe empty } "be valid keys in mapping if defined" in { val rxh = RichXHandler() - rxh.mapping(classOf[PlannerException]) must_== RichXHandler.RequireSinks - rxh.mapping(classOf[InvalidSourceException]) must_== RichXHandler.DataIsMissing - rxh.mapping(classOf[NoSuchMethodError]) must_== RichXHandler.BinaryProblem - rxh.mapping(classOf[AbstractMethodError]) must_== RichXHandler.BinaryProblem - rxh.mapping(classOf[NoClassDefFoundError]) must_== RichXHandler.BinaryProblem - rxh.mapping(classOf[NullPointerException]) must_== RichXHandler.Default + rxh.mapping(classOf[PlannerException]) shouldBe RichXHandler.RequireSinks + rxh.mapping(classOf[InvalidSourceException]) shouldBe RichXHandler.DataIsMissing + rxh.mapping(classOf[NoSuchMethodError]) shouldBe RichXHandler.BinaryProblem + rxh.mapping(classOf[AbstractMethodError]) shouldBe RichXHandler.BinaryProblem + rxh.mapping(classOf[NoClassDefFoundError]) shouldBe RichXHandler.BinaryProblem + rxh.mapping(classOf[NullPointerException]) shouldBe RichXHandler.Default } "create a URL link in GitHub wiki" in { val NoClassDefFoundErrorString = "javalangnoclassdeffounderror" @@ -54,12 +54,11 @@ class XHandlerTest extends Specification { val NoSuchMethodErrorString = "javalangnosuchmethoderror" val InvalidSouceExceptionString = "comtwitterscaldinginvalidsourceexception" val PlannerExceptionString = "cascadingflowplannerplannerexception" - RichXHandler.createXUrl(new PlannerException) must_== RichXHandler.gitHubUrl + PlannerExceptionString - RichXHandler.createXUrl(new InvalidSourceException("Invalid Source")) must_== RichXHandler.gitHubUrl + InvalidSouceExceptionString - RichXHandler.createXUrl(new NoSuchMethodError) must_== RichXHandler.gitHubUrl + NoSuchMethodErrorString - RichXHandler.createXUrl(new AbstractMethodError) must_== RichXHandler.gitHubUrl + AbstractMethodErrorString - RichXHandler.createXUrl(new NoClassDefFoundError) must_== RichXHandler.gitHubUrl + NoClassDefFoundErrorString + RichXHandler.createXUrl(new PlannerException) shouldBe (RichXHandler.gitHubUrl + PlannerExceptionString) + RichXHandler.createXUrl(new InvalidSourceException("Invalid Source")) shouldBe (RichXHandler.gitHubUrl + InvalidSouceExceptionString) + RichXHandler.createXUrl(new NoSuchMethodError) shouldBe (RichXHandler.gitHubUrl + NoSuchMethodErrorString) + RichXHandler.createXUrl(new AbstractMethodError) shouldBe (RichXHandler.gitHubUrl + AbstractMethodErrorString) + RichXHandler.createXUrl(new NoClassDefFoundError) shouldBe (RichXHandler.gitHubUrl + NoClassDefFoundErrorString) } - } } diff --git a/scalding-core/src/test/scala/com/twitter/scalding/bdd/MultipleSourcesSpecTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/bdd/MultipleSourcesSpecTest.scala index 8ab4f960ca..047b049000 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/bdd/MultipleSourcesSpecTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/bdd/MultipleSourcesSpecTest.scala @@ -1,12 +1,12 @@ package com.twitter.scalding.bdd -import org.specs.Specification +import org.scalatest.{ Matchers, WordSpec } import com.twitter.scalding.Dsl._ import com.twitter.scalding.RichPipe import scala.collection.mutable.Buffer import cascading.tuple.Tuple -class MultipleSourcesSpecTest extends Specification with BddDsl { +class MultipleSourcesSpecTest extends WordSpec with Matchers with BddDsl { "A test with two sources" should { "accept an operation with two input pipes" in { @@ -26,7 +26,7 @@ class MultipleSourcesSpecTest extends Specification with BddDsl { { buffer.forall({ case (_, _, _, addressTransf) => addressTransf.endsWith("_transf") - }) mustBe true + }) shouldBe true } } } @@ -48,7 +48,7 @@ class MultipleSourcesSpecTest extends Specification with BddDsl { { buffer.forall({ case (_, _, _, addressTransf) => addressTransf.endsWith("_transf") - }) mustBe true + }) shouldBe true } } } @@ -76,7 +76,7 @@ class MultipleSourcesSpecTest extends Specification with BddDsl { } Then { buffer: Buffer[Tuple] => { - buffer.forall(tuple => tuple.getString(2).endsWith("_transf")) mustBe true + buffer.forall(tuple => tuple.getString(2).endsWith("_transf")) shouldBe true } } } @@ -84,31 +84,33 @@ class MultipleSourcesSpecTest extends Specification with BddDsl { "A test with four sources" should { "compile mixing an operation with inconsistent number of input pipes but fail at runtime" in { - Given { - List(("col1_1", "col2_1"), ("col1_2", "col2_2")) withSchema ('col1, 'col2) - } And { - List(("col1_1", "col2_1"), ("col1_2", "col2_2")) withSchema ('col1, 'col3) - } And { - List(("col1_1", "col2_1"), ("col1_2", "col2_2")) withSchema ('col1, 'col4) - } And { - List(("col1_1", "col2_1"), ("col1_2", "col2_2")) withSchema ('col1, 'col5) - } When { - (pipe1: RichPipe, pipe2: RichPipe, pipe3: RichPipe) => - { - pipe1 - .joinWithSmaller('col1 -> 'col1, pipe2) - .joinWithSmaller('col1 -> 'col1, pipe3) - .joinWithSmaller('col1 -> 'col1, pipe3) - .map('col1 -> 'col1_transf) { - col1: String => col1 + "_transf" - } - } - } Then { - buffer: Buffer[Tuple] => - { - buffer.forall(tuple => tuple.getString(2).endsWith("_transf")) mustBe true - } - } must throwA[IllegalArgumentException] + an[IllegalArgumentException] should be thrownBy { + Given { + List(("col1_1", "col2_1"), ("col1_2", "col2_2")) withSchema ('col1, 'col2) + } And { + List(("col1_1", "col2_1"), ("col1_2", "col2_2")) withSchema ('col1, 'col3) + } And { + List(("col1_1", "col2_1"), ("col1_2", "col2_2")) withSchema ('col1, 'col4) + } And { + List(("col1_1", "col2_1"), ("col1_2", "col2_2")) withSchema ('col1, 'col5) + } When { + (pipe1: RichPipe, pipe2: RichPipe, pipe3: RichPipe) => + { + pipe1 + .joinWithSmaller('col1 -> 'col1, pipe2) + .joinWithSmaller('col1 -> 'col1, pipe3) + .joinWithSmaller('col1 -> 'col1, pipe3) + .map('col1 -> 'col1_transf) { + col1: String => col1 + "_transf" + } + } + } Then { + buffer: Buffer[Tuple] => + { + buffer.forall(tuple => tuple.getString(2).endsWith("_transf")) shouldBe true + } + } + } } "be used with a function accepting a list of sources because there is no implicit for functions with more than three input pipes" in { @@ -135,7 +137,7 @@ class MultipleSourcesSpecTest extends Specification with BddDsl { } Then { buffer: Buffer[Tuple] => { - buffer.forall(tuple => tuple.getString(2).endsWith("_transf")) mustBe true + buffer.forall(tuple => tuple.getString(2).endsWith("_transf")) shouldBe true } } } diff --git a/scalding-core/src/test/scala/com/twitter/scalding/bdd/SingleSourceSpecTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/bdd/SingleSourceSpecTest.scala index c9ce8d3a22..b5a7589cc6 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/bdd/SingleSourceSpecTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/bdd/SingleSourceSpecTest.scala @@ -1,14 +1,13 @@ package com.twitter.scalding.bdd -import org.specs.Specification +import org.scalatest.{ Matchers, WordSpec } import com.twitter.scalding.{ Dsl, RichPipe } import scala.collection.mutable.Buffer import cascading.pipe.Pipe import cascading.tuple.Tuple import com.twitter.scalding.Dsl._ -class SingleSourceSpecTest extends Specification with BddDsl { - +class SingleSourceSpecTest extends WordSpec with Matchers with BddDsl { "A test with single source" should { "accept an operation with a single input rich pipe" in { Given { @@ -25,7 +24,7 @@ class SingleSourceSpecTest extends Specification with BddDsl { { buffer.forall({ case (_, _, transformed) => transformed.endsWith("_transf") - }) mustBe true + }) shouldBe true } } } @@ -45,7 +44,7 @@ class SingleSourceSpecTest extends Specification with BddDsl { { buffer.forall({ case (_, _, transformed) => transformed.endsWith("_transf") - }) mustBe true + }) shouldBe true } } } @@ -63,7 +62,7 @@ class SingleSourceSpecTest extends Specification with BddDsl { } Then { buffer: Buffer[Tuple] => { - buffer.forall(tuple => tuple.getString(2).endsWith("_transf")) mustBe true + buffer.forall(tuple => tuple.getString(2).endsWith("_transf")) shouldBe true } } } @@ -81,7 +80,7 @@ class SingleSourceSpecTest extends Specification with BddDsl { } Then { buffer: Buffer[Tuple] => { - buffer.forall(tuple => tuple.getString(1).endsWith("_transf")) mustBe true + buffer.forall(tuple => tuple.getString(1).endsWith("_transf")) shouldBe true } } } @@ -99,7 +98,7 @@ class SingleSourceSpecTest extends Specification with BddDsl { } Then { buffer: Buffer[Tuple] => { - buffer.forall(tuple => tuple.getString(2).endsWith("_transf")) mustBe true + buffer.forall(tuple => tuple.getString(2).endsWith("_transf")) shouldBe true } } } diff --git a/scalding-core/src/test/scala/com/twitter/scalding/bdd/SourceListSpecTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/bdd/SourceListSpecTest.scala index 25e381c8c2..fdf4f74b27 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/bdd/SourceListSpecTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/bdd/SourceListSpecTest.scala @@ -1,36 +1,38 @@ package com.twitter.scalding.bdd -import org.specs.Specification +import org.scalatest.{ Matchers, WordSpec } import com.twitter.scalding.RichPipe import scala.collection.mutable.Buffer import cascading.tuple.Tuple import cascading.pipe.Pipe import com.twitter.scalding.Dsl._ -class SourceListSpecTest extends Specification with BddDsl { +class SourceListSpecTest extends WordSpec with Matchers with BddDsl { "A test with a list of sources" should { "compile mixing it with a multi pipe function but fail if not same cardinality between given and when clause" in { - Given { - List( - (List(("col1_1", "col2_1"), ("col1_2", "col2_2")) withSchema ('col1, 'col2)), - (List(("col1_1", "col2_1"), ("col1_2", "col2_2")) withSchema ('col1, 'col3)), - (List(("col1_1", "col2_1"), ("col1_2", "col2_2")) withSchema ('col1, 'col4))) - } When { - (pipe1: RichPipe, pipe2: RichPipe) => - { - pipe1 - .joinWithSmaller('col1 -> 'col1, pipe2) - .map('col1 -> 'col1_transf) { - col1: String => col1 + "_transf" - } - } - } Then { - buffer: Buffer[Tuple] => - { - buffer.forall(tuple => tuple.getString(2).endsWith("_transf")) mustBe true - } - } must throwA[IllegalArgumentException] + an[IllegalArgumentException] should be thrownBy { + Given { + List( + (List(("col1_1", "col2_1"), ("col1_2", "col2_2")) withSchema ('col1, 'col2)), + (List(("col1_1", "col2_1"), ("col1_2", "col2_2")) withSchema ('col1, 'col3)), + (List(("col1_1", "col2_1"), ("col1_2", "col2_2")) withSchema ('col1, 'col4))) + } When { + (pipe1: RichPipe, pipe2: RichPipe) => + { + pipe1 + .joinWithSmaller('col1 -> 'col1, pipe2) + .map('col1 -> 'col1_transf) { + col1: String => col1 + "_transf" + } + } + } Then { + buffer: Buffer[Tuple] => + { + buffer.forall(tuple => tuple.getString(2).endsWith("_transf")) shouldBe true + } + } + } } "work properly with a multi rich-pipe function with same cardinality" in { @@ -51,7 +53,7 @@ class SourceListSpecTest extends Specification with BddDsl { } Then { buffer: Buffer[Tuple] => { - buffer.forall(tuple => tuple.getString(2).endsWith("_transf")) mustBe true + buffer.forall(tuple => tuple.getString(2).endsWith("_transf")) shouldBe true } } } @@ -74,7 +76,7 @@ class SourceListSpecTest extends Specification with BddDsl { } Then { buffer: Buffer[Tuple] => { - buffer.forall(tuple => tuple.getString(2).endsWith("_transf")) mustBe true + buffer.forall(tuple => tuple.getString(2).endsWith("_transf")) shouldBe true } } } @@ -97,7 +99,7 @@ class SourceListSpecTest extends Specification with BddDsl { } Then { buffer: Buffer[Tuple] => { - buffer.forall(tuple => tuple.getString(2).endsWith("_transf")) mustBe true + buffer.forall(tuple => tuple.getString(2).endsWith("_transf")) shouldBe true } } } @@ -120,7 +122,7 @@ class SourceListSpecTest extends Specification with BddDsl { } Then { buffer: Buffer[Tuple] => { - buffer.forall(tuple => tuple.getString(2).endsWith("_transf")) mustBe true + buffer.forall(tuple => tuple.getString(2).endsWith("_transf")) shouldBe true } } } diff --git a/scalding-core/src/test/scala/com/twitter/scalding/bdd/TypedApiTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/bdd/TypedApiTest.scala index e8ddf2447f..e53f4760c6 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/bdd/TypedApiTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/bdd/TypedApiTest.scala @@ -3,7 +3,7 @@ package com.twitter.scalding.bdd import cascading.flow.FlowException import com.twitter.scalding.typed.TDsl import com.twitter.scalding._ -import org.specs.Specification +import org.scalatest.{ Matchers, WordSpec } import scala.math._ import scala.collection.mutable @@ -13,10 +13,8 @@ case class UserWithAge(name: String, age: Int) case class UserInfo(name: String, gender: String, age: Int) case class EstimatedContribution(name: String, suggestedPensionContributionPerMonth: Double) -class TypedApiTest extends Specification with TBddDsl { - +class TypedApiTest extends WordSpec with Matchers with TBddDsl { "A test with a single source" should { - "accept an operation from working with a single tuple-typed pipe" in { Given { List(("Joe", "M", 40), ("Sarah", "F", 22)) @@ -30,7 +28,7 @@ class TypedApiTest extends Specification with TBddDsl { } } Then { buffer: mutable.Buffer[(String, Double)] => - buffer.toList mustEqual List(("Joe", 1000.0 / 32), ("Sarah", 1000.0 / 58)) + buffer.toList shouldBe List(("Joe", 1000.0 / 32), ("Sarah", 1000.0 / 58)) } } @@ -47,7 +45,7 @@ class TypedApiTest extends Specification with TBddDsl { } } Then { buffer: mutable.Buffer[EstimatedContribution] => - buffer.toList mustEqual List(EstimatedContribution("Joe", 1000.0 / 32), EstimatedContribution("Sarah", 1000.0 / 58)) + buffer.toList shouldBe List(EstimatedContribution("Joe", 1000.0 / 32), EstimatedContribution("Sarah", 1000.0 / 58)) } } } @@ -71,7 +69,7 @@ class TypedApiTest extends Specification with TBddDsl { } } Then { buffer: mutable.Buffer[(String, String, Int)] => - buffer.toList mustEqual List(("Joe", "M", 40), ("Sarah", "F", 22)) + buffer.toList shouldBe List(("Joe", "M", 40), ("Sarah", "F", 22)) } } @@ -92,7 +90,7 @@ class TypedApiTest extends Specification with TBddDsl { .values } Then { buffer: mutable.Buffer[UserInfo] => - buffer.toList mustEqual List(UserInfo("Joe", "M", 40), UserInfo("Sarah", "F", 22)) + buffer.toList shouldBe List(UserInfo("Joe", "M", 40), UserInfo("Sarah", "F", 22)) } } } @@ -118,32 +116,34 @@ class TypedApiTest extends Specification with TBddDsl { .values } Then { buffer: mutable.Buffer[UserInfo] => - buffer.toList mustEqual List(UserInfo("Joe", "M", 40), UserInfo("Sarah", "F", 22)) + buffer.toList shouldBe List(UserInfo("Joe", "M", 40), UserInfo("Sarah", "F", 22)) } } "not checking the types of the sources and fail if any error occurs" in { - GivenSources { - List( - List(UserWithGender("Joe", "M"), UserWithGender("Sarah", "F")), - List(("Joe", 40), ("Sarah", 22))) - } When { - pipes: List[TypedPipe[_]] => - val gender = pipes(0).asInstanceOf[TypedPipe[UserWithGender]] - val age = pipes(1).asInstanceOf[TypedPipe[UserWithAge]] - - gender - .groupBy(_.name) - .join(age.groupBy(_.name)) - .mapValues { value: (UserWithGender, UserWithAge) => - val (withGender, withAge) = value - UserInfo(withGender.name, withGender.gender, withAge.age) - } - .values - } Then { - buffer: mutable.Buffer[UserInfo] => - buffer.toList mustEqual List(UserInfo("Joe", "M", 40), UserInfo("Sarah", "F", 22)) - } must throwA[FlowException] + an[FlowException] should be thrownBy { + GivenSources { + List( + List(UserWithGender("Joe", "M"), UserWithGender("Sarah", "F")), + List(("Joe", 40), ("Sarah", 22))) + } When { + pipes: List[TypedPipe[_]] => + val gender = pipes(0).asInstanceOf[TypedPipe[UserWithGender]] + val age = pipes(1).asInstanceOf[TypedPipe[UserWithAge]] + + gender + .groupBy(_.name) + .join(age.groupBy(_.name)) + .mapValues { value: (UserWithGender, UserWithAge) => + val (withGender, withAge) = value + UserInfo(withGender.name, withGender.gender, withAge.age) + } + .values + } Then { + buffer: mutable.Buffer[UserInfo] => + buffer.toList shouldBe List(UserInfo("Joe", "M", 40), UserInfo("Sarah", "F", 22)) + } + } } "be created when adding a source to four sources" in { @@ -187,7 +187,7 @@ class TypedApiTest extends Specification with TBddDsl { .values } Then { buffer: mutable.Buffer[EstimatedContribution] => - buffer.toList mustEqual List(EstimatedContribution("Joe", 35.0), EstimatedContribution("Sarah", 13.0)) + buffer.toList shouldBe List(EstimatedContribution("Joe", 35.0), EstimatedContribution("Sarah", 13.0)) } } } diff --git a/scalding-core/src/test/scala/com/twitter/scalding/examples/WeightedPageRankFromMatrixTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/examples/WeightedPageRankFromMatrixTest.scala index d10e19e9db..771d20b8b2 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/examples/WeightedPageRankFromMatrixTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/examples/WeightedPageRankFromMatrixTest.scala @@ -17,14 +17,14 @@ package com.twitter.scalding.examples import scala.collection._ -import org.specs._ +import org.scalatest.{ Matchers, WordSpec } import com.twitter.scalding._ import com.twitter.scalding.Dsl._ import WeightedPageRankFromMatrixSpec._ -class WeightedPageRankFromMatrixSpec extends Specification { +class WeightedPageRankFromMatrixSpec extends WordSpec with Matchers { "Weighted PageRank from Matrix job" should { @@ -49,65 +49,65 @@ class WeightedPageRankFromMatrixSpec extends Specification { val expectedSolution = Array(0.28, 0.173333, 0.173333, 0.173333, 0.2) - JobTest("com.twitter.scalding.examples.WeightedPageRankFromMatrix"). - arg("d", d.toString). - arg("n", n.toString). - arg("convergenceThreshold", "0.0001"). - arg("maxIterations", "1"). - arg("currentIteration", "0"). - arg("rootDir", "root"). - source(TypedTsv[(Int, Int, Double)]("root/edges"), edges). - source(TypedTsv[(Int, Double)]("root/onesVector"), onesVector). - source(TypedTsv[(Int, Double)]("root/iterations/0"), iterationZeroVector). - sink[(Int, Int, Double)](Tsv("root/constants/M_hat")) { outputBuffer => - outputBuffer.size must be (7) + JobTest(new WeightedPageRankFromMatrix(_)) + .arg("d", d.toString) + .arg("n", n.toString) + .arg("convergenceThreshold", "0.0001") + .arg("maxIterations", "1") + .arg("currentIteration", "0") + .arg("rootDir", "root") + .source(TypedTsv[(Int, Int, Double)]("root/edges"), edges) + .source(TypedTsv[(Int, Double)]("root/onesVector"), onesVector) + .source(TypedTsv[(Int, Double)]("root/iterations/0"), iterationZeroVector) + .sink[(Int, Int, Double)](Tsv("root/constants/M_hat")) { outputBuffer => + outputBuffer should have size 7 val outputMap = toSparseMap(outputBuffer) - outputMap((0 -> 1)) must beCloseTo (0.4, 0) - outputMap((0 -> 2)) must beCloseTo (0.4, 0) - outputMap((1 -> 3)) must beCloseTo (0.26666, 0.00001) - outputMap((2 -> 3)) must beCloseTo (0.13333, 0.00001) - outputMap((2 -> 4)) must beCloseTo (0.13333, 0.00001) - outputMap((3 -> 4)) must beCloseTo (0.26666, 0.00001) - outputMap((4 -> 0)) must beCloseTo (0.4, 0) - }. - sink[(Int, Double)](Tsv("root/constants/priorVector")) { outputBuffer => - outputBuffer.size must be (5) + outputMap((0 -> 1)) shouldBe 0.4 + outputMap((0 -> 2)) shouldBe 0.4 + outputMap((1 -> 3)) shouldBe 0.26666 +- 0.00001 + outputMap((2 -> 3)) shouldBe 0.13333 +- 0.00001 + outputMap((2 -> 4)) shouldBe 0.13333 +- 0.00001 + outputMap((3 -> 4)) shouldBe 0.26666 +- 0.00001 + outputMap((4 -> 0)) shouldBe 0.4 + } + .sink[(Int, Double)](Tsv("root/constants/priorVector")) { outputBuffer => + outputBuffer should have size 5 val expectedValue = ((1 - d) / 2) * d assertVectorsEqual( new Array[Double](5).map { v => expectedValue }, outputBuffer.map(_._2).toArray) - }. - sink[(Int, Double)](Tsv("root/iterations/1")) { outputBuffer => - outputBuffer.size must be (5) + } + .sink[(Int, Double)](Tsv("root/iterations/1")) { outputBuffer => + outputBuffer should have size 5 assertVectorsEqual( expectedSolution, outputBuffer.map(_._2).toArray, 0.00001) - }. - sink[Double](TypedTsv[Double]("root/diff")) { outputBuffer => - outputBuffer.size must be (1) + } + .typedSink(TypedTsv[Double]("root/diff")) { outputBuffer => + outputBuffer should have size 1 val expectedDiff = expectedSolution.zip(iterationZeroVector.map(_._2)). map { case (a, b) => math.abs(a - b) }. sum - outputBuffer.head must beCloseTo (expectedDiff, 0.00001) - }. - run. - finish + outputBuffer.head shouldBe expectedDiff +- 0.00001 + } + .run + .finish } private def assertVectorsEqual(expected: Array[Double], actual: Array[Double], variance: Double) { actual.zipWithIndex.foreach { case (value, i) => - value must beCloseTo (expected(i), variance) + value shouldBe (expected(i)) +- variance } } private def assertVectorsEqual(expected: Array[Double], actual: Array[Double]) { actual.zipWithIndex.foreach { case (value, i) => - value must beCloseTo (expected(i), 0) + value shouldBe (expected(i)) } } } 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 95ecaf2dad..70aec69590 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 @@ -20,11 +20,11 @@ import com.twitter.scalding._ import java.io.File import java.net.URI import org.apache.hadoop.conf.Configuration -import org.specs.Specification -import org.specs.mock.Mockito +import org.scalatest.{ Matchers, WordSpec } import scala.collection.mutable - -class DistributedCacheFileSpec extends Specification with Mockito { +/* +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] @@ -52,7 +52,7 @@ class DistributedCacheFileSpec extends Specification with Mockito { "DistributedCacheFile" should { "symlinkNameFor must return a hashed name" in { - DistributedCacheFile.symlinkNameFor(uri) must_== hashedFilename + DistributedCacheFile.symlinkNameFor(uri) shouldBe hashedFilename } } @@ -63,8 +63,8 @@ class DistributedCacheFileSpec extends Specification with Mockito { "use the local file path" in { val cf = dcf.add()(mode) - cf.path must_== uri.getPath - cf.file must_== new File(uri.getPath).getCanonicalFile + cf.path shouldBe (uri.getPath) + cf.file shouldBe (new File(uri.getPath).getCanonicalFile) } } @@ -78,7 +78,8 @@ class DistributedCacheFileSpec extends Specification with Mockito { "throw RuntimeException when the current mode isn't recognized" in { val mode = smartMock[UnknownMode] - dcf.add()(mode) must throwA[RuntimeException] + an[RuntimeException] should be thrownBy (dcf.add()(mode)) } } } +*/ diff --git a/scalding-core/src/test/scala/com/twitter/scalding/mathematics/CombinatoricsTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/mathematics/CombinatoricsTest.scala index 5e4a00c26b..7418d378f6 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/mathematics/CombinatoricsTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/mathematics/CombinatoricsTest.scala @@ -15,7 +15,7 @@ limitations under the License. */ package com.twitter.scalding.mathematics -import org.specs._ +import org.scalatest.{ Matchers, WordSpec } import com.twitter.scalding._ class CombinatoricsJob(args: Args) extends Job(args) { @@ -35,34 +35,30 @@ class CombinatoricsJob(args: Args) extends Job(args) { } -class CombinatoricsJobTest extends Specification { - noDetailedDiffs() +class CombinatoricsJobTest extends WordSpec with Matchers { import Dsl._ "A Combinatorics Job" should { JobTest(new CombinatoricsJob(_)) .sink[(Int, Int)](Tsv("perms.txt")) { pbuf => - val psize = pbuf.toList.size "correctly compute 10 permute 3 equals 720" in { - psize must be_==(720) + pbuf.toList should have size 720 } } .sink[(Int, Int)](Tsv("combs.txt")) { buf => val csize = buf.toList.size "correctly compute 5 choose 2 equals 10" in { - csize must be_==(10) + buf.toList should have size 10 } } .sink[(Int, Int, Int, Int)](Tsv("invest.txt")) { buf => - val isize = buf.toList.size "correctly compute 169 tuples that allow you to invest $1000 among the 4 given stocks" in { - isize must be_==(169) + buf.toList should have size 169 } } .sink[(Int, Int, Int, Int)](Tsv("investpos.txt")) { buf => - val ipsize = buf.toList.size "correctly compute 101 non-zero tuples that allow you to invest $1000 among the 4 given stocks" in { - ipsize must be_==(101) + buf.toList should have size 101 } } .run diff --git a/scalding-core/src/test/scala/com/twitter/scalding/mathematics/HistogramTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/mathematics/HistogramTest.scala index 3c035f9857..e2d8e02a99 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/mathematics/HistogramTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/mathematics/HistogramTest.scala @@ -15,7 +15,7 @@ limitations under the License. */ package com.twitter.scalding.mathematics -import org.specs._ +import org.scalatest.{ Matchers, WordSpec } import com.twitter.scalding._ class HistogramJob(args: Args) extends Job(args) { @@ -36,36 +36,35 @@ class HistogramJob(args: Args) extends Job(args) { } } -class HistogramJobTest extends Specification { - noDetailedDiffs() +class HistogramJobTest extends WordSpec with Matchers { import Dsl._ val values = List(1.0, 2.5, 1.5, 3.0, 3.0, 3.0, 4.2, 2.0, 8.0, 1.0) val inputData = values.map(Tuple1(_)) val cdfOutput = Set((1.0, 0.3), (2.0, 0.5), (3.0, 0.8), (4.0, 0.9), (8.0, 1.0)) "A HistogramJob" should { - JobTest("com.twitter.scalding.mathematics.HistogramJob") + JobTest(new HistogramJob(_)) .source(Tsv("input", ('n)), inputData) .sink[(Double, Double, Double, Double, Double)](Tsv("stats-output")) { buf => val (min, max, sum, mean, stdDev) = buf.head "correctly compute the min" in { - min must be_==(values.map(_.floor).min) + min shouldBe (values.map(_.floor).min) } "correctly compute the max" in { - max must be_==(values.map(_.floor).max) + max shouldBe (values.map(_.floor).max) } "correctly compute the sum" in { - sum must be_==(values.map(_.floor).sum) + sum shouldBe (values.map(_.floor).sum) } "correctly compute the mean" in { - mean must be_==(values.map(_.floor).sum / values.size) + mean shouldBe (values.map(_.floor).sum / values.size) } "correctly compute the stdDev" in { - stdDev must beCloseTo(1.989974874, 0.000000001) + stdDev shouldBe 1.989974874 +- 0.000000001 } } .sink[(Double, Double)](Tsv("cdf-output")) { buf => "correctly compute a CDF" in { - buf.toSet must be_==(cdfOutput) + buf.toSet shouldBe cdfOutput } } .run diff --git a/scalding-core/src/test/scala/com/twitter/scalding/mathematics/Matrix2OptimizationTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/mathematics/Matrix2OptimizationTest.scala index 406d7b7cea..1350e8f949 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/mathematics/Matrix2OptimizationTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/mathematics/Matrix2OptimizationTest.scala @@ -21,7 +21,7 @@ import org.scalacheck.Properties import org.scalacheck.Prop.forAll import org.scalacheck._ import org.scalacheck.Gen._ -import org.specs._ +import org.scalatest.{ Matchers, WordSpec } import com.twitter.scalding._ import Matrix2._ import cascading.flow.FlowDef @@ -32,7 +32,7 @@ import com.twitter.scalding.IterableSource * Unit tests used in development * (stronger properties are tested in ScalaCheck tests at the end) */ -class Matrix2OptimizationSpec extends Specification { +class Matrix2OptimizationSpec extends WordSpec with Matchers { import Dsl._ import com.twitter.scalding.Test @@ -132,56 +132,56 @@ class Matrix2OptimizationSpec extends Specification { "handle a single matrix" in { val p = IndexedSeq(literal(globM, FiniteHint(30, 35))) val result = optimizeProductChain(p, Some(ring, MatrixJoiner2.default)) - (result == (0, literal(globM, FiniteHint(30, 35)))) must beTrue + result shouldBe (0, literal(globM, FiniteHint(30, 35))) } "handle two matrices" in { val p = IndexedSeq(literal(globM, FiniteHint(30, 35)), literal(globM, FiniteHint(35, 25))) val result = optimizeProductChain(p, Some(ring, MatrixJoiner2.default)) - ((simplePlanCost, simplePlan) == result) must beTrue + (simplePlanCost, simplePlan) shouldBe result } "handle an example with 6 matrices" in { val result = optimizeProductChain(productSequence, Some(ring, MatrixJoiner2.default)) - ((optimizedPlanCost, optimizedPlan) == result) must beTrue + (optimizedPlanCost, optimizedPlan) shouldBe result } "not change an optimized plan" in { - ((optimizedPlanCost, optimizedPlan) == optimize(optimizedPlan)) must beTrue + (optimizedPlanCost, optimizedPlan) shouldBe optimize(optimizedPlan) } "change an unoptimized plan" in { - ((optimizedPlanCost, optimizedPlan) == optimize(unoptimizedPlan)) must beTrue + (optimizedPlanCost, optimizedPlan) shouldBe optimize(unoptimizedPlan) } "handle an optimized plan with sum" in { - ((combinedOptimizedPlanCost, combinedOptimizedPlan) == optimize(combinedOptimizedPlan)) must beTrue + (combinedOptimizedPlanCost, combinedOptimizedPlan) shouldBe optimize(combinedOptimizedPlan) } "handle an unoptimized plan with sum" in { - ((combinedOptimizedPlanCost, combinedOptimizedPlan) == optimize(combinedUnoptimizedPlan)) must beTrue + (combinedOptimizedPlanCost, combinedOptimizedPlan) shouldBe (optimize(combinedUnoptimizedPlan)) } "not break A*(B+C)" in { - (planWithSum == optimize(planWithSum)._2) must beTrue + planWithSum shouldBe (optimize(planWithSum)._2) } "handle an unoptimized global plan" in { - (optimizedGlobalPlan == optimize(unoptimizedGlobalPlan)._2) must beTrue + optimizedGlobalPlan shouldBe (optimize(unoptimizedGlobalPlan)._2) } "handle an optimized global plan" in { - (optimizedGlobalPlan == optimize(optimizedGlobalPlan)._2) must beTrue + optimizedGlobalPlan shouldBe (optimize(optimizedGlobalPlan)._2) } "handle a G^5 V plan" in { - (optimizedGraphVectorPlan == optimize(unoptimizedGraphVectorPlan)._2) must beTrue + optimizedGraphVectorPlan shouldBe (optimize(unoptimizedGraphVectorPlan)._2) } "handle an optimized G^5 V plan" in { - (optimizedGraphVectorPlan == optimize(optimizedGraphVectorPlan)._2) must beTrue + optimizedGraphVectorPlan shouldBe (optimize(optimizedGraphVectorPlan)._2) } "handle a G^8 plan" in { - (optimizedGraph8 == optimize(g ^ 8)._2) must beTrue + optimizedGraph8 shouldBe (optimize(g ^ 8)._2) } } @@ -235,14 +235,18 @@ object Matrix2Props extends Properties("Matrix2") { } } - def genNode(depth: Int) = for { + def genNode(depth: Int): Gen[Matrix2[Any, Any, Double]] = for { v <- arbitrary[Int] p <- Gen.choose(1, 10) left <- genFormula(depth + 1) right <- genFormula(depth + 1) } yield if (depth > 5) randomProduct(p) else (if (v > 0) randomProduct(p) else Sum(left, right, ring)) - def genFormula(depth: Int): Gen[Matrix2[Any, Any, Double]] = if (depth > 5) genLeaf((0, 0))._1 else (oneOf(genNode(depth + 1), genLeaf((0, 0))._1)) + def genFormula(depth: Int): Gen[Matrix2[Any, Any, Double]] = + if (depth > 5) + genLeaf((0, 0))._1 + else + (oneOf(genNode(depth + 1), Gen.const(genLeaf((0, 0))._1))) implicit def arbT: Arbitrary[Matrix2[Any, Any, Double]] = Arbitrary(genFormula(0)) 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 0a888a2ace..f3faee0c1b 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 @@ -17,7 +17,7 @@ package com.twitter.scalding.mathematics import com.twitter.scalding._ import cascading.pipe.joiner._ -import org.specs._ +import org.scalatest.{ Matchers, WordSpec } import com.twitter.algebird.{ Ring, Group } class Matrix2Sum(args: Args) extends Job(args) { @@ -244,8 +244,7 @@ class Scalar2Ops(args: Args) extends Job(args) { } -class Matrix2Test extends Specification { - noDetailedDiffs() // For scala 2.9 +class Matrix2Test extends WordSpec with Matchers { import Dsl._ def toSparseMat[Row, Col, V](iter: Iterable[(Row, Col, V)]): Map[(Row, Col), V] = { @@ -257,13 +256,12 @@ class Matrix2Test extends Specification { "A MatrixSum job" should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.Matrix2Sum") + JobTest(new Matrix2Sum(_)) .source(Tsv("mat1", ('x1, 'y1, 'v1)), List((1, 1, 1.0), (2, 2, 3.0), (1, 2, 4.0))) .source(Tsv("mat2", ('x2, 'y2, 'v2)), List((1, 3, 3.0), (2, 1, 8.0), (1, 2, 4.0))) - .sink[(Int, Int, Double)](TypedTsv[(Int, Int, Double)]("sum")) { ob => + .typedSink(TypedTsv[(Int, Int, Double)]("sum")) { ob => "correctly compute sums" in { - val pMap = toSparseMat(ob) - pMap must be_==(Map((1, 1) -> 1.0, (1, 2) -> 8.0, (1, 3) -> 3.0, (2, 1) -> 8.0, (2, 2) -> 3.0)) + toSparseMat(ob) shouldBe Map((1, 1) -> 1.0, (1, 2) -> 8.0, (1, 3) -> 3.0, (2, 1) -> 8.0, (2, 2) -> 3.0) } } .runHadoop @@ -273,15 +271,14 @@ class Matrix2Test extends Specification { "A Matrix2Sum3 job, where the Matrix contains tuples as values," should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.Matrix2Sum3") + JobTest(new Matrix2Sum3(_)) .source(Tsv("mat1", ('x1, 'y1, 'v1)), List((1, 1, (1.0, 3.0, 5.0)), (2, 2, (3.0, 2.0, 1.0)), (1, 2, (4.0, 5.0, 2.0)))) - .sink[(Int, Int, String)](TypedTsv[(Int, Int, (Double, Double, Double))]("sum")) { ob => + .typedSink(TypedTsv[(Int, Int, (Double, Double, Double))]("sum")) { ob => "correctly compute sums" in { // Treat (Double, Double, Double) as string because that is what is actually returned // when using runHadoop - val pMap = toSparseMat(ob) val result = Map((1, 1) -> (2.0, 6.0, 10.0), (2, 2) -> (6.0, 4.0, 2.0), (1, 2) -> (8.0, 10.0, 4.0)).mapValues(_.toString) - pMap must be_==(result) + toSparseMat(ob) shouldBe result } } .runHadoop @@ -291,14 +288,13 @@ class Matrix2Test extends Specification { "A Matrix2SumChain job" should { TUtil.printStack { - JobTest(new com.twitter.scalding.mathematics.Matrix2SumChain(_)) + JobTest(new Matrix2SumChain(_)) .source(Tsv("mat1", ('x1, 'y1, 'v1)), List((1, 1, 1.0), (2, 2, 3.0), (1, 2, 4.0))) .source(Tsv("mat2", ('x2, 'y2, 'v2)), List((1, 3, 3.0), (2, 1, 8.0), (1, 2, 4.0))) .source(Tsv("mat3", ('x3, 'y3, 'v3)), List((1, 3, 4.0), (2, 1, 1.0), (1, 2, 4.0))) - .sink[(Int, Int, Double)](TypedTsv[(Int, Int, Double)]("sum")) { ob => + .typedSink(TypedTsv[(Int, Int, Double)]("sum")) { ob => "correctly compute sums" in { - val pMap = toSparseMat(ob) - pMap must be_==(Map((1, 1) -> 1.0, (1, 2) -> 12.0, (1, 3) -> 7.0, (2, 1) -> 9.0, (2, 2) -> 3.0)) + toSparseMat(ob) shouldBe Map((1, 1) -> 1.0, (1, 2) -> 12.0, (1, 3) -> 7.0, (2, 1) -> 9.0, (2, 2) -> 3.0) } } .runHadoop @@ -308,14 +304,13 @@ class Matrix2Test extends Specification { "A Matrix2HadSum job" should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.Matrix2HadSum") + JobTest(new Matrix2HadSum(_)) .source(Tsv("mat1", ('x1, 'y1, 'v1)), List((1, 3, 1.0), (2, 2, 3.0))) .source(Tsv("mat2", ('x2, 'y2, 'v2)), List((1, 3, 3.0), (2, 1, 8.0), (1, 2, 4.0))) .source(Tsv("mat3", ('x3, 'y3, 'v3)), List((1, 3, 4.0), (2, 1, 1.0), (1, 2, 4.0))) - .sink[(Int, Int, Double)](TypedTsv[(Int, Int, Double)]("hadSum")) { ob => + .typedSink(TypedTsv[(Int, Int, Double)]("hadSum")) { ob => "correctly compute a combination of a Hadamard product and a sum" in { - val pMap = toSparseMat(ob) - pMap must be_==(Map((1, 3) -> 7.0)) + toSparseMat(ob) shouldBe Map((1, 3) -> 7.0) } } .runHadoop @@ -325,12 +320,11 @@ class Matrix2Test extends Specification { "A Matrix2 RowRowHad job" should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.Matrix2RowRowHad") + JobTest(new Matrix2RowRowHad(_)) .source(Tsv("mat1", ('x1, 'y1, 'v1)), List((1, 1, 1.0), (2, 2, 3.0), (1, 2, 4.0))) - .sink[(Int, Double)](TypedTsv[(Int, Double)]("rowRowHad")) { ob => + .typedSink(TypedTsv[(Int, Double)]("rowRowHad")) { ob => "correctly compute a Hadamard product of row vectors" in { - val pMap = oneDtoSparseMat(ob) - pMap must be_==(Map((1, 1) -> 1.0, (2, 2) -> 16.0)) + oneDtoSparseMat(ob) shouldBe Map((1, 1) -> 1.0, (2, 2) -> 16.0) } } .runHadoop @@ -340,13 +334,12 @@ class Matrix2Test extends Specification { "A Matrix2 ZeroHad job" should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.Matrix2ZeroHad") + JobTest(new Matrix2ZeroHad(_)) .source(Tsv("mat1", ('x1, 'y1, 'v1)), List((1, 1, 1.0), (2, 2, 3.0), (1, 2, 4.0))) .source(Tsv("mat2", ('x2, 'y2, 'v2)), List()) - .sink[(Int, Int, Double)](TypedTsv[(Int, Int, Double)]("zeroHad")) { ob => + .typedSink(TypedTsv[(Int, Int, Double)]("zeroHad")) { ob => "correctly compute a Hadamard product with a zero matrix" in { - val pMap = toSparseMat(ob) - pMap must be_==(Map()) + toSparseMat(ob) shouldBe empty } } .runHadoop @@ -356,12 +349,11 @@ class Matrix2Test extends Specification { "A Matrix2Prod job" should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.Matrix2Prod") + JobTest(new Matrix2Prod(_)) .source(Tsv("mat1", ('x1, 'y1, 'v1)), List((1, 1, 1.0), (2, 2, 3.0), (1, 2, 4.0))) - .sink[(Int, Int, Double)](TypedTsv[(Int, Int, Double)]("product")) { ob => + .typedSink(TypedTsv[(Int, Int, Double)]("product")) { ob => "correctly compute products" in { - val pMap = toSparseMat(ob) - pMap must be_==(Map((1, 1) -> 17.0, (1, 2) -> 12.0, (2, 1) -> 12.0, (2, 2) -> 9.0)) + toSparseMat(ob) shouldBe Map((1, 1) -> 17.0, (1, 2) -> 12.0, (2, 1) -> 12.0, (2, 2) -> 9.0) } } .runHadoop @@ -371,12 +363,11 @@ class Matrix2Test extends Specification { "A Matrix2JProd job" should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.Matrix2JProd") + JobTest(new Matrix2JProd(_)) .source(Tsv("mat1", ('x1, 'y1, 'v1)), List((1, 1, 1.0), (2, 2, 3.0), (1, 2, 4.0))) - .sink[(Int, Int, Double)](TypedTsv[(Int, Int, Double)]("product")) { ob => + .typedSink(TypedTsv[(Int, Int, Double)]("product")) { ob => "correctly compute products with infinite matrices" in { - val pMap = toSparseMat(ob) - pMap must be_==(Map((1, 1) -> 5.0, (1, 2) -> 35.0, (2, 1) -> 3.0, (2, 2) -> 21.0)) + toSparseMat(ob) shouldBe Map((1, 1) -> 5.0, (1, 2) -> 35.0, (2, 1) -> 3.0, (2, 2) -> 21.0) } } .runHadoop @@ -384,15 +375,14 @@ class Matrix2Test extends Specification { } } - "A Matrix2Prod job" should { + "A Matrix2ProdSum job" should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.Matrix2ProdSum") + JobTest(new Matrix2ProdSum(_)) .source(Tsv("mat1", ('x1, 'y1, 'v1)), List((1, 1, 1.0), (2, 2, 3.0), (1, 2, 4.0))) .source(Tsv("mat2", ('x2, 'y2, 'v2)), List((1, 1, 1.0), (1, 2, 1.0), (2, 1, 1.0), (2, 2, 1.0))) - .sink[(Int, Int, Double)](TypedTsv[(Int, Int, Double)]("product-sum")) { ob => + .typedSink(TypedTsv[(Int, Int, Double)]("product-sum")) { ob => "correctly compute products" in { - val pMap = toSparseMat(ob) - pMap must be_==(Map((1, 1) -> 18.0, (1, 2) -> 13.0, (2, 1) -> 13.0, (2, 2) -> 10.0)) + toSparseMat(ob) shouldBe Map((1, 1) -> 18.0, (1, 2) -> 13.0, (2, 1) -> 13.0, (2, 2) -> 10.0) } } .runHadoop @@ -414,14 +404,14 @@ class Matrix2Test extends Specification { .source(TypedTsv[(Int, Int, Int)]("graph"), List((0, 1, 1), (0, 2, 1), (1, 2, 1), (2, 0, 1))) .source(TypedTsv[(Int, Double)]("row"), List((0, 1.0), (1, 2.0), (2, 4.0))) .source(TypedTsv[(Int, Double)]("col"), List((0, 1.0), (1, 2.0), (2, 4.0))) - .sink[(Int, Double)](TypedTsv[(Int, Double)]("prop-col")) { ob => + .typedSink(TypedTsv[(Int, Double)]("prop-col")) { ob => "correctly propagate columns" in { - ob.toMap must be_==(Map(0 -> 6.0, 1 -> 4.0, 2 -> 1.0)) + ob.toMap shouldBe Map(0 -> 6.0, 1 -> 4.0, 2 -> 1.0) } } - .sink[(Int, Double)](TypedTsv[(Int, Double)]("prop-row")) { ob => + .typedSink(TypedTsv[(Int, Double)]("prop-row")) { ob => "correctly propagate rows" in { - ob.toMap must be_==(Map(0 -> 4.0, 1 -> 1.0, 2 -> 3.0)) + ob.toMap shouldBe Map(0 -> 4.0, 1 -> 1.0, 2 -> 3.0) } } .runHadoop @@ -431,12 +421,11 @@ class Matrix2Test extends Specification { "A Matrix2 Cosine job" should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.Matrix2Cosine") + JobTest(new Matrix2Cosine(_)) .source(Tsv("mat1", ('x1, 'y1, 'v1)), List((1, 1, 1.0), (2, 2, 3.0), (1, 2, 4.0))) - .sink[(Int, Int, Double)](TypedTsv[(Int, Int, Double)]("cosine")) { ob => + .typedSink(TypedTsv[(Int, Int, Double)]("cosine")) { ob => "correctly compute cosine similarity" in { - val pMap = toSparseMat(ob) - pMap must be_==(Map((1, 1) -> 1.0, (1, 2) -> 0.9701425001453319, (2, 1) -> 0.9701425001453319, (2, 2) -> 1.0)) + toSparseMat(ob) shouldBe Map((1, 1) -> 1.0, (1, 2) -> 0.9701425001453319, (2, 1) -> 0.9701425001453319, (2, 2) -> 1.0) } } .runHadoop @@ -446,41 +435,40 @@ class Matrix2Test extends Specification { "A Matrix2 Scalar2Ops job" should { TUtil.printStack { - JobTest(new com.twitter.scalding.mathematics.Scalar2Ops(_)) + JobTest(new Scalar2Ops(_)) .source(Tsv("mat1", ('x1, 'y1, 'v1)), List((1, 1, 1.0), (2, 2, 3.0), (1, 2, 4.0))) - .sink[(Int, Int, Double)](TypedTsv[(Int, Int, Double)]("times3")) { ob => + .typedSink(TypedTsv[(Int, Int, Double)]("times3")) { ob => "correctly compute M * 3" in { - toSparseMat(ob) must be_==(Map((1, 1) -> 3.0, (2, 2) -> 9.0, (1, 2) -> 12.0)) + toSparseMat(ob) shouldBe Map((1, 1) -> 3.0, (2, 2) -> 9.0, (1, 2) -> 12.0) } } - .sink[(Int, Int, Double)](TypedTsv[(Int, Int, Double)]("div3")) { ob => + .typedSink(TypedTsv[(Int, Int, Double)]("div3")) { ob => "correctly compute M / 3" in { - toSparseMat(ob) must be_==(Map((1, 1) -> (1.0 / 3.0), (2, 2) -> (3.0 / 3.0), (1, 2) -> (4.0 / 3.0))) + toSparseMat(ob) shouldBe Map((1, 1) -> (1.0 / 3.0), (2, 2) -> (3.0 / 3.0), (1, 2) -> (4.0 / 3.0)) } } - .sink[(Int, Int, Double)](TypedTsv[(Int, Int, Double)]("3times")) { ob => + .typedSink(TypedTsv[(Int, Int, Double)]("3times")) { ob => "correctly compute 3 * M" in { - toSparseMat(ob) must be_==(Map((1, 1) -> 3.0, (2, 2) -> 9.0, (1, 2) -> 12.0)) + toSparseMat(ob) shouldBe Map((1, 1) -> 3.0, (2, 2) -> 9.0, (1, 2) -> 12.0) } } - .sink[(Int, Int, Double)](TypedTsv[(Int, Int, Double)]("timestrace")) { ob => + .typedSink(TypedTsv[(Int, Int, Double)]("timestrace")) { ob => "correctly compute M * Tr(M)" in { - toSparseMat(ob) must be_==(Map((1, 1) -> 4.0, (2, 2) -> 12.0, (1, 2) -> 16.0)) + toSparseMat(ob) shouldBe Map((1, 1) -> 4.0, (2, 2) -> 12.0, (1, 2) -> 16.0) } } - .sink[(Int, Int, Double)](TypedTsv[(Int, Int, Double)]("tracetimes")) { ob => + .typedSink(TypedTsv[(Int, Int, Double)]("tracetimes")) { ob => "correctly compute Tr(M) * M" in { - toSparseMat(ob) must be_==(Map((1, 1) -> 4.0, (2, 2) -> 12.0, (1, 2) -> 16.0)) + toSparseMat(ob) shouldBe Map((1, 1) -> 4.0, (2, 2) -> 12.0, (1, 2) -> 16.0) } } - .sink[(Int, Int, Double)](TypedTsv[(Int, Int, Double)]("divtrace")) { ob => + .typedSink(TypedTsv[(Int, Int, Double)]("divtrace")) { ob => "correctly compute M / Tr(M)" in { - toSparseMat(ob) must be_==(Map((1, 1) -> (1.0 / 4.0), (2, 2) -> (3.0 / 4.0), (1, 2) -> (4.0 / 4.0))) + toSparseMat(ob) shouldBe Map((1, 1) -> (1.0 / 4.0), (2, 2) -> (3.0 / 4.0), (1, 2) -> (4.0 / 4.0)) } } .runHadoop .finish } } - } diff --git a/scalding-core/src/test/scala/com/twitter/scalding/mathematics/MatrixTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/mathematics/MatrixTest.scala index cc8456df3a..211c27272c 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/mathematics/MatrixTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/mathematics/MatrixTest.scala @@ -17,7 +17,7 @@ package com.twitter.scalding.mathematics import com.twitter.scalding._ import cascading.pipe.joiner._ -import org.specs._ +import org.scalatest.{ Matchers, WordSpec } import com.twitter.algebird.Group object TUtil { @@ -428,8 +428,7 @@ class RowNormalize(args: Args) extends Job(args) { row1.L1Normalize.write(Tsv("rowLOneNorm")) } -class MatrixTest extends Specification { - noDetailedDiffs() // For scala 2.9 +class MatrixTest extends WordSpec with Matchers { import Dsl._ def toSparseMat[Row, Col, V](iter: Iterable[(Row, Col, V)]): Map[(Row, Col), V] = { @@ -441,12 +440,11 @@ class MatrixTest extends Specification { "A MatrixProd job" should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.MatrixProd") + JobTest(new MatrixProd(_)) .source(Tsv("mat1", ('x1, 'y1, 'v1)), List((1, 1, 1.0), (2, 2, 3.0), (1, 2, 4.0))) .sink[(Int, Int, Double)](Tsv("product")) { ob => "correctly compute products" in { - val pMap = toSparseMat(ob) - pMap must be_==(Map((1, 1) -> 17.0, (1, 2) -> 12.0, (2, 1) -> 12.0, (2, 2) -> 9.0)) + toSparseMat(ob) shouldBe Map((1, 1) -> 17.0, (1, 2) -> 12.0, (2, 1) -> 12.0, (2, 2) -> 9.0) } } .run @@ -456,12 +454,11 @@ class MatrixTest extends Specification { "A MatrixBlockProd job" should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.MatrixBlockProd") + JobTest(new MatrixBlockProd(_)) .source(Tsv("mat1", ('x1, 'y1, 'v1)), List(("alpha1", 1, 1.0), ("alpha1", 2, 2.0), ("beta1", 1, 5.0), ("beta1", 2, 6.0), ("alpha2", 1, 3.0), ("alpha2", 2, 4.0), ("beta2", 1, 7.0), ("beta2", 2, 8.0))) .sink[(String, String, Double)](Tsv("product")) { ob => "correctly compute block products" in { - val pMap = toSparseMat(ob) - pMap must be_==(Map( + toSparseMat(ob) shouldBe Map( ("alpha1", "alpha1") -> 5.0, ("alpha1", "alpha2") -> 11.0, ("alpha2", "alpha1") -> 11.0, @@ -469,7 +466,7 @@ class MatrixTest extends Specification { ("beta1", "beta1") -> 61.0, ("beta1", "beta2") -> 83.0, ("beta2", "beta1") -> 83.0, - ("beta2", "beta2") -> 113.0)) + ("beta2", "beta2") -> 113.0) } } .run @@ -479,13 +476,12 @@ class MatrixTest extends Specification { "A MatrixSum job" should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.MatrixSum") + JobTest(new MatrixSum(_)) .source(Tsv("mat1", ('x1, 'y1, 'v1)), List((1, 1, 1.0), (2, 2, 3.0), (1, 2, 4.0))) .source(Tsv("mat2", ('x2, 'y2, 'v2)), List((1, 3, 3.0), (2, 1, 8.0), (1, 2, 4.0))) .sink[(Int, Int, Double)](Tsv("sum")) { ob => "correctly compute sums" in { - val pMap = toSparseMat(ob) - pMap must be_==(Map((1, 1) -> 1.0, (1, 2) -> 8.0, (1, 3) -> 3.0, (2, 1) -> 8.0, (2, 2) -> 3.0)) + toSparseMat(ob) shouldBe Map((1, 1) -> 1.0, (1, 2) -> 8.0, (1, 3) -> 3.0, (2, 1) -> 8.0, (2, 2) -> 3.0) } } .run @@ -499,8 +495,7 @@ class MatrixTest extends Specification { .source(Tsv("mat1", ('x1, 'y1, 'v1)), List((1, 1, (1.0, 3.0, 5.0)), (2, 2, (3.0, 2.0, 1.0)), (1, 2, (4.0, 5.0, 2.0)))) .sink[(Int, Int, (Double, Double, Double))](Tsv("sum")) { ob => "correctly compute sums" in { - val pMap = toSparseMat(ob) - pMap must be_==(Map((1, 1) -> (2.0, 6.0, 10.0), (2, 2) -> (6.0, 4.0, 2.0), (1, 2) -> (8.0, 10.0, 4.0))) + toSparseMat(ob) shouldBe Map((1, 1) -> (2.0, 6.0, 10.0), (2, 2) -> (6.0, 4.0, 2.0), (1, 2) -> (8.0, 10.0, 4.0)) } } .run @@ -510,7 +505,7 @@ class MatrixTest extends Specification { "A Matrix Randwalk job" should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.Randwalk") + JobTest(new Randwalk(_)) /* * 1.0 4.0 * 0.0 3.0 @@ -531,7 +526,7 @@ class MatrixTest extends Specification { grp.minus(pMap, exact) .mapValues { x => x * x } .map { _._2 } - .sum must be_<(0.0001) + .sum should be < 0.0001 } } .run @@ -540,12 +535,11 @@ class MatrixTest extends Specification { } "A Matrix Cosine job" should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.Cosine") + JobTest(new Cosine(_)) .source(Tsv("mat1", ('x1, 'y1, 'v1)), List((1, 1, 1.0), (2, 2, 3.0), (1, 2, 4.0))) .sink[(Int, Int, Double)](Tsv("cosine")) { ob => "correctly compute cosine similarity" in { - val pMap = toSparseMat(ob) - pMap must be_==(Map((1, 1) -> 1.0, (1, 2) -> 0.9701425001453319, (2, 1) -> 0.9701425001453319, (2, 2) -> 1.0)) + toSparseMat(ob) shouldBe Map((1, 1) -> 1.0, (1, 2) -> 0.9701425001453319, (2, 1) -> 0.9701425001453319, (2, 2) -> 1.0) } } .run @@ -554,12 +548,11 @@ class MatrixTest extends Specification { } "A Matrix Covariance job" should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.Covariance") + JobTest(new Covariance(_)) .source(Tsv("mat1", ('x1, 'y1, 'v1)), List((1, 1, 1.0), (2, 2, 3.0), (1, 2, 4.0))) .sink[(Int, Int, Double)](Tsv("cov")) { ob => "correctly compute matrix covariance" in { - val pMap = toSparseMat(ob) - pMap must be_==(Map((1, 1) -> 0.25, (1, 2) -> -0.25, (2, 1) -> -0.25, (2, 2) -> 0.25)) + toSparseMat(ob) shouldBe Map((1, 1) -> 0.25, (1, 2) -> -0.25, (2, 1) -> -0.25, (2, 2) -> 0.25) } } .run @@ -568,11 +561,11 @@ class MatrixTest extends Specification { } "A Matrix VctProd job" should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.VctProd") + JobTest(new VctProd(_)) .source(Tsv("mat1", ('x1, 'y1, 'v1)), List((1, 1, 1.0), (2, 2, 3.0), (1, 2, 4.0))) .sink[Double](Tsv("vctProd")) { ob => "correctly compute vector inner products" in { - ob(0) must be_==(17.0) + ob(0) shouldBe 17.0 } } .run @@ -581,12 +574,11 @@ class MatrixTest extends Specification { } "A Matrix VctDiv job" should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.VctDiv") + JobTest(new VctDiv(_)) .source(Tsv("mat1", ('x1, 'y1, 'v1)), List((1, 1, 1.0), (2, 2, 3.0), (1, 2, 4.0))) .sink[(Int, Double)](Tsv("vctDiv")) { ob => "correctly compute vector element-wise division" in { - val pMap = oneDtoSparseMat(ob) - pMap must be_==(Map((2, 2) -> 1.3333333333333333)) + oneDtoSparseMat(ob) shouldBe Map((2, 2) -> 1.3333333333333333) } } .run @@ -595,36 +587,36 @@ class MatrixTest extends Specification { } "A Matrix ScalarOps job" should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.ScalarOps") + JobTest(new ScalarOps(_)) .source(Tsv("mat1", ('x1, 'y1, 'v1)), List((1, 1, 1.0), (2, 2, 3.0), (1, 2, 4.0))) .sink[(Int, Int, Double)](Tsv("times3")) { ob => "correctly compute M * 3" in { - toSparseMat(ob) must be_==(Map((1, 1) -> 3.0, (2, 2) -> 9.0, (1, 2) -> 12.0)) + toSparseMat(ob) shouldBe Map((1, 1) -> 3.0, (2, 2) -> 9.0, (1, 2) -> 12.0) } } .sink[(Int, Int, Double)](Tsv("3times")) { ob => "correctly compute 3 * M" in { - toSparseMat(ob) must be_==(Map((1, 1) -> 3.0, (2, 2) -> 9.0, (1, 2) -> 12.0)) + toSparseMat(ob) shouldBe Map((1, 1) -> 3.0, (2, 2) -> 9.0, (1, 2) -> 12.0) } } .sink[(Int, Int, Double)](Tsv("div3")) { ob => "correctly compute M / 3" in { - toSparseMat(ob) must be_==(Map((1, 1) -> (1.0 / 3.0), (2, 2) -> (3.0 / 3.0), (1, 2) -> (4.0 / 3.0))) + toSparseMat(ob) shouldBe Map((1, 1) -> (1.0 / 3.0), (2, 2) -> (3.0 / 3.0), (1, 2) -> (4.0 / 3.0)) } } .sink[(Int, Int, Double)](Tsv("timestrace")) { ob => "correctly compute M * Tr(M)" in { - toSparseMat(ob) must be_==(Map((1, 1) -> 4.0, (2, 2) -> 12.0, (1, 2) -> 16.0)) + toSparseMat(ob) shouldBe Map((1, 1) -> 4.0, (2, 2) -> 12.0, (1, 2) -> 16.0) } } .sink[(Int, Int, Double)](Tsv("tracetimes")) { ob => "correctly compute Tr(M) * M" in { - toSparseMat(ob) must be_==(Map((1, 1) -> 4.0, (2, 2) -> 12.0, (1, 2) -> 16.0)) + toSparseMat(ob) shouldBe Map((1, 1) -> 4.0, (2, 2) -> 12.0, (1, 2) -> 16.0) } } .sink[(Int, Int, Double)](Tsv("divtrace")) { ob => "correctly compute M / Tr(M)" in { - toSparseMat(ob) must be_==(Map((1, 1) -> (1.0 / 4.0), (2, 2) -> (3.0 / 4.0), (1, 2) -> (4.0 / 4.0))) + toSparseMat(ob) shouldBe Map((1, 1) -> (1.0 / 4.0), (2, 2) -> (3.0 / 4.0), (1, 2) -> (4.0 / 4.0)) } } .run @@ -640,30 +632,27 @@ class MatrixTest extends Specification { .source(Tsv("mat1", ('x1, 'y1, 'v1)), List((1, 1, 1.0), (2, 2, 3.0), (1, 2, 4.0))) .sink[(Int, Int, Double)](Tsv("diag-mat")) { ob => "correctly compute diag * matrix" in { - val pMap = toSparseMat(ob) - pMap must be_==(Map((1, 1) -> 1.0, (1, 2) -> 4.0, (2, 2) -> 9.0)) + toSparseMat(ob) shouldBe Map((1, 1) -> 1.0, (1, 2) -> 4.0, (2, 2) -> 9.0) } } .sink[(Int, Double)](Tsv("diag-diag")) { ob => "correctly compute diag * diag" in { - val pMap = oneDtoSparseMat(ob) - pMap must be_==(Map((1, 1) -> 1.0, (2, 2) -> 9.0)) + oneDtoSparseMat(ob) shouldBe Map((1, 1) -> 1.0, (2, 2) -> 9.0) } } .sink[(Int, Int, Double)](Tsv("mat-diag")) { ob => "correctly compute matrix * diag" in { - val pMap = toSparseMat(ob) - pMap must be_==(Map((1, 1) -> 1.0, (1, 2) -> 12.0, (2, 2) -> 9.0)) + toSparseMat(ob) shouldBe Map((1, 1) -> 1.0, (1, 2) -> 12.0, (2, 2) -> 9.0) } } .sink[(Int, Double)](Tsv("diag-col")) { ob => "correctly compute diag * col" in { - ob.toMap must be_==(Map(1 -> 1.0)) + ob.toMap shouldBe Map(1 -> 1.0) } } .sink[(Int, Double)](Tsv("row-diag")) { ob => "correctly compute row * diag" in { - ob.toMap must be_==(Map(1 -> 1.0, 2 -> 12.0)) + ob.toMap shouldBe Map(1 -> 1.0, 2 -> 12.0) } } .run @@ -684,12 +673,12 @@ class MatrixTest extends Specification { .source(TypedTsv[(Int, Double)]("col"), List((0, 1.0), (1, 2.0), (2, 4.0))) .sink[(Int, Double)](Tsv("prop-col")) { ob => "correctly propagate columns" in { - ob.toMap must be_==(Map(0 -> 6.0, 1 -> 4.0, 2 -> 1.0)) + ob.toMap shouldBe Map(0 -> 6.0, 1 -> 4.0, 2 -> 1.0) } } .sink[(Int, Double)](Tsv("prop-row")) { ob => "correctly propagate rows" in { - ob.toMap must be_==(Map(0 -> 4.0, 1 -> 1.0, 2 -> 3.0)) + ob.toMap shouldBe Map(0 -> 4.0, 1 -> 1.0, 2 -> 3.0) } } .run @@ -703,12 +692,12 @@ class MatrixTest extends Specification { .source(TypedTsv[(Int, Double)]("row"), List((0, 1.0), (1, 2.0), (2, 4.0))) .sink[(Int, Double)](Tsv("first")) { ob => "correctly mapWithIndex on Row" in { - ob.toMap must be_==(Map(0 -> 1.0)) + ob.toMap shouldBe Map(0 -> 1.0) } } .sink[(Int, Int, Int)](Tsv("diag")) { ob => "correctly mapWithIndex on Matrix" in { - toSparseMat(ob) must be_==(Map((1, 1) -> 3)) + toSparseMat(ob) shouldBe Map((1, 1) -> 3) } } .run @@ -717,12 +706,11 @@ class MatrixTest extends Specification { "A Matrix RowMatProd job" should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.RowMatProd") + JobTest(new RowMatProd(_)) .source(Tsv("mat1", ('x1, 'y1, 'v1)), List((1, 1, 1.0), (2, 2, 3.0), (1, 2, 4.0))) .sink[(Int, Double)](Tsv("rowMatPrd")) { ob => "correctly compute a new row vector" in { - val pMap = oneDtoSparseMat(ob) - pMap must be_==(Map((1, 1) -> 1.0, (2, 2) -> 16.0)) + oneDtoSparseMat(ob) shouldBe Map((1, 1) -> 1.0, (2, 2) -> 16.0) } } .run @@ -732,12 +720,11 @@ class MatrixTest extends Specification { "A Matrix MatColProd job" should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.MatColProd") + JobTest(new MatColProd(_)) .source(Tsv("mat1", ('x1, 'y1, 'v1)), List((1, 1, 1.0), (2, 2, 3.0), (1, 2, 4.0))) .sink[(Int, Double)](Tsv("matColPrd")) { ob => "correctly compute a new column vector" in { - val pMap = oneDtoSparseMat(ob) - pMap must be_==(Map((1, 1) -> 1.0)) + oneDtoSparseMat(ob) shouldBe Map((1, 1) -> 1.0) } } .run @@ -747,12 +734,11 @@ class MatrixTest extends Specification { "A Matrix RowRowDiff job" should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.RowRowDiff") + JobTest(new RowRowDiff(_)) .source(Tsv("mat1", ('x1, 'y1, 'v1)), List((1, 1, 1.0), (2, 2, 3.0), (1, 2, 4.0))) .sink[(Int, Double)](Tsv("rowRowDiff")) { ob => "correctly subtract row vectors" in { - val pMap = oneDtoSparseMat(ob) - pMap must be_==(Map((1, 1) -> 1.0, (2, 2) -> 1.0)) + oneDtoSparseMat(ob) shouldBe Map((1, 1) -> 1.0, (2, 2) -> 1.0) } } .run @@ -762,12 +748,11 @@ class MatrixTest extends Specification { "A Matrix VctOuterProd job" should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.VctOuterProd") + JobTest(new VctOuterProd(_)) .source(Tsv("mat1", ('x1, 'y1, 'v1)), List((1, 1, 1.0), (2, 2, 3.0), (1, 2, 4.0))) .sink[(Int, Int, Double)](Tsv("outerProd")) { ob => "correctly compute the outer product of a column and row vector" in { - val pMap = toSparseMat(ob) - pMap must be_==(Map((1, 1) -> 1.0, (1, 2) -> 4.0, (2, 1) -> 4.0, (2, 2) -> 16.0)) + toSparseMat(ob) shouldBe Map((1, 1) -> 1.0, (1, 2) -> 4.0, (2, 1) -> 4.0, (2, 2) -> 16.0) } } .run @@ -777,12 +762,11 @@ class MatrixTest extends Specification { "A Matrix RowRowSum job" should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.RowRowSum") + JobTest(new RowRowSum(_)) .source(Tsv("mat1", ('x1, 'y1, 'v1)), List((1, 1, 1.0), (2, 2, 3.0), (1, 2, 4.0))) .sink[(Int, Double)](Tsv("rowRowSum")) { ob => "correctly add row vectors" in { - val pMap = oneDtoSparseMat(ob) - pMap must be_==(Map((1, 1) -> 2.0, (2, 2) -> 8.0)) + oneDtoSparseMat(ob) shouldBe Map((1, 1) -> 2.0, (2, 2) -> 8.0) } } .run @@ -792,12 +776,11 @@ class MatrixTest extends Specification { "A Matrix RowRowHad job" should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.RowRowHad") + JobTest(new RowRowHad(_)) .source(Tsv("mat1", ('x1, 'y1, 'v1)), List((1, 1, 1.0), (2, 2, 3.0), (1, 2, 4.0))) .sink[(Int, Double)](Tsv("rowRowHad")) { ob => "correctly compute a Hadamard product of row vectors" in { - val pMap = oneDtoSparseMat(ob) - pMap must be_==(Map((1, 1) -> 1.0, (2, 2) -> 16.0)) + oneDtoSparseMat(ob) shouldBe Map((1, 1) -> 1.0, (2, 2) -> 16.0) } } .run @@ -807,19 +790,17 @@ class MatrixTest extends Specification { "A FilterMatrix job" should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.FilterMatrix") + JobTest(new FilterMatrix(_)) .source(Tsv("mat1", ('x, 'y, 'v)), List((1, 1, 1.0), (2, 2, 3.0), (1, 2, 4.0), (2, 1, 2.0))) .source(Tsv("mat2", ('x, 'y, 'v)), List((1, 1, 5.0), (2, 2, 9.0))) .sink[(Int, Int, Double)](Tsv("removeMatrix")) { ob => "correctly remove elements" in { - val pMap = toSparseMat(ob) - pMap must be_==(Map((1, 2) -> 4.0, (2, 1) -> 2.0)) + toSparseMat(ob) shouldBe Map((1, 2) -> 4.0, (2, 1) -> 2.0) } } .sink[(Int, Int, Double)](Tsv("keepMatrix")) { ob => "correctly keep elements" in { - val pMap = toSparseMat(ob) - pMap must be_==(Map((1, 1) -> 1.0, (2, 2) -> 3.0)) + toSparseMat(ob) shouldBe Map((1, 1) -> 1.0, (2, 2) -> 3.0) } } .run @@ -829,19 +810,17 @@ class MatrixTest extends Specification { "A KeepRowsCols job" should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.KeepRowsCols") + JobTest(new KeepRowsCols(_)) .source(Tsv("mat1", ('x, 'y, 'v)), List((1, 1, 1.0), (2, 2, 3.0), (1, 2, 4.0), (2, 1, 2.0))) .source(Tsv("col1", ('x, 'v)), List((1, 5.0))) .sink[(Int, Int, Double)](Tsv("keepRows")) { ob => "correctly keep row vectors" in { - val pMap = toSparseMat(ob) - pMap must be_==(Map((1, 2) -> 4.0, (1, 1) -> 1.0)) + toSparseMat(ob) shouldBe Map((1, 2) -> 4.0, (1, 1) -> 1.0) } } .sink[(Int, Int, Double)](Tsv("keepCols")) { ob => "correctly keep col vectors" in { - val pMap = toSparseMat(ob) - pMap must be_==(Map((2, 1) -> 2.0, (1, 1) -> 1.0)) + toSparseMat(ob) shouldBe Map((2, 1) -> 2.0, (1, 1) -> 1.0) } } .run @@ -851,19 +830,17 @@ class MatrixTest extends Specification { "A RemoveRowsCols job" should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.RemoveRowsCols") + JobTest(new RemoveRowsCols(_)) .source(Tsv("mat1", ('x, 'y, 'v)), List((1, 1, 1.0), (2, 2, 3.0), (1, 2, 4.0), (2, 1, 2.0))) .source(Tsv("col1", ('x, 'v)), List((1, 5.0))) .sink[(Int, Int, Double)](Tsv("removeRows")) { ob => "correctly keep row vectors" in { - val pMap = toSparseMat(ob) - pMap must be_==(Map((2, 2) -> 3.0, (2, 1) -> 2.0)) + toSparseMat(ob) shouldBe Map((2, 2) -> 3.0, (2, 1) -> 2.0) } } .sink[(Int, Int, Double)](Tsv("removeCols")) { ob => "correctly keep col vectors" in { - val pMap = toSparseMat(ob) - pMap must be_==(Map((2, 2) -> 3.0, (1, 2) -> 4.0)) + toSparseMat(ob) shouldBe Map((2, 2) -> 3.0, (1, 2) -> 4.0) } } .run @@ -873,20 +850,21 @@ class MatrixTest extends Specification { "A Scalar Row Right job" should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.ScalarRowRight") + var idx = 0 + JobTest(new ScalarRowRight(_)) .source(Tsv("sca1", ('v)), List(3.0)) .source(Tsv("row1", ('x, 'v)), List((1, 1.0), (2, 2.0), (3, 6.0))) .sink[(Int, Double)](Tsv("scalarRowRight")) { ob => - "correctly compute a new row vector" in { - val pMap = ob.toMap - pMap must be_==(Map(1 -> 3.0, 2 -> 6.0, 3 -> 18.0)) + s"$idx: correctly compute a new row vector" in { + ob.toMap shouldBe Map(1 -> 3.0, 2 -> 6.0, 3 -> 18.0) } + idx += 1 } .sink[(Int, Double)](Tsv("scalarObjRowRight")) { ob => - "correctly compute a new row vector" in { - val pMap = ob.toMap - pMap must be_==(Map(1 -> 3.0, 2 -> 6.0, 3 -> 18.0)) + s"$idx: correctly compute a new row vector" in { + ob.toMap shouldBe Map(1 -> 3.0, 2 -> 6.0, 3 -> 18.0) } + idx += 1 } .run .finish @@ -895,20 +873,21 @@ class MatrixTest extends Specification { "A Scalar Row Left job" should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.ScalarRowLeft") + var idx = 0 + JobTest(new ScalarRowLeft(_)) .source(Tsv("sca1", ('v)), List(3.0)) .source(Tsv("row1", ('x, 'v)), List((1, 1.0), (2, 2.0), (3, 6.0))) .sink[(Int, Double)](Tsv("scalarRowLeft")) { ob => - "correctly compute a new row vector" in { - val pMap = ob.toMap - pMap must be_==(Map(1 -> 3.0, 2 -> 6.0, 3 -> 18.0)) + s"$idx: correctly compute a new row vector" in { + ob.toMap shouldBe Map(1 -> 3.0, 2 -> 6.0, 3 -> 18.0) } + idx += 1 } .sink[(Int, Double)](Tsv("scalarObjRowLeft")) { ob => - "correctly compute a new row vector" in { - val pMap = ob.toMap - pMap must be_==(Map(1 -> 3.0, 2 -> 6.0, 3 -> 18.0)) + s"$idx: correctly compute a new row vector" in { + ob.toMap shouldBe Map(1 -> 3.0, 2 -> 6.0, 3 -> 18.0) } + idx += 1 } .run .finish @@ -917,20 +896,21 @@ class MatrixTest extends Specification { "A Scalar Col Right job" should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.ScalarColRight") + var idx = 0 + JobTest(new ScalarColRight(_)) .source(Tsv("sca1", ('v)), List(3.0)) .source(Tsv("col1", ('x, 'v)), List((1, 1.0), (2, 2.0), (3, 6.0))) .sink[(Int, Double)](Tsv("scalarColRight")) { ob => - "correctly compute a new col vector" in { - val pMap = ob.toMap - pMap must be_==(Map(1 -> 3.0, 2 -> 6.0, 3 -> 18.0)) + s"$idx: correctly compute a new col vector" in { + ob.toMap shouldBe Map(1 -> 3.0, 2 -> 6.0, 3 -> 18.0) } + idx += 1 } .sink[(Int, Double)](Tsv("scalarObjColRight")) { ob => - "correctly compute a new col vector" in { - val pMap = ob.toMap - pMap must be_==(Map(1 -> 3.0, 2 -> 6.0, 3 -> 18.0)) + s"$idx: correctly compute a new col vector" in { + ob.toMap shouldBe Map(1 -> 3.0, 2 -> 6.0, 3 -> 18.0) } + idx += 1 } .run .finish @@ -939,20 +919,21 @@ class MatrixTest extends Specification { "A Scalar Col Left job" should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.ScalarColLeft") + var idx = 0 + JobTest(new ScalarColLeft(_)) .source(Tsv("sca1", ('v)), List(3.0)) .source(Tsv("col1", ('x, 'v)), List((1, 1.0), (2, 2.0), (3, 6.0))) .sink[(Int, Double)](Tsv("scalarColLeft")) { ob => - "correctly compute a new col vector" in { - val pMap = ob.toMap - pMap must be_==(Map(1 -> 3.0, 2 -> 6.0, 3 -> 18.0)) + s"$idx: correctly compute a new col vector" in { + ob.toMap shouldBe Map(1 -> 3.0, 2 -> 6.0, 3 -> 18.0) } + idx += 1 } .sink[(Int, Double)](Tsv("scalarObjColLeft")) { ob => - "correctly compute a new col vector" in { - val pMap = ob.toMap - pMap must be_==(Map(1 -> 3.0, 2 -> 6.0, 3 -> 18.0)) + s"$idx: correctly compute a new col vector" in { + ob.toMap shouldBe Map(1 -> 3.0, 2 -> 6.0, 3 -> 18.0) } + idx += 1 } .run .finish @@ -961,20 +942,21 @@ class MatrixTest extends Specification { "A Scalar Diag Right job" should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.ScalarDiagRight") + var idx = 0 + JobTest(new ScalarDiagRight(_)) .source(Tsv("sca1", ('v)), List(3.0)) .source(Tsv("diag1", ('x, 'v)), List((1, 1.0), (2, 2.0), (3, 6.0))) .sink[(Int, Double)](Tsv("scalarDiagRight")) { ob => - "correctly compute a new diag matrix" in { - val pMap = ob.toMap - pMap must be_==(Map(1 -> 3.0, 2 -> 6.0, 3 -> 18.0)) + s"$idx: correctly compute a new diag matrix" in { + ob.toMap shouldBe Map(1 -> 3.0, 2 -> 6.0, 3 -> 18.0) } + idx += 1 } .sink[(Int, Double)](Tsv("scalarObjDiagRight")) { ob => - "correctly compute a new diag matrix" in { - val pMap = ob.toMap - pMap must be_==(Map(1 -> 3.0, 2 -> 6.0, 3 -> 18.0)) + s"$idx: correctly compute a new diag matrix" in { + ob.toMap shouldBe Map(1 -> 3.0, 2 -> 6.0, 3 -> 18.0) } + idx += 1 } .run .finish @@ -983,20 +965,21 @@ class MatrixTest extends Specification { "A Scalar Diag Left job" should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.ScalarDiagLeft") + var idx = 0 + JobTest(new ScalarDiagLeft(_)) .source(Tsv("sca1", ('v)), List(3.0)) .source(Tsv("diag1", ('x, 'v)), List((1, 1.0), (2, 2.0), (3, 6.0))) .sink[(Int, Double)](Tsv("scalarDiagLeft")) { ob => - "correctly compute a new diag matrix" in { - val pMap = ob.toMap - pMap must be_==(Map(1 -> 3.0, 2 -> 6.0, 3 -> 18.0)) + s"$idx: correctly compute a new diag matrix" in { + ob.toMap shouldBe Map(1 -> 3.0, 2 -> 6.0, 3 -> 18.0) } + idx += 1 } .sink[(Int, Double)](Tsv("scalarObjDiagLeft")) { ob => - "correctly compute a new diag matrix" in { - val pMap = ob.toMap - pMap must be_==(Map(1 -> 3.0, 2 -> 6.0, 3 -> 18.0)) + s"$idx: correctly compute a new diag matrix" in { + ob.toMap shouldBe Map(1 -> 3.0, 2 -> 6.0, 3 -> 18.0) } + idx += 1 } .run .finish @@ -1005,19 +988,20 @@ class MatrixTest extends Specification { "A Col Normalizing job" should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.ColNormalize") + var idx = 0 + JobTest(new ColNormalize(_)) .source(Tsv("col1", ('x, 'v)), List((1, 1.0), (2, -2.0), (3, 6.0))) .sink[(Int, Double)](Tsv("colLZeroNorm")) { ob => - "correctly compute a new col vector" in { - val pMap = ob.toMap - pMap must be_==(Map(1 -> (1.0 / 3.0), 2 -> (-2.0 / 3.0), 3 -> (6.0 / 3.0))) + s"$idx: correctly compute a new col vector" in { + ob.toMap shouldBe Map(1 -> (1.0 / 3.0), 2 -> (-2.0 / 3.0), 3 -> (6.0 / 3.0)) } + idx += 1 } .sink[(Int, Double)](Tsv("colLOneNorm")) { ob => - "correctly compute a new col vector" in { - val pMap = ob.toMap - pMap must be_==(Map(1 -> (1.0 / 9.0), 2 -> (-2.0 / 9.0), 3 -> (6.0 / 9.0))) + s"$idx: correctly compute a new col vector" in { + ob.toMap shouldBe Map(1 -> (1.0 / 9.0), 2 -> (-2.0 / 9.0), 3 -> (6.0 / 9.0)) } + idx += 1 } .run .finish @@ -1028,30 +1012,30 @@ class MatrixTest extends Specification { TUtil.printStack { "correctly compute the size of the diagonal matrix" in { val col = new ColDiagonal(Mode.putMode(new Test(Map.empty), new Args(Map.empty))) - col.sizeHintTotal must be_==(100L) + col.sizeHintTotal shouldBe 100L } } } "A Row Normalizing job" should { TUtil.printStack { - JobTest("com.twitter.scalding.mathematics.RowNormalize") + var idx = 0 + JobTest(new RowNormalize(_)) .source(Tsv("row1", ('x, 'v)), List((1, 1.0), (2, -2.0), (3, 6.0))) .sink[(Int, Double)](Tsv("rowLZeroNorm")) { ob => - "correctly compute a new row vector" in { - val pMap = ob.toMap - pMap must be_==(Map(1 -> (1.0 / 3.0), 2 -> (-2.0 / 3.0), 3 -> (6.0 / 3.0))) + s"$idx: correctly compute a new row vector" in { + ob.toMap shouldBe Map(1 -> (1.0 / 3.0), 2 -> (-2.0 / 3.0), 3 -> (6.0 / 3.0)) } + idx += 1 } .sink[(Int, Double)](Tsv("rowLOneNorm")) { ob => - "correctly compute a new row vector" in { - val pMap = ob.toMap - pMap must be_==(Map(1 -> (1.0 / 9.0), 2 -> (-2.0 / 9.0), 3 -> (6.0 / 9.0))) + s"$idx: correctly compute a new row vector" in { + ob.toMap shouldBe Map(1 -> (1.0 / 9.0), 2 -> (-2.0 / 9.0), 3 -> (6.0 / 9.0)) } + idx += 1 } .run .finish } } - } 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 ab775fa7df..37e0e3f924 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 @@ -16,7 +16,6 @@ limitations under the License. package com.twitter.scalding.mathematics import com.twitter.scalding._ -import org.specs._ import org.scalacheck.Arbitrary import org.scalacheck.Arbitrary.arbitrary diff --git a/scalding-core/src/test/scala/com/twitter/scalding/mathematics/TypedSimilarityTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/mathematics/TypedSimilarityTest.scala index 4263cd9994..9238a8c801 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/mathematics/TypedSimilarityTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/mathematics/TypedSimilarityTest.scala @@ -21,7 +21,7 @@ import com.twitter.algebird.Group import TDsl._ -import org.specs._ +import org.scalatest.{ Matchers, WordSpec } import GraphOperations._ @@ -55,7 +55,7 @@ class TypedDimsumCosineSimJob(args: Args) extends Job(args) { .write(TypedTsv[(Int, Int, Double)]("out")) } -class TypedSimilarityTest extends Specification { +class TypedSimilarityTest extends WordSpec with Matchers { val nodes = 50 val rand = new java.util.Random(1) val edges = (0 to nodes).flatMap { n => @@ -107,7 +107,7 @@ class TypedSimilarityTest extends Specification { .sink[(Int, Int, Double)](TypedTsv[(Int, Int, Double)]("out")) { ob => val result = ob.map { case (n1, n2, d) => ((n1 -> n2) -> d) }.toMap val error = Group.minus(result, cosineOf(edges)) - dot(error, error) must beLessThan(0.001) + dot(error, error) should be < 0.001 } .run .finish @@ -118,7 +118,7 @@ class TypedSimilarityTest extends Specification { .sink[(Int, Int, Double)](TypedTsv[(Int, Int, Double)]("out")) { ob => val result = ob.map { case (n1, n2, d) => ((n1 -> n2) -> d) }.toMap val error = Group.minus(result, weightedCosineOf(weightedEdges)) - dot(error, error) must beLessThan(0.01 * error.size) + dot(error, error) should be < (0.01 * error.size) } .run .finish diff --git a/scalding-core/src/test/scala/com/twitter/scalding/typed/BijectedSourceSinkTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/typed/BijectedSourceSinkTest.scala index 0e2fd6539f..19a76f897f 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/typed/BijectedSourceSinkTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/typed/BijectedSourceSinkTest.scala @@ -15,7 +15,7 @@ limitations under the License. */ package com.twitter.scalding.typed -import org.specs._ +import org.scalatest.{ Matchers, WordSpec } import com.twitter.scalding._ @@ -40,23 +40,23 @@ class MutatedSourceJob(args: Args) extends Job(args) { .write(BijectedSourceSink(TypedTsv[Long]("output"))) } -class MutatedSourceTest extends Specification { +class MutatedSourceTest extends WordSpec with Matchers { import Dsl._ "A MutatedSourceJob" should { "Not throw when using a converted source" in { JobTest(new MutatedSourceJob(_)) .source(TypedTsv[Long]("input0"), List(8L, 4123423431L, 12L)) - .sink[Long](TypedTsv[Long]("output")) { outBuf => + .typedSink(TypedTsv[Long]("output")) { outBuf => val unordered = outBuf.toSet // Size should be unchanged - unordered.size must be_==(3) + unordered should have size 3 // Simple case, 2*8L won't run into the packer logic - unordered(16L) must be_==(true) + unordered should contain (16L) // Big one that should be in both the high and low 4 bytes of the Long val big = 4123423431L val newBig = LongIntPacker.lr(LongIntPacker.l(big) * 2, LongIntPacker.r(big) * 2) - unordered(newBig) must be_==(true) + unordered should contain (newBig) } .run .runHadoop @@ -71,23 +71,23 @@ class ContraMappedAndThenSourceJob(args: Args) extends Job(args) { .write(TypedTsv[Long]("output").contraMap { case (l, r) => LongIntPacker.lr(l, r) }) } -class ContraMappedAndThenSourceTest extends Specification { +class ContraMappedAndThenSourceTest extends WordSpec with Matchers { import Dsl._ "A ContraMappedAndThenSourceJob" should { "Not throw when using a converted source" in { JobTest(new ContraMappedAndThenSourceJob(_)) .source(TypedTsv[Long]("input0"), List(8L, 4123423431L, 12L)) - .sink[Long](TypedTsv[Long]("output")) { outBuf => + .typedSink(TypedTsv[Long]("output")) { outBuf => val unordered = outBuf.toSet // Size should be unchanged - unordered.size must be_==(3) + unordered should have size 3 // Simple case, 2*8L won't run into the packer logic - unordered(16L) must be_==(true) + unordered should contain (16L) // Big one that should be in both the high and low 4 bytes of the Long val big = 4123423431L val newBig = LongIntPacker.lr(LongIntPacker.l(big) * 2, LongIntPacker.r(big) * 2) - unordered(newBig) must be_==(true) + unordered should contain (newBig) } .run .runHadoop diff --git a/scalding-core/src/test/scala/com/twitter/scalding/typed/ExecutionTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/typed/ExecutionTest.scala index c188fa6c06..1047a7bb97 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/typed/ExecutionTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/typed/ExecutionTest.scala @@ -15,7 +15,7 @@ limitations under the License. */ package com.twitter.scalding.typed -import org.specs._ +import org.scalatest.{ Matchers, WordSpec } import com.twitter.scalding._ import com.twitter.algebird.monad.Reader @@ -68,7 +68,7 @@ class WordCountEc(args: Args) extends ExecutionJob[Unit](args) { } } -class ExecutionTest extends Specification { +class ExecutionTest extends WordSpec with Matchers { "An Executor" should { "work synchonously" in { val (r, stats) = Execution.waitFor(Config.default, Local(false)) { implicit ec: ExecutionContext => @@ -80,8 +80,8 @@ class ExecutionTest extends Specification { { () => sink.readResults } } - stats.isSuccess must beTrue - r().toMap must be_==((0 to 100).groupBy(_ % 3).mapValues(_.sum).toMap) + stats.isSuccess shouldBe true + r().toMap shouldBe ((0 to 100).groupBy(_ % 3).mapValues(_.sum).toMap) } "work asynchonously" in { val (r, fstats) = Execution.run(Config.default, Local(false)) { implicit ec: ExecutionContext => @@ -93,21 +93,20 @@ class ExecutionTest extends Specification { { () => sink.readResults } } - Try(Await.result(fstats, Duration.Inf)).isSuccess must beTrue - r().toMap must be_==((0 to 100).groupBy(_ % 3).mapValues(_.sum).toMap) + Try(Await.result(fstats, Duration.Inf)).isSuccess shouldBe true + r().toMap shouldBe ((0 to 100).groupBy(_ % 3).mapValues(_.sum).toMap) } } "An Execution" should { "run" in { ExecutionTestJobs.wordCount2(TypedPipe.from(List("a b b c c c", "d d d d"))) - .waitFor(Config.default, Local(false)).get.toMap must be_==( - Map("a" -> 1L, "b" -> 2L, "c" -> 3L, "d" -> 4L)) + .waitFor(Config.default, Local(false)).get.toMap shouldBe Map("a" -> 1L, "b" -> 2L, "c" -> 3L, "d" -> 4L) } "run with zip" in { (ExecutionTestJobs.zipped(TypedPipe.from(0 until 100), TypedPipe.from(100 until 200)) .waitFor(Config.default, Local(false)).get match { case (it1, it2) => (it1.head, it2.head) - }) must be_==((0 until 100).sum, (100 until 200).sum) + }) shouldBe ((0 until 100).sum, (100 until 200).sum) } "merge fanouts without error" in { def unorderedEq[T](l: Iterable[T], r: Iterable[T]): Boolean = @@ -120,7 +119,7 @@ class ExecutionTest extends Specification { val input = (0 to 100).toList val result = ExecutionTestJobs.mergeFanout(input).waitFor(Config.default, Local(false)).get val cres = correct(input) - unorderedEq(cres, result.toList) must beTrue + unorderedEq(cres, result.toList) shouldBe true } } "Execution K-means" should { @@ -151,7 +150,7 @@ class ExecutionTest extends Specification { byCluster.foreach { case (clusterId, vs) => val id = vs.head._1 - vs.forall { case (thisId, _) => id must be_==(thisId) } + vs.foreach { case (thisId, _) => id shouldBe thisId } } } } @@ -159,24 +158,23 @@ class ExecutionTest extends Specification { val parser = new ExecutionApp { def job = Execution.from(()) } "parse hadoop args correctly" in { val conf = parser.config(Array("-Dmapred.reduce.tasks=100", "--local"))._1 - conf.get("mapred.reduce.tasks") must be_==(Some("100")) - conf.getArgs.boolean("local") must beTrue + conf.get("mapred.reduce.tasks") should contain ("100") + conf.getArgs.boolean("local") shouldBe true val (conf1, Hdfs(_, hconf)) = parser.config(Array("--test", "-Dmapred.reduce.tasks=110", "--hdfs")) - conf1.get("mapred.reduce.tasks") must be_==(Some("110")) - conf1.getArgs.boolean("test") must beTrue - hconf.get("mapred.reduce.tasks") must be_==("110") + conf1.get("mapred.reduce.tasks") should contain ("110") + conf1.getArgs.boolean("test") shouldBe true + hconf.get("mapred.reduce.tasks") shouldBe "110" } } "An ExecutionJob" should { "run correctly" in { - JobTest(new com.twitter.scalding.typed.WordCountEc(_)) + JobTest(new WordCountEc(_)) .arg("input", "in") .arg("output", "out") .source(TextLine("in"), List((0, "hello world"), (1, "goodbye world"))) - .sink[(String, Long)](TypedTsv[(String, Long)]("out")) { outBuf => - val results = outBuf.toMap - results must be_==(Map("hello" -> 1L, "world" -> 2L, "goodbye" -> 1L)) + .typedSink(TypedTsv[(String, Long)]("out")) { outBuf => + outBuf.toMap shouldBe Map("hello" -> 1L, "world" -> 2L, "goodbye" -> 1L) } .run .runHadoop @@ -199,7 +197,7 @@ class ExecutionTest extends Specification { */ val res = e3.zip(e2) res.waitFor(Config.default, Local(true)) - (first, second, third) must be_==((1, 1, 1)) + assert((first, second, third) == (1, 1, 1)) } } } diff --git a/scalding-core/src/test/scala/com/twitter/scalding/typed/MultiJoinTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/typed/MultiJoinTest.scala index 88b3ce9831..bcafb7fd68 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/typed/MultiJoinTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/typed/MultiJoinTest.scala @@ -15,12 +15,12 @@ limitations under the License. */ package com.twitter.scalding.typed -import org.specs._ +import org.scalatest.WordSpec import com.twitter.scalding._ import com.twitter.scalding.typed.FlattenGroup._ -class MultiJoinTest extends Specification { +class MultiJoinTest extends WordSpec { def addKeys[V](t: Seq[V]): Seq[(Int, V)] = t.iterator.zipWithIndex.map { case (v, k) => (k, v) }.toSeq diff --git a/scalding-core/src/test/scala/com/twitter/scalding/typed/PartitionedDelimitedSourceTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/typed/PartitionedDelimitedSourceTest.scala index 9592e32b56..ab8e6cf652 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/typed/PartitionedDelimitedSourceTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/typed/PartitionedDelimitedSourceTest.scala @@ -12,16 +12,16 @@ // See the License for the specific language governing permissions and // limitations under the License. -package com.twitter.scalding -package typed +package com.twitter.scalding.typed import java.io.File import scala.io.{ Source => ScalaSource } -import org.specs._ +import org.scalatest.{ Matchers, WordSpec } -import com.twitter.scalding.TDsl._ +import com.twitter.scalding._ +import TDsl._ object PartitionedDelimitedTestSources { val singlePartition = PartitionedCsv[String, (String, String)]("out", "%s") @@ -34,11 +34,9 @@ class PartitionedDelimitedWriteJob(args: Args) extends Job(args) { .write(singlePartition) } -class PartitionedDelimitedTest extends Specification { +class PartitionedDelimitedTest extends WordSpec with Matchers { import PartitionedDelimitedTestSources._ - noDetailedDiffs() - "PartitionedDelimited" should { "write out CSVs" in { val input = Seq(("A", "X", "1"), ("A", "Y", "2"), ("B", "Z", "3")) @@ -59,13 +57,13 @@ class PartitionedDelimitedTest extends Specification { val directory = new File(testMode.getWritePathFor(singlePartition)) - directory.listFiles().map({ _.getName() }).toSet mustEqual Set("A", "B") + directory.listFiles().map({ _.getName() }).toSet shouldBe Set("A", "B") val aSource = ScalaSource.fromFile(new File(directory, "A/part-00000-00000")) val bSource = ScalaSource.fromFile(new File(directory, "B/part-00000-00001")) - aSource.getLines.toList mustEqual Seq("X,1", "Y,2") - bSource.getLines.toList mustEqual Seq("Z,3") + aSource.getLines.toList shouldBe Seq("X,1", "Y,2") + bSource.getLines.toList shouldBe Seq("Z,3") } } } diff --git a/scalding-core/src/test/scala/com/twitter/scalding/typed/PartitionedTextLineTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/typed/PartitionedTextLineTest.scala index f9936065f5..b3897d7631 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/typed/PartitionedTextLineTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/typed/PartitionedTextLineTest.scala @@ -12,16 +12,17 @@ // See the License for the specific language governing permissions and // limitations under the License. -package com.twitter.scalding -package typed +package com.twitter.scalding.typed import java.io.File import scala.io.{ Source => ScalaSource } -import org.specs._ +import org.scalatest.{ Matchers, WordSpec } -import com.twitter.scalding.TDsl._ +import com.twitter.scalding._ + +import TDsl._ object PartitionedTextLineTestSources { val singlePartition = PartitionedTextLine[String]("out", "%s") @@ -40,11 +41,9 @@ class PartitionedTextLineMultipleWriteJob(args: Args) extends Job(args) { .write(multiplePartition) } -class PartitionedTextLineTest extends Specification { +class PartitionedTextLineTest extends WordSpec with Matchers { import PartitionedTextLineTestSources._ - noDetailedDiffs() - "PartitionedTextLine" should { "be able to split output by a single partition" in { val input = Seq(("A", "1"), ("A", "2"), ("B", "3")) @@ -66,13 +65,13 @@ class PartitionedTextLineTest extends Specification { val directory = new File(testMode.getWritePathFor(singlePartition)) println(directory) - directory.listFiles().map({ _.getName() }).toSet mustEqual Set("A", "B") + directory.listFiles().map({ _.getName() }).toSet shouldBe Set("A", "B") val aSource = ScalaSource.fromFile(new File(directory, "A/part-00000-00000")) val bSource = ScalaSource.fromFile(new File(directory, "B/part-00000-00001")) - aSource.getLines.toList mustEqual Seq("1", "2") - bSource.getLines.toList mustEqual Seq("3") + aSource.getLines.toList shouldBe Seq("1", "2") + bSource.getLines.toList shouldBe Seq("3") } "be able to split output by multiple partitions" in { val input = Seq(("A", "X", "1"), ("A", "Y", "2"), ("B", "Z", "3")) @@ -94,15 +93,15 @@ class PartitionedTextLineTest extends Specification { val directory = new File(testMode.getWritePathFor(multiplePartition)) println(directory) - directory.listFiles.flatMap(d => d.listFiles.map(d.getName + "/" + _.getName)).toSet mustEqual Set("A/X", "A/Y", "B/Z") + directory.listFiles.flatMap(d => d.listFiles.map(d.getName + "/" + _.getName)).toSet shouldBe Set("A/X", "A/Y", "B/Z") val axSource = ScalaSource.fromFile(new File(directory, "A/X/part-00000-00000")) val aySource = ScalaSource.fromFile(new File(directory, "A/Y/part-00000-00001")) val bzSource = ScalaSource.fromFile(new File(directory, "B/Z/part-00000-00002")) - axSource.getLines.toList mustEqual Seq("1") - aySource.getLines.toList mustEqual Seq("2") - bzSource.getLines.toList mustEqual Seq("3") + axSource.getLines.toList shouldBe Seq("1") + aySource.getLines.toList shouldBe Seq("2") + bzSource.getLines.toList shouldBe Seq("3") } } } diff --git a/scalding-date/src/main/scala/com/twitter/scalding/Globifier.scala b/scalding-date/src/main/scala/com/twitter/scalding/Globifier.scala index fa831aff4a..c3756a2966 100644 --- a/scalding-date/src/main/scala/com/twitter/scalding/Globifier.scala +++ b/scalding-date/src/main/scala/com/twitter/scalding/Globifier.scala @@ -58,7 +58,7 @@ class BaseGlobifier(dur: Duration, val sym: String, pattern: String, tz: TimeZon Nil } else if (child.isEmpty) { //There is only one block: - assert(sstr == estr, "Malformed heirarchy" + sstr + " != " + estr) + assert(sstr == estr, "Malformed hierarchy" + sstr + " != " + estr) List(sstr) } else { /* diff --git a/scalding-date/src/test/scala/com/twitter/scalding/CalendarOpsTest.scala b/scalding-date/src/test/scala/com/twitter/scalding/CalendarOpsTest.scala index bc5f46e61e..606a8a71f1 100644 --- a/scalding-date/src/test/scala/com/twitter/scalding/CalendarOpsTest.scala +++ b/scalding-date/src/test/scala/com/twitter/scalding/CalendarOpsTest.scala @@ -3,87 +3,85 @@ package com.twitter.scalding import java.text.SimpleDateFormat import java.util._ -import org.specs._ +import org.scalatest.WordSpec -class CalendarOpsTest extends Specification { - noDetailedDiffs() +class CalendarOpsTest extends WordSpec { + val cal = Calendar.getInstance() - val cal = Calendar.getInstance(); - - val dateParser = new SimpleDateFormat("MMM dd, yyyy", Locale.ENGLISH); - val dateTimeParser = new SimpleDateFormat("MMM dd, yyyy H:mm:ss.SSS", Locale.ENGLISH); + val dateParser = new SimpleDateFormat("MMM dd, yyyy", Locale.ENGLISH) + val dateTimeParser = new SimpleDateFormat("MMM dd, yyyy H:mm:ss.SSS", Locale.ENGLISH) "The CalendarOps truncate method" should { "not truncate if the specified field is milliseconds" in { cal.setTime(new Date(1384819200555L)) - cal.get(Calendar.MILLISECOND) must be equalTo 555 + assert(cal.get(Calendar.MILLISECOND) === 555) } "truncate to a year" in { - dateParser.parse("January 1, 2002") must be equalTo - CalendarOps.truncate(dateParser.parse("February 12, 2002 12:34:56.789"), Calendar.YEAR) + assert(dateParser.parse("January 1, 2002") === + CalendarOps.truncate(dateParser.parse("February 12, 2002 12:34:56.789"), Calendar.YEAR)) - dateParser.parse("January 1, 2001") must be equalTo - CalendarOps.truncate(dateParser.parse("November 18, 2001 1:23:11.321"), Calendar.YEAR) + assert(dateParser.parse("January 1, 2001") === + CalendarOps.truncate(dateParser.parse("November 18, 2001 1:23:11.321"), Calendar.YEAR)) } "truncate to a month" in { - dateParser.parse("February 1, 2002") must be equalTo - CalendarOps.truncate(dateParser.parse("February 12, 2002 12:34:56.789"), Calendar.MONTH) + assert(dateParser.parse("February 1, 2002") === + CalendarOps.truncate(dateParser.parse("February 12, 2002 12:34:56.789"), Calendar.MONTH)) - dateParser.parse("November 1, 2001") must be equalTo - CalendarOps.truncate(dateParser.parse("November 18, 2001 1:23:11.321"), Calendar.MONTH) + assert(dateParser.parse("November 1, 2001") === + CalendarOps.truncate(dateParser.parse("November 18, 2001 1:23:11.321"), Calendar.MONTH)) } "truncate to a date" in { - dateParser.parse("February 12, 2002") must be equalTo - CalendarOps.truncate(dateParser.parse("February 12, 2002 12:34:56.789"), Calendar.DATE) + assert(dateParser.parse("February 12, 2002") == + CalendarOps.truncate(dateParser.parse("February 12, 2002 12:34:56.789"), Calendar.DATE)) - dateParser.parse("November 18, 2001") must be equalTo - CalendarOps.truncate(dateParser.parse("November 18, 2001 1:23:11.321"), Calendar.DATE) + assert(dateParser.parse("November 18, 2001") === + CalendarOps.truncate(dateParser.parse("November 18, 2001 1:23:11.321"), Calendar.DATE)) } "truncate to a minute" in { - dateTimeParser.parse("February 12, 2002 12:34:00.000") must be equalTo - CalendarOps.truncate(dateTimeParser.parse("February 12, 2002 12:34:56.789"), Calendar.MINUTE) + assert(dateTimeParser.parse("February 12, 2002 12:34:00.000") === + CalendarOps.truncate(dateTimeParser.parse("February 12, 2002 12:34:56.789"), Calendar.MINUTE)) - dateTimeParser.parse("November 18, 2001 1:23:00.000") must be equalTo - CalendarOps.truncate(dateTimeParser.parse("November 18, 2001 1:23:11.321"), Calendar.MINUTE) + assert(dateTimeParser.parse("November 18, 2001 1:23:00.000") === + CalendarOps.truncate(dateTimeParser.parse("November 18, 2001 1:23:11.321"), Calendar.MINUTE)) } "truncate to a second" in { - dateTimeParser.parse("February 12, 2002 12:34:56.000") must be equalTo - CalendarOps.truncate(dateTimeParser.parse("February 12, 2002 12:34:56.789"), Calendar.SECOND) + assert(dateTimeParser.parse("February 12, 2002 12:34:56.000") === + CalendarOps.truncate(dateTimeParser.parse("February 12, 2002 12:34:56.789"), Calendar.SECOND)) - dateTimeParser.parse("November 18, 2001 1:23:11.000") must be equalTo - CalendarOps.truncate(dateTimeParser.parse("November 18, 2001 1:23:11.321"), Calendar.SECOND) + assert(dateTimeParser.parse("November 18, 2001 1:23:11.000") === + CalendarOps.truncate(dateTimeParser.parse("November 18, 2001 1:23:11.321"), Calendar.SECOND)) } "truncate to AM" in { - dateTimeParser.parse("February 3, 2002 00:00:00.000") must be equalTo - CalendarOps.truncate(dateTimeParser.parse("February 3, 2002 01:10:00.000"), Calendar.AM_PM) + assert(dateTimeParser.parse("February 3, 2002 00:00:00.000") === + CalendarOps.truncate(dateTimeParser.parse("February 3, 2002 01:10:00.000"), Calendar.AM_PM)) - dateTimeParser.parse("February 3, 2002 00:00:00.000") must be equalTo - CalendarOps.truncate(dateTimeParser.parse("February 3, 2002 11:10:00.000"), Calendar.AM_PM) + assert(dateTimeParser.parse("February 3, 2002 00:00:00.000") === + CalendarOps.truncate(dateTimeParser.parse("February 3, 2002 11:10:00.000"), Calendar.AM_PM)) } "truncate to PM" in { - dateTimeParser.parse("February 3, 2002 12:00:00.000") must be equalTo - CalendarOps.truncate(dateTimeParser.parse("February 3, 2002 13:10:00.000"), Calendar.AM_PM) + assert(dateTimeParser.parse("February 3, 2002 12:00:00.000") === + CalendarOps.truncate(dateTimeParser.parse("February 3, 2002 13:10:00.000"), Calendar.AM_PM)) - dateTimeParser.parse("February 3, 2002 12:00:00.000") must be equalTo - CalendarOps.truncate(dateTimeParser.parse("February 3, 2002 19:10:00.000"), Calendar.AM_PM) + assert(dateTimeParser.parse("February 3, 2002 12:00:00.000") === + CalendarOps.truncate(dateTimeParser.parse("February 3, 2002 19:10:00.000"), Calendar.AM_PM)) } "truncate respects DST" in { TimeZone.setDefault(TimeZone.getTimeZone("MET")) dateTimeParser.setTimeZone(TimeZone.getTimeZone("MET")) - dateTimeParser.parse("March 30, 2003 00:00:00.000") must be equalTo - CalendarOps.truncate(dateTimeParser.parse("March 30, 2003 05:30:45.000"), Calendar.DATE) + assert(dateTimeParser.parse("March 30, 2003 00:00:00.000") === + CalendarOps.truncate(dateTimeParser.parse("March 30, 2003 05:30:45.000"), Calendar.DATE)) - dateTimeParser.parse("October 26, 2003 00:00:00.000") must be equalTo - CalendarOps.truncate(dateTimeParser.parse("October 26, 2003 05:30:45.000"), Calendar.DATE) + assert(dateTimeParser.parse("October 26, 2003 00:00:00.000") === + CalendarOps.truncate(dateTimeParser.parse("October 26, 2003 05:30:45.000"), Calendar.DATE)) } } } 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 fa008cd1ba..543300d7df 100644 --- a/scalding-date/src/test/scala/com/twitter/scalding/DateTest.scala +++ b/scalding-date/src/test/scala/com/twitter/scalding/DateTest.scala @@ -15,11 +15,10 @@ limitations under the License. */ package com.twitter.scalding -import org.specs._ +import org.scalatest.WordSpec import java.util.Calendar -class DateTest extends Specification { - noDetailedDiffs() +class DateTest extends WordSpec { implicit val tz = DateOps.PACIFIC implicit def dateParser: DateParser = DateParser.default @@ -28,59 +27,59 @@ class DateTest extends Specification { "implicitly convert strings" in { val rd1: RichDate = "2011-10-20" val rd2: RichDate = "2011-10-20" - rd1 must be_==(rd2) + assert(rd1 === rd2) } "implicitly convert calendars" in { val rd1: RichDate = "2011-10-20" val cal = Calendar.getInstance(tz) cal.setTime(rd1.value) val rd2: RichDate = cal - rd1 must_== rd2 + assert(rd1 === rd2) } "deal with strings with spaces" in { val rd1: RichDate = " 2011-10-20 " val rd2: RichDate = "2011-10-20 " val rd3: RichDate = " 2011-10-20 " - rd1 must be_==(rd2) - rd1 must be_==(rd3) + assert(rd1 === rd2) + assert(rd1 === rd3) } "handle dates with slashes and underscores" in { val rd1: RichDate = "2011-10-20" val rd2: RichDate = "2011/10/20" val rd3: RichDate = "2011_10_20" - rd1 must be_==(rd2) - rd1 must be_==(rd3) + assert(rd1 === rd2) + assert(rd1 === rd3) } "be able to parse milliseconds" in { val rd1: RichDate = "2011-10-20 20:01:11.0" val rd2: RichDate = "2011-10-20 22:11:24.23" val rd3: RichDate = "2011-10-20 22:11:24.023 " - rd2 must_== rd3 + assert(rd2 === rd3) } "throw an exception when trying to parse illegal strings" in { // Natty is *really* generous about what it accepts - RichDate("jhbjhvhjv") must throwAn[IllegalArgumentException] - RichDate("99-99-99") must throwAn[IllegalArgumentException] + intercept[IllegalArgumentException] { RichDate("jhbjhvhjv") } + intercept[IllegalArgumentException] { RichDate("99-99-99") } } "be able to deal with arithmetic operations with whitespace" in { val rd1: RichDate = RichDate("2010-10-02") + Seconds(1) val rd2: RichDate = " 2010-10-02 T 00:00:01 " - rd1 must be_==(rd2) + assert(rd1 === rd2) } "Have same equals & hashCode as Date (crazy?)" in { val rd1: RichDate = "2011-10-20" - rd1.equals(rd1.value) must beTrue - rd1.hashCode must be_==(rd1.value.hashCode) + 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" - rd1 must be_<(rd2) - rd1 must be_<=(rd2) - rd2 must be_>(rd1) - rd2 must be_>=(rd1) - rd1 must be_>=(rd1) - rd2 must be_>=(rd2) + assert(rd1 < rd2) + assert(rd1 <= rd2) + assert(rd2 > rd1) + assert(rd2 >= rd1) + assert(rd1 >= rd1) + assert(rd2 >= rd2) } "implicitly convert from long" in { // This kind of implicit is not safe (what does the long mean?) @@ -90,56 +89,56 @@ class DateTest extends Specification { val long_val = 1319511818135L val rd1 = "2011-10-24T20:03:00" val rd2 = "2011-10-24T20:04:00" - DateRange(rd1, rd2).contains(RichDate(long_val)) must beTrue + assert(DateRange(rd1, rd2).contains(RichDate(long_val))) //Check edge cases: - DateRange(rd1, long_val).contains(long_val) must beTrue - DateRange(rd1, (long_val + 1)).contains(long_val) must beTrue - DateRange(long_val, rd2).contains(long_val) must beTrue - DateRange((long_val - 1), rd2).contains(long_val) must beTrue + assert(DateRange(rd1, long_val).contains(long_val)) + assert(DateRange(rd1, (long_val + 1)).contains(long_val)) + assert(DateRange(long_val, rd2).contains(long_val)) + assert(DateRange((long_val - 1), rd2).contains(long_val)) - DateRange(rd1, "2011-10-24T20:03:01").contains(long_val) must beFalse - DateRange(rd1, (long_val - 1)).contains(long_val) must beFalse - DateRange((long_val + 1), rd2).contains(long_val) must beFalse + assert(!DateRange(rd1, "2011-10-24T20:03:01").contains(long_val)) + assert(!DateRange(rd1, (long_val - 1)).contains(long_val)) + assert(!DateRange((long_val + 1), rd2).contains(long_val)) } "roundtrip successfully" in { val start_str = "2011-10-24 20:03:00" //string -> date -> string - RichDate(start_str).toString(DateOps.DATETIME_HMS_WITH_DASH) must_== start_str + assert(RichDate(start_str).toString(DateOps.DATETIME_HMS_WITH_DASH) === start_str) //long -> date == date -> long -> date val long_val = 1319511818135L val date = RichDate(long_val) val long2 = date.value.getTime val date2 = RichDate(long2) - date must_== date2 - long_val must_== long2 + assert(date === date2) + assert(long_val === long2) } "know the most recent time units" in { //10-25 is a Tuesday, earliest in week is a monday - Weeks(1).floorOf("2011-10-25") must_== (RichDate("2011-10-24")) - Days(1).floorOf("2011-10-25 10:01") must_== (RichDate("2011-10-25 00:00")) + assert(Weeks(1).floorOf("2011-10-25") === RichDate("2011-10-24")) + assert(Days(1).floorOf("2011-10-25 10:01") === RichDate("2011-10-25 00:00")) //Leaving off the time should give the same result: - Days(1).floorOf("2011-10-25 10:01") must_== (RichDate("2011-10-25")) - Hours(1).floorOf("2011-10-25 10:01") must_== (RichDate("2011-10-25 10:00")) + assert(Days(1).floorOf("2011-10-25 10:01") === RichDate("2011-10-25")) + assert(Hours(1).floorOf("2011-10-25 10:01") === RichDate("2011-10-25 10:00")) } "correctly do arithmetic" in { val d1: RichDate = "2011-10-24" (-4 to 4).foreach { n => List(Hours, Minutes, Seconds, Millisecs).foreach { u => val d2 = d1 + u(n) - (d2 - d1) must_== u(n) + assert((d2 - d1) === u(n)) } } } "correctly calculate upperBound" in { - Seconds(1).floorOf(RichDate.upperBound("2010-10-01")) must_== Seconds(1).floorOf(RichDate("2010-10-01 23:59:59")) - Seconds(1).floorOf(RichDate.upperBound("2010-10-01 14")) must_== Seconds(1).floorOf(RichDate("2010-10-01 14:59:59")) - Seconds(1).floorOf(RichDate.upperBound("2010-10-01 14:15")) must_== Seconds(1).floorOf(RichDate("2010-10-01 14:15:59")) + assert(Seconds(1).floorOf(RichDate.upperBound("2010-10-01")) === Seconds(1).floorOf(RichDate("2010-10-01 23:59:59"))) + assert(Seconds(1).floorOf(RichDate.upperBound("2010-10-01 14")) === Seconds(1).floorOf(RichDate("2010-10-01 14:59:59"))) + assert(Seconds(1).floorOf(RichDate.upperBound("2010-10-01 14:15")) === Seconds(1).floorOf(RichDate("2010-10-01 14:15:59"))) } } "A DateRange" should { "correctly iterate on each duration" in { def rangeContainTest(d1: DateRange, dur: Duration) = { - d1.each(dur).forall((d1r: DateRange) => d1.contains(d1r)) must beTrue + assert(d1.each(dur).forall((d1r: DateRange) => d1.contains(d1r))) } rangeContainTest(DateRange("2010-10-01", "2010-10-13"), Weeks(1)) rangeContainTest(DateRange("2010-10-01", "2010-10-13"), Weeks(2)) @@ -149,22 +148,22 @@ class DateTest extends Specification { //Prime number of Minutes rangeContainTest(DateRange("2010-10-01", "2010-10-13"), Minutes(13)) rangeContainTest(DateRange("2010-10-01", "2010-10-13"), Hours(13)) - DateRange("2010-10-01", "2010-10-10").each(Days(1)).size must_== 10 - DateRange("2010-10-01 00:00", RichDate("2010-10-02") - Millisecs(1)).each(Hours(1)).size must_== 24 - DateRange("2010-10-01 00:00", RichDate("2010-10-02") + Millisecs(1)).each(Hours(1)).size must_== 25 - DateRange("2010-10-01", RichDate.upperBound("2010-10-20")).each(Days(1)).size must_== 20 - DateRange("2010-10-01", RichDate.upperBound("2010-10-01")).each(Hours(1)).size must_== 24 - DateRange("2010-10-31", RichDate.upperBound("2010-10-31")).each(Hours(1)).size must_== 24 - DateRange("2010-10-31", RichDate.upperBound("2010-10-31")).each(Days(1)).size must_== 1 - DateRange("2010-10-31 12:00", RichDate.upperBound("2010-10-31 13")).each(Minutes(1)).size must_== 120 + assert(DateRange("2010-10-01", "2010-10-10").each(Days(1)).size === 10) + assert(DateRange("2010-10-01 00:00", RichDate("2010-10-02") - Millisecs(1)).each(Hours(1)).size === 24) + assert(DateRange("2010-10-01 00:00", RichDate("2010-10-02") + Millisecs(1)).each(Hours(1)).size === 25) + assert(DateRange("2010-10-01", RichDate.upperBound("2010-10-20")).each(Days(1)).size === 20) + assert(DateRange("2010-10-01", RichDate.upperBound("2010-10-01")).each(Hours(1)).size === 24) + assert(DateRange("2010-10-31", RichDate.upperBound("2010-10-31")).each(Hours(1)).size === 24) + assert(DateRange("2010-10-31", RichDate.upperBound("2010-10-31")).each(Days(1)).size === 1) + assert(DateRange("2010-10-31 12:00", RichDate.upperBound("2010-10-31 13")).each(Minutes(1)).size === 120) } "have each partition disjoint and adjacent" in { def eachIsDisjoint(d: DateRange, dur: Duration) { val dl = d.each(dur) - dl.zip(dl.tail).forall { + assert(dl.zip(dl.tail).forall { case (da, db) => da.isBefore(db.start) && db.isAfter(da.end) && ((da.end + Millisecs(1)) == db.start) - } must beTrue + }) } eachIsDisjoint(DateRange("2010-10-01", "2010-10-03"), Days(1)) eachIsDisjoint(DateRange("2010-10-01", "2010-10-03"), Weeks(1)) @@ -181,39 +180,39 @@ class DateTest extends Specification { (RichDate("2011-12-01") + d1) == (RichDate("2011-12-01") + d2) } "have 1000 milliseconds in a sec" in { - isSame(Millisecs(1000), Seconds(1)) must beTrue - Seconds(1).toMillisecs must_== 1000L - Millisecs(1000).toSeconds must_== 1.0 - Seconds(2).toMillisecs must_== 2000L - Millisecs(2000).toSeconds must_== 2.0 + assert(isSame(Millisecs(1000), Seconds(1))) + assert(Seconds(1).toMillisecs === 1000L) + assert(Millisecs(1000).toSeconds === 1.0) + assert(Seconds(2).toMillisecs === 2000L) + assert(Millisecs(2000).toSeconds === 2.0) } "have 60 seconds in a minute" in { - isSame(Seconds(60), Minutes(1)) must beTrue - Minutes(1).toSeconds must_== 60.0 - Minutes(1).toMillisecs must_== 60 * 1000L - Minutes(2).toSeconds must_== 120.0 - Minutes(2).toMillisecs must_== 120 * 1000L + assert(isSame(Seconds(60), Minutes(1))) + assert(Minutes(1).toSeconds === 60.0) + assert(Minutes(1).toMillisecs === 60 * 1000L) + assert(Minutes(2).toSeconds === 120.0) + assert(Minutes(2).toMillisecs === 120 * 1000L) } "have 60 minutes in a hour" in { - isSame(Minutes(60), Hours(1)) must beTrue - Hours(1).toSeconds must_== 60.0 * 60.0 - Hours(1).toMillisecs must_== 60 * 60 * 1000L - Hours(2).toSeconds must_== 2 * 60.0 * 60.0 - Hours(2).toMillisecs must_== 2 * 60 * 60 * 1000L + assert(isSame(Minutes(60), Hours(1))) + assert(Hours(1).toSeconds === 60.0 * 60.0) + assert(Hours(1).toMillisecs === 60 * 60 * 1000L) + assert(Hours(2).toSeconds === 2 * 60.0 * 60.0) + assert(Hours(2).toMillisecs === 2 * 60 * 60 * 1000L) } - "have 7 days in a week" in { isSame(Days(7), Weeks(1)) must beTrue } + "have 7 days in a week" in { assert(isSame(Days(7), Weeks(1))) } } "AbsoluteDurations" should { "behave as comparable" in { - (Hours(5) >= Hours(2)) must beTrue - (Minutes(60) >= Minutes(60)) must beTrue - (Hours(1) < Millisecs(3600001)) must beTrue + assert(Hours(5) >= Hours(2)) + assert(Minutes(60) >= Minutes(60)) + assert(Hours(1) < Millisecs(3600001)) } "add properly" in { - (Hours(2) + Hours(1)).compare(Hours(3)) must_== 0 + assert((Hours(2) + Hours(1)).compare(Hours(3)) === 0) } "have a well behaved max function" in { - AbsoluteDuration.max(Hours(1), Hours(2)).compare(Hours(2)) must_== 0 + assert(AbsoluteDuration.max(Hours(1), Hours(2)).compare(Hours(2)) === 0) } } "Globifiers" should { @@ -246,9 +245,7 @@ class DateTest extends Specification { List("/2011/11/*/", "/2011/12/01/", "/2011/12/02/")) :: Nil - testcases.foreach { tup => - tup._1 must_== tup._2 - } + testcases.foreach { case (l, r) => assert(l === r) } } "The forward and reverser should match" in { @@ -263,7 +260,7 @@ class DateTest extends Specification { hourlyTestCases.foreach { dr => val resultantDR = globifierOps.hourlyRtGlobifier(dr) - globifierOps.normalizeHrDr(dr) must_== globifierOps.normalizeHrDr(resultantDR) + assert(globifierOps.normalizeHrDr(dr) === globifierOps.normalizeHrDr(resultantDR)) } val dailyTestCases = List( @@ -275,7 +272,7 @@ class DateTest extends Specification { dailyTestCases.foreach { dr => val resultantDR = globifierOps.dailyRtGlobifier(dr) - globifierOps.normalizeDayDr(dr) must_== globifierOps.normalizeDayDr(resultantDR) + assert(globifierOps.normalizeDayDr(dr) === globifierOps.normalizeDayDr(resultantDR)) } } @@ -301,10 +298,10 @@ class DateTest extends Specification { val splits = bruteForce(pattern, dr, Hours(1)) val globed = t1.globify(dr) - eachElementDistinct(globed) must beTrue + assert(eachElementDistinct(globed)) //See that each path is matched by exactly one glob: - splits.map { path => globed.filter { globMatchesDate(_)(path) }.size } - .forall { _ == 1 } must beTrue + assert(splits.map { path => globed.filter { globMatchesDate(_)(path) }.size } + .forall { _ == 1 }) } } } 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 55fb59df08..92ce1c6fa1 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 @@ -123,7 +123,7 @@ class LocalCluster(mutex: Boolean = true) { val baseClassPath = List( getClass, classOf[JobConf], - classOf[ScalaObject], + classOf[Option[_]], classOf[LoggerFactory], classOf[Log4jLoggerAdapter], classOf[com.twitter.scalding.Args], diff --git a/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/Scalatest.scala b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/Scalatest.scala new file mode 100644 index 0000000000..ab2f15eb17 --- /dev/null +++ b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/Scalatest.scala @@ -0,0 +1,53 @@ +/* +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.{ BeforeAndAfterEach, Suite } + +/** + * This is a mixin fixture for scalatest which makes it easy to use a LocalCluster and will manage + * the lifecycle of one appropriately. + */ +trait HadoopPlatformTest extends BeforeAndAfterEach { 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 beforeEach() { + cluster.synchronized { + initialize() + } + super.beforeEach() + } + + //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 afterEach() { + try super.afterEach() + 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 00d68a52b5..3b3d05e7f8 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 @@ -17,7 +17,7 @@ package com.twitter.scalding.platform import com.twitter.scalding._ -import org.specs._ +import org.scalatest.{ Matchers, WordSpec } class InAndOutJob(args: Args) extends Job(args) { Tsv("input").read.write(Tsv("output")) @@ -69,61 +69,43 @@ 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 Specification { - +class PlatformTests extends WordSpec with Matchers with HadoopPlatformTest { 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) - noDetailedDiffs() //Fixes an issue with scala 2.9 - "An InAndOutTest" should { - val cluster = LocalCluster() - doFirst { cluster.initialize() } - val inAndOut = Seq("a", "b", "c") "reading then writing shouldn't change the data" in { HadoopPlatformJobTest(new InAndOutJob(_), cluster) .source("input", inAndOut) - .sink[String]("output") { _.toSet must_== inAndOut.toSet } + .sink[String]("output") { _.toSet shouldBe (inAndOut.toSet) } .run } - - doLast { cluster.shutdown() } } "A TinyJoinAndMergeJob" should { - val cluster = LocalCluster() - doFirst { cluster.initialize() } - import TinyJoinAndMergeJob._ "merge and joinWithTiny shouldn't duplicate data" in { HadoopPlatformJobTest(new TinyJoinAndMergeJob(_), cluster) .source(peopleInput, peopleData) .source(messageInput, messageData) - .sink(output) { _.toSet must_== outputData.toSet } + .sink(output) { _.toSet shouldBe (outputData.toSet) } .run } - - doLast { cluster.shutdown() } } "A TsvNoCacheJob" should { - val cluster = LocalCluster() - doFirst { cluster.initialize() } - import TsvNoCacheJob._ "Writing to a tsv in a flow shouldn't effect the output" in { HadoopPlatformJobTest(new TsvNoCacheJob(_), cluster) .source(dataInput, data) - .sink(typedThrowAwayOutput) { _.toSet.size must_== 4 } - .sink(typedRealOutput) { _.map{ f: Float => (f * 10).toInt }.toList must_== outputData.map{ f: Float => (f * 10).toInt }.toList } + .sink(typedThrowAwayOutput) { _.toSet should have size 4 } + .sink(typedRealOutput) { _.map{ f: Float => (f * 10).toInt }.toList shouldBe (outputData.map{ f: Float => (f * 10).toInt }.toList) } .run } - - doLast { cluster.shutdown() } } } @@ -147,36 +129,27 @@ class NormalDistinctJob(args: Args) extends Job(args) { TypedPipe.from(TypedTsv[String]("input")).distinct.write(TypedTsv("output")) } -class IterableSourceDistinctTest extends Specification { - noDetailedDiffs() - +class IterableSourceDistinctTest extends WordSpec with Matchers with HadoopPlatformTest { "A IterableSource" should { import IterableSourceDistinctJob._ - val cluster = LocalCluster() - doFirst { cluster.initialize() } - - setSequential() - "distinct properly from normal data" in { HadoopPlatformJobTest(new NormalDistinctJob(_), cluster) .source[String]("input", data ++ data ++ data) - .sink[String]("output") { _.toList must_== data } + .sink[String]("output") { _.toList shouldBe data } .run } "distinctBy(identity) properly from a list in memory" in { HadoopPlatformJobTest(new IterableSourceDistinctIdentityJob(_), cluster) - .sink[String]("output") { _.toList must_== data } + .sink[String]("output") { _.toList shouldBe data } .run } "distinct properly from a list" in { HadoopPlatformJobTest(new IterableSourceDistinctJob(_), cluster) - .sink[String]("output") { _.toList must_== data } + .sink[String]("output") { _.toList shouldBe data } .run } - - doLast { cluster.shutdown() } } } 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 62c6e65527..4975d6d2e0 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,9 +1,9 @@ package com.twitter.scalding.reducer_estimation import com.twitter.scalding._ -import com.twitter.scalding.platform.{ HadoopPlatformJobTest, LocalCluster } +import com.twitter.scalding.platform.{ HadoopPlatformJobTest, HadoopPlatformTest, LocalCluster } import org.apache.hadoop.mapred.JobConf -import org.specs._ +import org.scalatest.{ Matchers, WordSpec } import scala.collection.JavaConverters._ object HipJob { @@ -54,58 +54,44 @@ class SimpleJob(args: Args) extends Job(args) { .write(counts) } -class ReducerEstimatorTest extends Specification { - +class ReducerEstimatorTestSingle extends WordSpec with Matchers with HadoopPlatformTest { import HipJob._ - "Single-step job with reducer estimator" should { - - val cluster = LocalCluster() - - val conf = Config.empty - .addReducerEstimator(classOf[InputSizeReducerEstimator]) + - (InputSizeReducerEstimator.BytesPerReducer -> (1L << 10).toString) - - doFirst { cluster.initialize(conf) } + override def initialize() = cluster.initialize(Config.empty + .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.size must_== 1 + steps should have size 1 val conf = Config.fromHadoop(steps.head.getConfig) - conf.getNumReducers must_== Some(3) + conf.getNumReducers should contain (3) } .run } - - doLast { cluster.shutdown() } } +} +class ReducerEstimatorTestMulti extends WordSpec with Matchers with HadoopPlatformTest { + import HipJob._ - "Multi-step job with reducer estimator" should { - val cluster = LocalCluster() - - val conf = Config.empty - .addReducerEstimator(classOf[InputSizeReducerEstimator]) + - (InputSizeReducerEstimator.BytesPerReducer -> (1L << 16).toString) - - doFirst { cluster.initialize(conf) } + override def initialize() = cluster.initialize(Config.empty + .addReducerEstimator(classOf[InputSizeReducerEstimator]) + + (InputSizeReducerEstimator.BytesPerReducer -> (1L << 16).toString)) + "Multi-step job with reducer estimator" should { "run with correct number of reducers in each step" in { HadoopPlatformJobTest(new HipJob(_), cluster) - .sink[Double](out)(_.head must beCloseTo(2.86, 0.0001)) + .sink[Double](out)(_.head shouldBe 2.86 +- 0.0001) .inspectCompletedFlow { flow => val steps = flow.getFlowSteps.asScala - val reducers = steps.map(_.getConfig.getInt(Config.HadoopNumReducers, 0)).toList - reducers must_== List(1, 1, 2) + reducers shouldBe List(1, 1, 2) } .run } - - doLast { cluster.shutdown() } - } - } diff --git a/scalding-jdbc/src/test/scala/com/twitter/scalding/jdbc/JDBCSourceCompileTest.scala b/scalding-jdbc/src/test/scala/com/twitter/scalding/jdbc/JDBCSourceCompileTest.scala index 447c35d6ca..de2b015a1a 100644 --- a/scalding-jdbc/src/test/scala/com/twitter/scalding/jdbc/JDBCSourceCompileTest.scala +++ b/scalding-jdbc/src/test/scala/com/twitter/scalding/jdbc/JDBCSourceCompileTest.scala @@ -1,6 +1,6 @@ package com.twitter.scalding.jdbc -import org.specs._ +import org.scalatest.WordSpec class ExampleMysqlJdbcSource() extends JDBCSource with MysqlDriver { override val tableName = TableName("test") @@ -11,8 +11,7 @@ class ExampleMysqlJdbcSource() extends JDBCSource with MysqlDriver { datetime("off"), text("of"), double("my"), - smallint("cloud") - ) + smallint("cloud")) override def currentConfig = ConnectionSpec(ConnectUrl("how"), UserName("are"), Password("you")) } @@ -25,12 +24,11 @@ class ExampleVerticaJdbcSource() extends JDBCSource with VerticaJdbcDriver { datetime("off"), text("of"), double("my"), - smallint("cloud") - ) + smallint("cloud")) override def currentConfig = ConnectionSpec(ConnectUrl("how"), UserName("are"), Password("you")) } -class JDBCSourceCompileTest extends Specification { +class JDBCSourceCompileTest extends WordSpec { "JDBCSource" should { "Pick up correct column definitions for MySQL Driver" in { new ExampleMysqlJdbcSource().toSqlCreateString diff --git a/scalding-json/src/main/scala/com/twitter/scalding/JsonLine.scala b/scalding-json/src/main/scala/com/twitter/scalding/JsonLine.scala index 914c7ff7a5..f50850d647 100644 --- a/scalding-json/src/main/scala/com/twitter/scalding/JsonLine.scala +++ b/scalding-json/src/main/scala/com/twitter/scalding/JsonLine.scala @@ -90,9 +90,9 @@ object JsonLine extends scala.runtime.AbstractFunction5[String, Fields, SinkMode } private[this] def typeFromManifest(m: Manifest[_]): Type = { - if (m.typeArguments.isEmpty) { m.erasure } + if (m.typeArguments.isEmpty) { m.runtimeClass } else new ParameterizedType { - def getRawType = m.erasure + def getRawType = m.runtimeClass def getActualTypeArguments = m.typeArguments.map(typeFromManifest).toArray diff --git a/scalding-json/src/main/scala/com/twitter/scalding/TypedJson.scala b/scalding-json/src/main/scala/com/twitter/scalding/TypedJson.scala new file mode 100644 index 0000000000..9685059276 --- /dev/null +++ b/scalding-json/src/main/scala/com/twitter/scalding/TypedJson.scala @@ -0,0 +1,71 @@ +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 } + +import scala.collection.JavaConverters._ +import scala.util.Try + +import cascading.pipe.Pipe + +/** + * This type uses the structural type of a case class, but not it's name, to describe the Json using json4s. + * This is intended to be used for intermediate output from a REPL session. + * The intended use is to save adhoc data between sessions. + * The fully qualified class name of classes defined in a REPL is not stable between REPL sessions. + * + * We believe using a fixed schema, such as thrift or Avro is a much safer way to do long term productionized data + * pipelines to minimize risks of incompatible changes to schema that render old data unreadable. + */ + +object TypedJson { + private implicit val formats = native.Serialization.formats(NoTypeHints) + private def caseClass2Json[A <: AnyRef](implicit tt: Manifest[A], fmt: Formats): Injection[A, String] = new AbstractInjection[A, String] { + override def apply(a: A): String = write(a) + + override def invert(b: String): Try[A] = attempt(b)(read[A]) + } + + def apply[T <: AnyRef: Manifest](p: String) = new TypedJson(p) +} + +class TypedJson[T <: AnyRef: Manifest](p: String) extends FixedPathSource(p) + with TextSourceScheme + with SingleMappable[T] + with TypedSink[T] { + import Dsl._ + import TypedJson._ + + private[this] val fieldSym = 'jsonString + + @transient private[this] lazy val inj = caseClass2Json[T] + + override def transformForWrite(pipe: Pipe) = + pipe.mapTo((0) -> (fieldSym)) { inj.apply(_: T) } + + override def transformForRead(pipe: Pipe) = + pipe.mapTo(('line) -> (fieldSym)) { (jsonStr: String) => inj.invert(jsonStr).get } + + override def setter[U <: T] = TupleSetter.asSubSetter[T, U](TupleSetter.singleSetter[T]) + + override def toIterator(implicit config: Config, mode: Mode): Iterator[T] = { + val tap = createTap(Read)(mode) + mode.openForRead(config, tap) + .asScala + .map { te => + inj.invert(te.selectTuple('line).getObject(0).asInstanceOf[String]).get + } + } +} + +case class TypedJsonLzo[T <: AnyRef: Manifest](p: String) extends TypedJson[T](p) { + override def hdfsScheme = HadoopSchemeInstance(new LzoTextLine().asInstanceOf[cascading.scheme.Scheme[_, _, _, _, _]]) +} diff --git a/scalding-json/src/test/scala/com/twitter/scalding/JsonLineTest.scala b/scalding-json/src/test/scala/com/twitter/scalding/JsonLineTest.scala index 531fa64c2f..44cc9b8373 100644 --- a/scalding-json/src/test/scala/com/twitter/scalding/JsonLineTest.scala +++ b/scalding-json/src/test/scala/com/twitter/scalding/JsonLineTest.scala @@ -20,7 +20,7 @@ import cascading.flow.FlowException import cascading.tap.SinkMode import cascading.tuple.Fields import com.twitter.scalding.{ JsonLine => StandardJsonLine, _ } -import org.specs._ +import org.scalatest.WordSpec object JsonLine { def apply(p: String, fields: Fields = Fields.ALL, failOnEmptyLines: Boolean = true) = @@ -81,8 +81,7 @@ class JsonLineNestedInputJob(args: Args) extends Job(args) { } } -class JsonLineTest extends Specification { - noDetailedDiffs() +class JsonLineTest extends WordSpec { import com.twitter.scalding.Dsl._ "A JsonLine sink" should { @@ -91,7 +90,7 @@ class JsonLineTest extends Specification { .sink[String](JsonLine("output0")) { buf => val json = buf.head "not stringify lists or numbers and not escape single quotes" in { - json must be_==("""{"query":"doctor's mask","queryStats":[42.1,17.1]}""") + assert(json === """{"query":"doctor's mask","queryStats":[42.1,17.1]}""") } } .run @@ -102,7 +101,7 @@ class JsonLineTest extends Specification { .sink[String](JsonLine("output0", Tuple1('query))) { buf => val json = buf.head "only sink requested fields" in { - json must be_==("""{"query":"doctor's mask"}""") + assert(json === """{"query":"doctor's mask"}""") } } .run @@ -115,7 +114,7 @@ class JsonLineTest extends Specification { .sink[(Int, String)](Tsv("output0")) { outBuf => "read json line input" in { - outBuf.toList must be_==(List((3, "baz"))) + assert(outBuf.toList === List((3, "baz"))) } } .run @@ -128,7 +127,7 @@ class JsonLineTest extends Specification { .sink[(Int, String)](Tsv("output0")) { outBuf => "handle missing fields" in { - outBuf.toList must be_==(List((3, "baz"), (7, null))) + assert(outBuf.toList === List((3, "baz"), (7, null))) } } .run @@ -141,21 +140,23 @@ class JsonLineTest extends Specification { .sink[(Int, String)](Tsv("output0")) { outBuf => "handle nested fields" in { - outBuf.toList must be_==(List((0, "baz"), (9, null))) + assert(outBuf.toList === List((0, "baz"), (9, null))) } } .run .finish "fail on empty lines by default" in { - JobTest(new JsonLineInputJob(_)) - .source(JsonLine("input0", ('foo, 'bar)), List((0, json), (1, json2), (2, ""), (3, " "))) - .sink[(Int, String)](Tsv("output0")) { - outBuf => outBuf.toList + intercept[FlowException] { + JobTest(new JsonLineInputJob(_)) + .source(JsonLine("input0", ('foo, 'bar)), List((0, json), (1, json2), (2, ""), (3, " "))) + .sink[(Int, String)](Tsv("output0")) { + outBuf => outBuf.toList - } - .run - .finish must throwAnException[FlowException] + } + .run + .finish + } } JobTest(new JsonLineInputJobSkipEmptyLines(_)) @@ -163,7 +164,7 @@ class JsonLineTest extends Specification { .sink[(Int, String)](Tsv("output0")) { outBuf => "handle empty lines when `failOnEmptyLines` is set to false" in { - outBuf.toList.size must be_==(2) + assert(outBuf.toList.size === 2) } } .run diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/MacroImplicits.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/MacroImplicits.scala new file mode 100644 index 0000000000..aa3d822da7 --- /dev/null +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/MacroImplicits.scala @@ -0,0 +1,30 @@ +/* + 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 scala.language.experimental.macros + +import com.twitter.scalding._ +import com.twitter.scalding.macros.impl._ + +object MacroImplicits { + /** + * This method provides proof that the given type is a case class. + */ + implicit def materializeCaseClassTupleSetter[T]: TupleSetter[T] = macro TupleSetterImpl.caseClassTupleSetterImpl[T] + implicit def materializeCaseClassTupleConverter[T]: TupleConverter[T] = macro TupleConverterImpl.caseClassTupleConverterImpl[T] + implicit def materializeCaseClassTypeDescriptor[T]: TypeDescriptor[T] = macro TypeDescriptorProviderImpl.caseClassTypeDescriptorImpl[T] +} 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 new file mode 100644 index 0000000000..465a27ae5c --- /dev/null +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/Macros.scala @@ -0,0 +1,48 @@ +/* + 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 scala.language.experimental.macros + +import com.twitter.scalding._ +import com.twitter.scalding.macros.impl._ +import cascading.tuple.Fields + +object Macros { + + // There is two flavors of the below functions, the pure vs withUnknown. + // In both cases recursive case classes, primitive types, and options are flattened down onto cascading tuples. + // In the unknown casehowever if a type is reached that we don't know what to do we store that type into the tuple. + + def caseClassTupleSetter[T]: TupleSetter[T] = macro TupleSetterImpl.caseClassTupleSetterImpl[T] + def caseClassTupleSetterWithUnknown[T]: TupleSetter[T] = macro TupleSetterImpl.caseClassTupleSetterWithUnknownImpl[T] + + def caseClassTupleConverter[T]: TupleConverter[T] = macro TupleConverterImpl.caseClassTupleConverterImpl[T] + def caseClassTupleConverterWithUnknown[T]: TupleConverter[T] = macro TupleConverterImpl.caseClassTupleConverterWithUnknownImpl[T] + + def toFields[T]: Fields = macro FieldsProviderImpl.toFieldsImpl[T] + def toFieldsWithUnknown[T]: Fields = macro FieldsProviderImpl.toFieldsWithUnknownImpl[T] + + def toNamedFields[T]: Fields = macro FieldsProviderImpl.toFieldsImpl[T] + def toNamedFieldsWithUnknown[T]: Fields = macro FieldsProviderImpl.toFieldsWithUnknownImpl[T] + + def toIndexedFields[T]: Fields = macro FieldsProviderImpl.toIndexedFieldsImpl[T] + def toIndexedFieldsWithUnknown[T]: Fields = macro FieldsProviderImpl.toIndexedFieldsWithUnknownImpl[T] + + def caseClassTypeDescriptor[T]: TypeDescriptor[T] = macro TypeDescriptorProviderImpl.caseClassTypeDescriptorImpl[T] + def caseClassTypeDescriptorWithUnknown[T]: TypeDescriptor[T] = macro TypeDescriptorProviderImpl.caseClassTypeDescriptorWithUnknownImpl[T] + +} diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/FieldsProviderImpl.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/FieldsProviderImpl.scala new file mode 100644 index 0000000000..ce8dfd15d9 --- /dev/null +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/FieldsProviderImpl.scala @@ -0,0 +1,102 @@ +/* + 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.reflect.runtime.universe._ + +import com.twitter.scalding._ +import com.twitter.bijection.macros.IsCaseClass +import com.twitter.bijection.macros.impl.IsCaseClassImpl +/** + * This class contains the core macro implementations. This is in a separate module to allow it to be in + * a separate compilation unit, which makes it easier to provide helper methods interfacing with macros. + */ +object FieldsProviderImpl { + def toFieldsImpl[T](c: Context)(implicit T: c.WeakTypeTag[T]): c.Expr[cascading.tuple.Fields] = + toFieldsCommonImpl(c, true, false)(T) + + def toFieldsWithUnknownImpl[T](c: Context)(implicit T: c.WeakTypeTag[T]): c.Expr[cascading.tuple.Fields] = + toFieldsCommonImpl(c, true, true)(T) + + def toIndexedFieldsImpl[T](c: Context)(implicit T: c.WeakTypeTag[T]): c.Expr[cascading.tuple.Fields] = + toFieldsCommonImpl(c, false, false)(T) + + def toIndexedFieldsWithUnknownImpl[T](c: Context)(implicit T: c.WeakTypeTag[T]): c.Expr[cascading.tuple.Fields] = + toFieldsCommonImpl(c, false, true)(T) + + def toFieldsCommonImpl[T](c: Context, namedFields: Boolean, allowUnknownTypes: Boolean)(implicit T: c.WeakTypeTag[T]): c.Expr[cascading.tuple.Fields] = { + 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, outerName: String, fieldName: String, isOption: Boolean): List[(Tree, String)] = { + val returningType = if (isOption) q"""classOf[java.lang.Object]""" else q"""classOf[$fieldType]""" + val simpleRet = List((returningType, s"$outerName$fieldName")) + fieldType match { + case tpe if tpe =:= typeOf[String] => simpleRet + case tpe if tpe =:= typeOf[Boolean] => simpleRet + case tpe if tpe =:= typeOf[Short] => simpleRet + case tpe if tpe =:= typeOf[Int] => simpleRet + case tpe if tpe =:= typeOf[Long] => simpleRet + case tpe if tpe =:= typeOf[Float] => simpleRet + case tpe if tpe =:= typeOf[Double] => simpleRet + case tpe if tpe.erasure =:= typeOf[Option[Any]] && isOption == true => c.abort(c.enclosingPosition, s"Case class ${T} has nested options, not supported currently.") + 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"$outerName$fieldName.", isOption) + case tpe if allowUnknownTypes => simpleRet + 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, String)] = { + 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, isOption) + }.toList + } + + val expanded = expandMethod(T.tpe, "", false) + if (expanded.isEmpty) c.abort(c.enclosingPosition, s"Case class ${T} has no primitive types we were able to extract") + + val typeTrees = expanded.map(_._1) + + val res = if (namedFields) { + val fieldNames = expanded.map(_._2) + q""" + new _root_.cascading.tuple.Fields(_root_.scala.Array.apply[java.lang.Comparable[_]](..$fieldNames), + _root_.scala.Array.apply[java.lang.reflect.Type](..$typeTrees)) with _root_.com.twitter.bijection.macros.MacroGenerated + """ + } else { + val indices = typeTrees.zipWithIndex.map(_._2) + q""" + new _root_.cascading.tuple.Fields(_root_.scala.Array.apply[_root_.java.lang.Comparable[_]](..$indices), + _root_.scala.Array.apply[java.lang.reflect.Type](..$typeTrees)) with _root_.com.twitter.bijection.macros.MacroGenerated + """ + } + c.Expr[cascading.tuple.Fields](res) + } + +} diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/TupleConverterImpl.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/TupleConverterImpl.scala new file mode 100644 index 0000000000..835ee23391 --- /dev/null +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/TupleConverterImpl.scala @@ -0,0 +1,167 @@ +/* + 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._ +import com.twitter.bijection.macros.{ IsCaseClass, MacroGenerated } +import com.twitter.bijection.macros.impl.IsCaseClassImpl +/** + * This class contains the core macro implementations. This is in a separate module to allow it to be in + * a separate compilation unit, which makes it easier to provide helper methods interfacing with macros. + */ + +object TupleConverterImpl { + def caseClassTupleConverterImpl[T](c: Context)(implicit T: c.WeakTypeTag[T]): c.Expr[TupleConverter[T]] = + caseClassTupleConverterCommonImpl(c, false) + + def caseClassTupleConverterWithUnknownImpl[T](c: Context)(implicit T: c.WeakTypeTag[T]): c.Expr[TupleConverter[T]] = + caseClassTupleConverterCommonImpl(c, true) + + def caseClassTupleConverterCommonImpl[T](c: Context, allowUnknownTypes: Boolean)(implicit T: c.WeakTypeTag[T]): c.Expr[TupleConverter[T]] = { + 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 + } + + def matchField(outerTpe: Type, idx: Int, inOption: Boolean): (Int, Extractor, List[Builder]) = { + def getPrimitive(primitiveGetter: Tree, boxedType: Type, box: Option[Tree]) = { + if (inOption) { + val cachedResult = newTermName(c.fresh(s"cacheVal")) + val boxed = box.map{ b => q"""$b($primitiveGetter)""" }.getOrElse(primitiveGetter) + + val builder = q""" + val $cachedResult: $boxedType = if(t.getObject($idx) == null) { + null.asInstanceOf[$boxedType] + } else { + $boxed + } + """ + (idx + 1, + Extractor(boxedType, q"$cachedResult"), + List(Builder(builder))) + } else { + (idx + 1, Extractor(outerTpe, primitiveGetter), List[Builder]()) + } + } + + outerTpe match { + case tpe if tpe =:= typeOf[String] => getPrimitive(q"""t.getString(${idx})""", typeOf[java.lang.String], None) + case tpe if tpe =:= typeOf[Boolean] => getPrimitive(q"""t.getBoolean(${idx})""", typeOf[java.lang.Boolean], Some(q"_root_.java.lang.Boolean.valueOf")) + case tpe if tpe =:= typeOf[Short] => getPrimitive(q"""t.getShort(${idx})""", typeOf[java.lang.Short], Some(q"_root_.java.lang.Short.valueOf")) + case tpe if tpe =:= typeOf[Int] => getPrimitive(q"""t.getInteger(${idx})""", typeOf[java.lang.Integer], Some(q"_root_.java.lang.Integer.valueOf")) + case tpe if tpe =:= typeOf[Long] => getPrimitive(q"""t.getLong(${idx})""", typeOf[java.lang.Long], Some(q"_root_.java.lang.Long.valueOf")) + case tpe if tpe =:= typeOf[Float] => getPrimitive(q"""t.getFloat(${idx})""", typeOf[java.lang.Float], Some(q"_root_.java.lang.Float.valueOf")) + case tpe if tpe =:= typeOf[Double] => getPrimitive(q"""t.getDouble(${idx})""", typeOf[java.lang.Double], Some(q"_root_.java.lang.Double.valueOf")) + case tpe if tpe.erasure =:= typeOf[Option[Any]] && inOption => + 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 + + val (newIdx, extractor, builders) = matchField(innerType, idx, true) + + val cachedResult = newTermName(c.fresh(s"opti")) + val extractorTypeVal: Tree = if (extractor.tpe =:= innerType) + extractor.toTree + else + q"${innerType.typeSymbol.companionSymbol}.unbox($extractor)" + + val build = Builder(q""" + val $cachedResult = if($extractor == null) { + _root_.scala.Option.empty[$innerType] + } else { + _root_.scala.Some($extractorTypeVal) + } + """) + (newIdx, Extractor(tpe, q"""$cachedResult"""), builders :+ build) + case tpe if IsCaseClassImpl.isCaseClassType(c)(tpe) => expandCaseClass(tpe, idx, inOption) + case tpe if allowUnknownTypes => getPrimitive(q"""t.getObject(${idx}).asInstanceOf[$tpe]""", tpe, None) + case _ => c.abort(c.enclosingPosition, s"Case class ${T} is not pure primitives, Option of a primitive nested case classes") + } + } + + def expandCaseClass(outerTpe: Type, parentIdx: Int, inOption: Boolean): (Int, Extractor, List[Builder]) = { + val (idx, extractors, builders) = outerTpe + .declarations + .collect { case m: MethodSymbol if m.isCaseAccessor => m } + .foldLeft((parentIdx, List[Extractor](), List[Builder]())) { + case ((idx, oldExtractors, oldBuilders), accessorMethod) => + val (newIdx, extractors, builders) = matchField(accessorMethod.returnType, idx, inOption) + (newIdx, oldExtractors :+ extractors, oldBuilders ::: builders) + } + val cachedResult = newTermName(c.fresh(s"cacheVal")) + + val simpleBuilder = q"${outerTpe.typeSymbol.companionSymbol}(..$extractors)" + val builder = if (inOption) { + val tstOpt = extractors.map(e => q"$e == null").foldLeft(Option.empty[Tree]) { + case (e, nxt) => + e match { + case Some(t) => Some(q"$t || $nxt") + case None => Some(nxt) + } + } + tstOpt match { + case Some(tst) => + q""" + val $cachedResult: $outerTpe = if($tst) { + null + } else { + $simpleBuilder + } + """ + case None => q"val $cachedResult = $simpleBuilder" + } + } else { + q"val $cachedResult = $simpleBuilder" + } + ( + idx, + Extractor(outerTpe, q"$cachedResult"), + builders :+ Builder(builder)) + } + + val (finalIdx, extractor, builders) = expandCaseClass(T.tpe, 0, false) + if (finalIdx == 0) c.abort(c.enclosingPosition, "Didn't consume any elements in the tuple, possibly empty case class?") + + val res = q""" + new _root_.com.twitter.scalding.TupleConverter[$T] with _root_.com.twitter.bijection.macros.MacroGenerated { + override def apply(t: _root_.cascading.tuple.TupleEntry): $T = { + ..$builders + $extractor + } + override val arity: scala.Int = ${finalIdx} + } + """ + + c.Expr[TupleConverter[T]](res) + } +} diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/TupleSetterImpl.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/TupleSetterImpl.scala new file mode 100644 index 0000000000..92a5118317 --- /dev/null +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/TupleSetterImpl.scala @@ -0,0 +1,104 @@ +/* + 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 com.twitter.scalding._ +import com.twitter.bijection.macros.{ IsCaseClass, MacroGenerated } +import com.twitter.bijection.macros.impl.IsCaseClassImpl +/** + * This class contains the core macro implementations. This is in a separate module to allow it to be in + * a separate compilation unit, which makes it easier to provide helper methods interfacing with macros. + */ +object TupleSetterImpl { + + def caseClassTupleSetterImpl[T](c: Context)(implicit T: c.WeakTypeTag[T]): c.Expr[TupleSetter[T]] = + caseClassTupleSetterCommonImpl(c, false) + + def caseClassTupleSetterWithUnknownImpl[T](c: Context)(implicit T: c.WeakTypeTag[T]): c.Expr[TupleSetter[T]] = + caseClassTupleSetterCommonImpl(c, true) + + def caseClassTupleSetterCommonImpl[T](c: Context, allowUnknownTypes: Boolean)(implicit T: c.WeakTypeTag[T]): c.Expr[TupleSetter[T]] = { + 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(outerTpe: Type, idx: Int, pTree: Tree): (Int, Tree) = { + def simpleType(accessor: Tree) = + (idx + 1, q"""${accessor}(${idx}, $pTree)""") + + outerTpe match { + case tpe if tpe =:= typeOf[String] => simpleType(q"tup.setString") + case tpe if tpe =:= typeOf[Boolean] => simpleType(q"tup.setBoolean") + case tpe if tpe =:= typeOf[Short] => simpleType(q"tup.setShort") + case tpe if tpe =:= typeOf[Int] => simpleType(q"tup.setInteger") + case tpe if tpe =:= typeOf[Long] => simpleType(q"tup.setLong") + case tpe if tpe =:= typeOf[Float] => simpleType(q"tup.setFloat") + case tpe if tpe =:= typeOf[Double] => simpleType(q"tup.setDouble") + case tpe if tpe.erasure =:= typeOf[Option[Any]] => + val cacheName = newTermName(c.fresh(s"optiIndx")) + val (newIdx, subTree) = + matchField(tpe.asInstanceOf[TypeRefApi].args.head, idx, q"$cacheName") + val nullSetters = (idx until newIdx).map { curIdx => + q"""tup.set($curIdx, null)""" + } + + (newIdx, q""" + if($pTree.isDefined) { + val $cacheName = $pTree.get + $subTree + } else { + ..$nullSetters + } + """) + + case tpe if IsCaseClassImpl.isCaseClassType(c)(tpe) => expandMethod(tpe, idx, pTree) + case tpe if allowUnknownTypes => simpleType(q"tup.set") + case _ => c.abort(c.enclosingPosition, s"Case class ${T} is not pure primitives, Option of a primitive nested case classes") + } + } + + def expandMethod(outerTpe: Type, parentIdx: Int, 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(accessorMethod.returnType, idx, q"""$pTree.$accessorMethod""") + (newIdx, q""" + $existingTree + $subTree""") + } + + val (finalIdx, set) = expandMethod(T.tpe, 0, q"t") + if (finalIdx == 0) c.abort(c.enclosingPosition, "Didn't consume any elements in the tuple, possibly empty case class?") + val res = q""" + new _root_.com.twitter.scalding.TupleSetter[$T] with _root_.com.twitter.bijection.macros.MacroGenerated { + override def apply(t: $T): _root_.cascading.tuple.Tuple = { + val tup = _root_.cascading.tuple.Tuple.size($finalIdx) + $set + tup + } + override val arity: _root_.scala.Int = $finalIdx + } + """ + c.Expr[TupleSetter[T]](res) + } +} diff --git a/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/TypeDescriptorProviderImpl.scala b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/TypeDescriptorProviderImpl.scala new file mode 100644 index 0000000000..cee8f42f59 --- /dev/null +++ b/scalding-macros/src/main/scala/com/twitter/scalding/macros/impl/TypeDescriptorProviderImpl.scala @@ -0,0 +1,57 @@ +/* + 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.reflect.runtime.universe._ + +import com.twitter.scalding._ +import com.twitter.bijection.macros.IsCaseClass +import com.twitter.bijection.macros.impl.IsCaseClassImpl +/** + * This class contains the core macro implementations. This is in a separate module to allow it to be in + * a separate compilation unit, which makes it easier to provide helper methods interfacing with macros. + */ +object TypeDescriptorProviderImpl { + + def caseClassTypeDescriptorImpl[T](c: Context)(implicit T: c.WeakTypeTag[T]): c.Expr[TypeDescriptor[T]] = + caseClassTypeDescriptorCommonImpl(c, false)(T) + + def caseClassTypeDescriptorWithUnknownImpl[T](c: Context)(implicit T: c.WeakTypeTag[T]): c.Expr[TypeDescriptor[T]] = + caseClassTypeDescriptorCommonImpl(c, true)(T) + + def caseClassTypeDescriptorCommonImpl[T](c: Context, allowUnknownTypes: Boolean)(implicit T: c.WeakTypeTag[T]): c.Expr[TypeDescriptor[T]] = { + 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.""") + val converter = TupleConverterImpl.caseClassTupleConverterCommonImpl[T](c, allowUnknownTypes) + val setter = TupleSetterImpl.caseClassTupleSetterCommonImpl[T](c, allowUnknownTypes) + val fields = FieldsProviderImpl.toFieldsCommonImpl[T](c, true, allowUnknownTypes) + + val res = q""" + new _root_.com.twitter.scalding.TypeDescriptor[$T] with _root_.com.twitter.bijection.macros.MacroGenerated { + override val converter = $converter + override val setter = $setter + override val fields = $fields + } + """ + c.Expr[TypeDescriptor[T]](res) + } + +} diff --git a/scalding-macros/src/test/scala/com/twitter/scalding/macros/MacroDepHygiene.scala b/scalding-macros/src/test/scala/com/twitter/scalding/macros/MacroDepHygiene.scala new file mode 100644 index 0000000000..2d10728c7e --- /dev/null +++ b/scalding-macros/src/test/scala/com/twitter/scalding/macros/MacroDepHygiene.scala @@ -0,0 +1,71 @@ +/* + 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.WordSpec +import com.twitter.scalding.macros.{ _ => _ } + +/** + * This test is intended to ensure that the macros do not require any imported code in scope. This is why all + * references are via absolute paths. + */ +class MacroDepHygiene extends WordSpec { + import com.twitter.bijection.macros.impl.IsCaseClassImpl + + case class A(x: Int, y: String) + case class B(x: A, y: String, z: A) + class C + + def isMg(a: Any) = a.isInstanceOf[com.twitter.bijection.macros.MacroGenerated] + + "TupleSetter macro" should { + def isTupleSetterAvailable[T](implicit proof: com.twitter.scalding.TupleSetter[T]) = isMg(proof) + + "work fine without any imports" in { + com.twitter.scalding.macros.Macros.caseClassTupleSetter[A] + com.twitter.scalding.macros.Macros.caseClassTupleSetter[B] + } + + "implicitly work fine without any imports" in { + import com.twitter.scalding.macros.MacroImplicits.materializeCaseClassTupleSetter + assert(isTupleSetterAvailable[A]) + assert(isTupleSetterAvailable[B]) + } + + "fail if not a case class" in { + assert(!isTupleSetterAvailable[C]) + } + } + + "TupleConverter macro" should { + def isTupleConverterAvailable[T](implicit proof: com.twitter.scalding.TupleConverter[T]) = isMg(proof) + + "work fine without any imports" in { + com.twitter.scalding.macros.Macros.caseClassTupleConverter[A] + com.twitter.scalding.macros.Macros.caseClassTupleConverter[B] + } + + "implicitly work fine without any imports" in { + import com.twitter.scalding.macros.MacroImplicits.materializeCaseClassTupleConverter + assert(isTupleConverterAvailable[A]) + assert(isTupleConverterAvailable[B]) + } + + "fail if not a case class" in { + assert(!isTupleConverterAvailable[C]) + } + } +} 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 new file mode 100644 index 0000000000..7fdb83c9ae --- /dev/null +++ b/scalding-macros/src/test/scala/com/twitter/scalding/macros/MacrosUnitTests.scala @@ -0,0 +1,250 @@ +/* + 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 cascading.tuple.{ Tuple => CTuple, TupleEntry } + +import org.scalatest.{ Matchers, WordSpec } + +import com.twitter.scalding._ +import com.twitter.scalding.macros._ +import com.twitter.scalding.macros.impl._ +import com.twitter.scalding.serialization.Externalizer + +import com.twitter.bijection.macros.{ IsCaseClass, MacroGenerated } + +// We avoid nesting these just to avoid any complications in the serialization test +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: Option[SampleClassC]) +case class SampleClassE(a: String, b: Boolean, c: Short, d: Int, e: Long, f: Float, g: Double) +case class SampleClassF(a: Option[Int]) +case class SampleClassG(a: java.util.Date) + +case class SampleClassFail(a: Option[Option[Int]]) + +class MacrosUnitTests extends WordSpec with Matchers { + import MacroImplicits._ + def isMg[T](t: T): T = { + t shouldBe a[MacroGenerated] + t + } + + private val dummy = new TupleConverter[Nothing] { + def apply(te: TupleEntry) = sys.error("dummy") + override val arity = 1 + } + + private val dummy2 = new TypeDescriptor[Nothing] { + def setter = sys.error("dummy") + def converter = sys.error("dummy") + def fields = sys.error("dummy") + } + + def isMacroTupleConverterAvailable[T](implicit proof: TupleConverter[T] = dummy.asInstanceOf[TupleConverter[T]]) = + proof.isInstanceOf[MacroGenerated] + + def isMacroTypeDescriptorAvailable[T](implicit proof: TypeDescriptor[T] = dummy2.asInstanceOf[TypeDescriptor[T]]) = + proof.isInstanceOf[MacroGenerated] + + def mgConv[T](te: TupleEntry)(implicit conv: TupleConverter[T]): T = isMg(conv)(te) + def mgSet[T](t: T)(implicit set: TupleSetter[T]): TupleEntry = new TupleEntry(isMg(set)(t)) + + def shouldRoundTrip[T: IsCaseClass: TupleSetter: TupleConverter](t: T) { + t shouldBe mgConv(mgSet(t)) + } + + def shouldRoundTripOther[T: IsCaseClass: TupleSetter: TupleConverter](te: TupleEntry, t: T) { + val inter = mgConv(te) + inter shouldBe t + mgSet(inter) shouldBe te + } + + def canExternalize(t: AnyRef) { Externalizer(t).javaWorks shouldBe true } + + "MacroGenerated TupleSetter" should { + + "Generate the setter SampleClassA" in { Macros.caseClassTupleSetter[SampleClassA] } + "Generate the setter SampleClassB" in { Macros.caseClassTupleSetter[SampleClassB] } + "Generate the setter SampleClassC" in { Macros.caseClassTupleSetter[SampleClassC] } + "Generate the setter SampleClassD" in { Macros.caseClassTupleSetter[SampleClassD] } + "Generate the setter SampleClassE" in { Macros.caseClassTupleSetter[SampleClassE] } + "Generate the setter SampleClassF" in { Macros.caseClassTupleSetter[SampleClassF] } + "Generate the setter SampleClassG" in { Macros.caseClassTupleSetterWithUnknown[SampleClassG] } + + def doesJavaWork[T](implicit set: TupleSetter[T]) { canExternalize(isMg(set)) } + "be serializable for case class A" in { doesJavaWork[SampleClassA] } + "be serializable for case class B" in { doesJavaWork[SampleClassB] } + "be serializable for case class C" in { doesJavaWork[SampleClassC] } + "be serializable for case class D" in { doesJavaWork[SampleClassD] } + "be serializable for case class E" in { doesJavaWork[SampleClassE] } + "be serializable for case class F" in { doesJavaWork[SampleClassF] } + } + + "MacroGenerated TupleConverter" should { + "Generate the converter SampleClassA" in { Macros.caseClassTupleConverter[SampleClassA] } + "Generate the converter SampleClassB" in { Macros.caseClassTupleConverter[SampleClassB] } + "Generate the converter SampleClassC" in { Macros.caseClassTupleConverter[SampleClassC] } + "Generate the converter SampleClassD" in { Macros.caseClassTupleConverter[SampleClassD] } + "Generate the converter SampleClassE" in { Macros.caseClassTupleConverter[SampleClassE] } + "Generate the converter SampleClassF" in { Macros.caseClassTupleConverter[SampleClassF] } + "Generate the converter SampleClassG" in { Macros.caseClassTupleConverterWithUnknown[SampleClassG] } + + "Not generate a convertor for SampleClassFail" in { isMacroTupleConverterAvailable[SampleClassFail] shouldBe false } + + def doesJavaWork[T](implicit conv: TupleConverter[T]) { canExternalize(isMg(conv)) } + "be serializable for case class A" in { doesJavaWork[SampleClassA] } + "be serializable for case class B" in { doesJavaWork[SampleClassB] } + "be serializable for case class C" in { doesJavaWork[SampleClassC] } + "be serializable for case class D" in { doesJavaWork[SampleClassD] } + "be serializable for case class E" in { doesJavaWork[SampleClassE] } + "be serializable for case class F" in { doesJavaWork[SampleClassF] } + } + + "MacroGenerated TypeDescriptor" should { + "Generate the converter SampleClassA" in { Macros.caseClassTypeDescriptor[SampleClassA] } + "Generate the converter SampleClassB" in { Macros.caseClassTypeDescriptor[SampleClassB] } + "Generate the converter SampleClassC" in { Macros.caseClassTypeDescriptor[SampleClassC] } + "Generate the converter SampleClassD" in { Macros.caseClassTypeDescriptor[SampleClassD] } + "Generate the converter SampleClassE" in { Macros.caseClassTypeDescriptor[SampleClassE] } + "Generate the converter SampleClassF" in { Macros.caseClassTypeDescriptor[SampleClassF] } + "Generate the converter SampleClassG" in { Macros.caseClassTypeDescriptorWithUnknown[SampleClassG] } + + "Not generate a convertor for SampleClassFail" in { isMacroTypeDescriptorAvailable[SampleClassFail] shouldBe false } + + def doesJavaWork[T](implicit conv: TypeDescriptor[T]) { canExternalize(isMg(conv)) } + "be serializable for case class A" in { doesJavaWork[SampleClassA] } + "be serializable for case class B" in { doesJavaWork[SampleClassB] } + "be serializable for case class C" in { doesJavaWork[SampleClassC] } + "be serializable for case class D" in { doesJavaWork[SampleClassD] } + "be serializable for case class E" in { doesJavaWork[SampleClassE] } + "be serializable for case class F" in { doesJavaWork[SampleClassF] } + } + + "MacroGenerated TupleSetter and TupleConverter" should { + "round trip class -> tupleentry -> class" in { + shouldRoundTrip(SampleClassA(100, "onehundred")) + shouldRoundTrip(SampleClassB(SampleClassA(100, "onehundred"), SampleClassA(-1, "zero"), "what")) + val a = SampleClassA(73, "hrmA1") + val b = SampleClassB(a, a, "hrmB1") + val c = SampleClassC(a, b, SampleClassA(123980, "heyA2"), SampleClassB(a, SampleClassA(-1, "zeroA3"), "zooB2"), b) + shouldRoundTrip(b) + shouldRoundTrip(c) + shouldRoundTrip(SampleClassD(Some(c))) + shouldRoundTrip(SampleClassD(None)) + + implicit val tupSetterG = Macros.caseClassTupleSetterWithUnknown[SampleClassG] + implicit val tupConverterG = Macros.caseClassTupleConverterWithUnknown[SampleClassG] + shouldRoundTrip(SampleClassG(new java.util.Date(123412L))) + } + + "Case Class should form expected tuple" in { + val input = SampleClassC(SampleClassA(1, "asdf"), + SampleClassB(SampleClassA(2, "bcdf"), SampleClassA(5, "jkfs"), "wetew"), + SampleClassA(9, "xcmv"), + SampleClassB(SampleClassA(23, "ck"), SampleClassA(13, "dafk"), "xcv"), + SampleClassB(SampleClassA(34, "were"), SampleClassA(654, "power"), "adsfmx")) + val setter = implicitly[TupleSetter[SampleClassC]] + val tup = setter(input) + assert(tup.size == 19) + assert(tup.getInteger(0) === 1) + assert(tup.getString(18) === "adsfmx") + } + + "round trip tupleentry -> class -> tupleEntry" in { + val a_tup = CTuple.size(2) + a_tup.setInteger(0, 100) + a_tup.setString(1, "onehundred") + val a_te = new TupleEntry(a_tup) + val a = SampleClassA(100, "onehundred") + shouldRoundTripOther(a_te, a) + + val b_tup = CTuple.size(5) + b_tup.setInteger(0, 100) + b_tup.setString(1, "onehundred") + b_tup.setInteger(2, 100) + b_tup.setString(3, "onehundred") + b_tup.setString(4, "what") + val b_te = new TupleEntry(b_tup) + val b = SampleClassB(a, a, "what") + shouldRoundTripOther(b_te, b) + + val c_tup = CTuple.size(19) + c_tup.setInteger(0, 100) + c_tup.setString(1, "onehundred") + + c_tup.setInteger(2, 100) + c_tup.setString(3, "onehundred") + c_tup.setInteger(4, 100) + c_tup.setString(5, "onehundred") + c_tup.setString(6, "what") + + c_tup.setInteger(7, 100) + c_tup.setString(8, "onehundred") + + c_tup.setInteger(9, 100) + c_tup.setString(10, "onehundred") + c_tup.setInteger(11, 100) + c_tup.setString(12, "onehundred") + c_tup.setString(13, "what") + + c_tup.setInteger(14, 100) + c_tup.setString(15, "onehundred") + c_tup.setInteger(16, 100) + c_tup.setString(17, "onehundred") + c_tup.setString(18, "what") + + val c_te = new TupleEntry(c_tup) + val c = SampleClassC(a, b, a, b, b) + shouldRoundTripOther(c_te, c) + } + + "Case Class should form expected Fields" in { + val fields = Macros.toFields[SampleClassB] + assert(fields.size === 5) + assert(fields.getTypes === Array[java.lang.reflect.Type](classOf[Int], classOf[String], classOf[Int], classOf[String], classOf[String])) + val names = List("a1.x", "a1.y", "a2.x", "a2.y", "y") + names.zipWithIndex.foreach { + case (name, indx) => + assert(fields.get(indx) === name) + } + } + + "Case Class should form expected Fields with Options" in { + val fields = Macros.toFields[SampleClassD] + assert(fields.size === 19) + assert(fields.getTypes === Array.fill[java.lang.reflect.Type](19)(classOf[java.lang.Object])) + } + + "Case Class should form expected Fields with Unknown types" in { + val fields = Macros.toFieldsWithUnknown[SampleClassG] + assert(fields.size === 1) + assert(fields.getTypes === Array[java.lang.reflect.Type](classOf[java.util.Date])) + } + + "Case Class should form expected Indexed Fields" in { + val fields = Macros.toIndexedFields[SampleClassB] + assert(fields.size === 5) + assert(fields.getTypes === Array[java.lang.reflect.Type](classOf[Int], classOf[String], classOf[Int], classOf[String], classOf[String])) + val names = (0 until fields.size) + names.zipWithIndex.foreach { + case (name, indx) => + assert(fields.get(indx) === name) + } + } + } +} diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/thrift/ParquetThrift.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/thrift/ParquetThrift.scala index bdd9578e1a..9308992b9d 100644 --- a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/thrift/ParquetThrift.scala +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/thrift/ParquetThrift.scala @@ -33,9 +33,7 @@ trait ParquetThriftBase[T] extends FileSource with SingleMappable[T] with TypedS def mf: Manifest[T] def config: ParquetValueScheme.Config[T] = { - - val config = new ParquetValueScheme.Config[T].withRecordClass(mf.erasure.asInstanceOf[Class[T]]) - + val config = new ParquetValueScheme.Config[T].withRecordClass(mf.runtimeClass.asInstanceOf[Class[T]]) val configWithFp = withFilter match { case Some(fp) => config.withFilterPredicate(fp) case None => config @@ -51,7 +49,6 @@ trait ParquetThriftBase[T] extends FileSource with SingleMappable[T] with TypedS } override def setter[U <: T] = TupleSetter.asSubSetter[T, U](TupleSetter.singleSetter[T]) - } trait ParquetThrift[T <: ParquetThrift.ThriftBase] extends ParquetThriftBase[T] { @@ -114,4 +111,4 @@ class HourlySuffixParquetThrift[T <: ParquetThrift.ThriftBase]( extends HourlySuffixSource(path, dateRange) with ParquetThrift[T] class FixedPathParquetThrift[T <: ParquetThrift.ThriftBase](paths: String*)(implicit override val mf: Manifest[T]) - extends FixedPathSource(paths: _*) with ParquetThrift[T] \ No newline at end of file + extends FixedPathSource(paths: _*) with ParquetThrift[T] diff --git a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/ParquetSourcesTests.scala b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/ParquetSourcesTests.scala index 4ecb729faa..b34be1aa5d 100644 --- a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/ParquetSourcesTests.scala +++ b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/ParquetSourcesTests.scala @@ -7,11 +7,11 @@ import com.twitter.scalding.{ DateRange, RichDate, Source } import java.lang.{ Integer => JInt } import org.apache.thrift.protocol.TProtocol import org.apache.thrift.{ TBase, TFieldIdEnum } -import org.specs.Specification +import org.scalatest.{ Matchers, WordSpec } import parquet.filter2.predicate.FilterApi._ import parquet.filter2.predicate.{ FilterApi, FilterPredicate } -abstract class ParquetSourcesTestsBase extends Specification { +abstract class ParquetSourcesTestsBase extends WordSpec with Matchers { val dateRange = DateRange(RichDate(0L), RichDate(0L)) val path = "/a/path" @@ -26,27 +26,27 @@ abstract class ParquetSourcesTestsBase extends Specification { def testDefaultFilter[S <: Source with HasFilterPredicate](src: S) = { "default to no filter predicate" in { - src.withFilter must be equalTo None + src.withFilter shouldEqual None } } def testReturnProvidedFilter[S <: Source with HasFilterPredicate](src: S) = { "return the provided filter" in { - src.withFilter must be equalTo Some(filter1) + src.withFilter shouldEqual Some(filter1) } } def testDefaultColumns[S <: Source with HasColumnProjection](src: S) = { "default to no column projection" in { - src.columnGlobs must beEmpty - src.globsInParquetStringFormat must be equalTo None + src.columnGlobs shouldBe empty + assert(src.globsInParquetStringFormat === None) } } def testReturnProvidedColumns[S <: Source with HasColumnProjection](src: S) = { "return the provided columns" in { - src.columnGlobs must be equalTo columns + assert(src.columnGlobs === columns) } "correctly format globs into parquet's expected format" in { @@ -55,7 +55,7 @@ abstract class ParquetSourcesTestsBase extends Specification { } private def verifyParquetStringFormat(s: String, expected: Set[String]) = { - s.split(";").toSet must be equalTo expected + assert(s.split(";").toSet === expected) } } @@ -160,4 +160,4 @@ class MockTBase extends TBase[MockTBase, TFieldIdEnum] { override def deepCopy(): TBase[MockTBase, TFieldIdEnum] = null override def clear(): Unit = () override def compareTo(o: MockTBase): Int = 0 -} \ No newline at end of file +} 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 new file mode 100644 index 0000000000..dbf9df69c2 --- /dev/null +++ b/scalding-repl/src/main/scala-2.10/com/twitter/scalding/ILoopCompat.scala @@ -0,0 +1,5 @@ +package com.twitter.scalding + +import scala.tools.nsc.interpreter.{ ILoop, IMain } + +trait ILoopCompat extends ILoop diff --git a/scalding-repl/src/main/scala-2.11/com/twitter/scalding/ILoopCompat.scala b/scalding-repl/src/main/scala-2.11/com/twitter/scalding/ILoopCompat.scala new file mode 100644 index 0000000000..632f0fc91f --- /dev/null +++ b/scalding-repl/src/main/scala-2.11/com/twitter/scalding/ILoopCompat.scala @@ -0,0 +1,7 @@ +package com.twitter.scalding + +import scala.tools.nsc.interpreter.ILoop + +trait ILoopCompat extends ILoop { + def addThunk(f: => Unit): Unit = intp.initialize(f) +} 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 349440f953..db851fe4df 100644 --- a/scalding-repl/src/main/scala/com/twitter/scalding/ReplImplicits.scala +++ b/scalding-repl/src/main/scala/com/twitter/scalding/ReplImplicits.scala @@ -31,6 +31,23 @@ object ReplImplicits extends FieldConversions { var flowDef: FlowDef = getEmptyFlowDef /** Defaults to running in local mode if no mode is specified. */ var mode: Mode = com.twitter.scalding.Local(false) + /** + * If the repl is started in Hdfs mode, this field is used to preserve the settings + * when switching Modes. + */ + private[scalding] var storedHdfsMode: Option[Hdfs] = None + + /** Switch to Local mode */ + def useLocalMode() { mode = Local(false) } + def useStrictLocalMode() { mode = Local(true) } + + /** Switch to Hdfs mode */ + 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.") + } + } /** * Configuration to use for REPL executions. @@ -216,6 +233,8 @@ object ReplImplicits extends FieldConversions { * used everywhere. */ object ReplImplicitContext { + /** Implicit execution context for using the Execution monad */ + implicit val executionContext = ConcurrentExecutionContext.global /** Implicit flowDef for this Scalding shell session. */ implicit def flowDefImpl = ReplImplicits.flowDef /** Defaults to running in local mode if no mode is specified. */ diff --git a/scalding-repl/src/main/scala/com/twitter/scalding/ScaldingILoop.scala b/scalding-repl/src/main/scala/com/twitter/scalding/ScaldingILoop.scala index b25dd0a134..7e5c6bcf08 100644 --- a/scalding-repl/src/main/scala/com/twitter/scalding/ScaldingILoop.scala +++ b/scalding-repl/src/main/scala/com/twitter/scalding/ScaldingILoop.scala @@ -17,13 +17,13 @@ package com.twitter.scalding import java.io.File -import scala.tools.nsc.interpreter.ILoop +import scala.tools.nsc.interpreter.IR /** * A class providing Scalding specific commands for inclusion in the Scalding REPL. */ class ScaldingILoop - extends ILoop { + extends ILoopCompat { override def printWelcome() { val fc = Console.YELLOW val wc = Console.RED @@ -56,6 +56,10 @@ class ScaldingILoop */ override def prompt: String = ScaldingShell.prompt() + private[this] def addImports(ids: String*): IR.Result = + if (ids.isEmpty) IR.Success + else intp.interpret("import " + ids.mkString(", ")) + /** * Search for files with the given name in all directories from current directory * up to root. @@ -73,17 +77,20 @@ class ScaldingILoop */ override def commands: List[LoopCommand] = super.commands ++ scaldingCommands - addThunk { - intp.beQuietDuring { - intp.addImports( - "com.twitter.scalding._", - "com.twitter.scalding.ReplImplicits._", - "com.twitter.scalding.ReplImplicitContext._") + override def createInterpreter() { + super.createInterpreter() + addThunk { + intp.beQuietDuring { + addImports( + "com.twitter.scalding._", + "com.twitter.scalding.ReplImplicits._", + "com.twitter.scalding.ReplImplicitContext._") - // interpret all files named ".scalding_repl" from the current directory up to the root - findAllUpPath(".scalding_repl") - .reverse // work down from top level file to more specific ones - .foreach(f => loadCommand(f.toString)) + // interpret all files named ".scalding_repl" from the current directory up to the root + findAllUpPath(".scalding_repl") + .reverse // work down from top level file to more specific ones + .foreach(f => loadCommand(f.toString)) + } } } } 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 33a083e82d..c1febab692 100644 --- a/scalding-repl/src/main/scala/com/twitter/scalding/ScaldingShell.scala +++ b/scalding-repl/src/main/scala/com/twitter/scalding/ScaldingShell.scala @@ -80,6 +80,12 @@ object ScaldingShell extends MainGenericRunner { scaldingREPL = Some(new ScaldingILoop) ReplImplicits.mode = mode + // if in Hdfs mode, store the mode to enable switching between Local and Hdfs + mode match { + case m @ Hdfs(_, _) => ReplImplicits.storedHdfsMode = Some(m) + case _ => () + } + scaldingREPL.get.process(command.settings) } @@ -123,7 +129,7 @@ object ScaldingShell extends MainGenericRunner { val virtualDirectory = repl.virtualDirectory val tempJar = new File(Files.createTempDir(), "scalding-repl-session-" + System.currentTimeMillis() + ".jar") - createJar(virtualDirectory, tempJar) + createJar(virtualDirectory.asInstanceOf[VirtualDirectory], tempJar) } } 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 02520d8ffc..7c3b62319d 100644 --- a/scalding-repl/src/main/scala/com/twitter/scalding/ShellPipe.scala +++ b/scalding-repl/src/main/scala/com/twitter/scalding/ShellPipe.scala @@ -61,6 +61,7 @@ class ShellTypedPipe[T](pipe: TypedPipe[T]) { class ShellValuePipe[T](vp: ValuePipe[T]) { import ReplImplicits.execute // This might throw if the value is empty + def dump: Unit = println(toOption) def get: T = execute(vp.getExecution) def getOrElse(t: => T): T = execute(vp.getOrElseExecution(t)) def toOption: Option[T] = execute(vp.toOptionExecution) diff --git a/scalding-repl/src/test/scala/com/twitter/scalding/ReplTest.scala b/scalding-repl/src/test/scala/com/twitter/scalding/ReplTest.scala index 25ea813ab7..36a0a61182 100644 --- a/scalding-repl/src/test/scala/com/twitter/scalding/ReplTest.scala +++ b/scalding-repl/src/test/scala/com/twitter/scalding/ReplTest.scala @@ -16,11 +16,11 @@ limitations under the License. package com.twitter.scalding import cascading.tuple.Fields -import org.specs._ +import org.scalatest.WordSpec import scala.collection.JavaConverters._ import org.apache.hadoop.mapred.JobConf -class ReplTest extends Specification { +class ReplTest extends WordSpec { import ReplImplicits._ import ReplImplicitContext._ @@ -42,24 +42,23 @@ class ReplTest extends Specification { hello.save(out) val output = out.toIterator.toList - output must_== helloRef + assert(output === helloRef) } - "snapshot" in { - + "snapshot" should { "only -- TypedPipe[String]" in { val hello = TypedPipe.from(TextLine(helloPath)) val s: TypedPipe[String] = hello.snapshot // shallow verification that the snapshot was created correctly without // actually running a new flow to check the contents (just check that // it's a TypedPipe from a MemorySink or SequenceFile) - s.toString must beMatching("IterablePipe|TypedPipeFactory") + assert(s.toString.contains("IterablePipe") || s.toString.contains("TypedPipeFactory")) val pipeName = mode match { case m: HadoopMode => m.jobConf.get("hadoop.tmp.dir") case _ => "IterableSource" } - s.toPipe(Fields.ALL).toString must beMatching(pipeName) + assert(s.toPipe(Fields.ALL).toString.contains(pipeName)) } @@ -74,7 +73,7 @@ class ReplTest extends Specification { s.map(_.toLowerCase).save(out) val output = out.toIterator.toList - output must_== helloRef.flatMap(_.split("\\s+")).map(_.toLowerCase) + assert(output === helloRef.flatMap(_.split("\\s+")).map(_.toLowerCase)) } "tuples -- TypedPipe[(String,Int)]" in { @@ -84,26 +83,26 @@ class ReplTest extends Specification { .snapshot val output = s.toList - output must_== helloRef.flatMap(_.split("\\s+")).map(w => (w.toLowerCase, w.length)) + assert(output === helloRef.flatMap(_.split("\\s+")).map(w => (w.toLowerCase, w.length))) } - "grouped -- Grouped[String,String]" in { + "grouped -- Grouped[String,String]" which { val grp = TypedPipe.from(TextLine(helloPath)) .groupBy(_.toLowerCase) val correct = helloRef.map(l => (l.toLowerCase, l)) - "explicit" in { - grp.snapshot.toList must_== correct + "is explicit" in { + (grp.snapshot.toList === correct) } // Note: Must explicitly to toIterator because `grp.toList` resolves to `KeyedList.toList` - "implicit" in { - grp.toIterator.toList must_== correct + "is implicit" in { + assert(grp.toIterator.toList === correct) } } - "joined -- CoGrouped[String, Long]" in { + "joined -- CoGrouped[String, Long]" which { val linesByWord = TypedPipe.from(TextLine(helloPath)) .flatMap(_.split("\\s+")) .groupBy(_.toLowerCase) @@ -115,12 +114,12 @@ class ReplTest extends Specification { val correct = Map("hello" -> 1.0, "goodbye" -> 3.0, "world" -> 4.0) - "explicit" in { + "is explicit" in { val s = grp.snapshot - s.toIterator.toMap must_== correct + assert(s.toIterator.toMap === correct) } - "implicit" in { - grp.toIterator.toMap must_== correct + "is implicit" in { + assert(grp.toIterator.toMap === correct) } } @@ -128,13 +127,13 @@ class ReplTest extends Specification { val hello = TypedPipe.from(TextLine(helloPath)) val res = hello.map(_.length).sum val correct = helloRef.map(_.length).sum - res.toOption must_== Some(correct) + assert(res.toOption === Some(correct)) } } "reset flow" in { resetFlowDef() - flowDef.getSources.asScala must beEmpty + assert(flowDef.getSources.asScala.isEmpty) } "run entire flow" in { @@ -147,39 +146,36 @@ class ReplTest extends Specification { val out = TypedTsv[String](testPath + "words.tsv") hello.write(out) - run + ReplImplicits.run val words = out.toIterator.toSet - words must_== Set("hello", "world", "goodbye") + assert(words === Set("hello", "world", "goodbye")) } - "TypedPipe of a TextLine supports" in { + "TypedPipe of a TextLine" should { val hello = TypedPipe.from(TextLine(helloPath)) - "toIterator" in { + "support toIterator" in { hello.toIterator.foreach { line: String => - line must beMatching("Hello world|Goodbye world") + assert(line.contains("Hello world") || line.contains("Goodbye world")) } } - "toList" in { - hello.toList must_== helloRef + "support toList" in { + assert(hello.toList === helloRef) } } - "toIterator should generate a snapshot for" in { + "toIterator should generate a snapshot for TypedPipe with" should { val hello = TypedPipe.from(TextLine(helloPath)) - "TypedPipe with flatMap" in { + "flatMap" in { val out = hello.flatMap(_.split("\\s+")).toList - out must_== helloRef.flatMap(_.split("\\s+")) + assert(out === helloRef.flatMap(_.split("\\s+"))) } - "TypedPipe with tuple" in { - hello.map(l => (l, l.length)).toList must_== helloRef.map(l => (l, l.length)) + "tuple" in { + assert(hello.map(l => (l, l.length)).toList === helloRef.map(l => (l, l.length))) } } - } - sequential - "REPL in Local mode" should { ReplImplicits.mode = Local(strictSources = true) test() diff --git a/scripts/lib/trollop.rb b/scripts/lib/trollop.rb index c3b0d4f56d..647b8a4feb 100644 --- a/scripts/lib/trollop.rb +++ b/scripts/lib/trollop.rb @@ -92,7 +92,7 @@ def initialize *a, &b ## [+:long+] Specify the long form of the argument, i.e. the form with two dashes. If unspecified, will be automatically derived based on the argument name by turning the +name+ option into a string, and replacing any _'s by -'s. ## [+:short+] Specify the short form of the argument, i.e. the form with one dash. If unspecified, will be automatically derived from +name+. ## [+:type+] Require that the argument take a parameter or parameters of type +type+. For a single parameter, the value can be a member of +SINGLE_ARG_TYPES+, or a corresponding Ruby class (e.g. +Integer+ for +:int+). For multiple-argument parameters, the value can be any member of +MULTI_ARG_TYPES+ constant. If unset, the default argument type is +:flag+, meaning that the argument does not take a parameter. The specification of +:type+ is not necessary if a +:default+ is given. - ## [+:default+] Set the default value for an argument. Without a default value, the hash returned by #parse (and thus Trollop::options) will have a +nil+ value for this key unless the argument is given on the commandline. The argument type is derived automatically from the class of the default value given, so specifying a +:type+ is not necessary if a +:default+ is given. (But see below for an important caveat when +:multi+: is specified too.) If the argument is a flag, and the default is set to +true+, then if it is specified on the the commandline the value will be +false+. + ## [+:default+] Set the default value for an argument. Without a default value, the hash returned by #parse (and thus Trollop::options) will have a +nil+ value for this key unless the argument is given on the commandline. The argument type is derived automatically from the class of the default value given, so specifying a +:type+ is not necessary if a +:default+ is given. (But see below for an important caveat when +:multi+: is specified too.) If the argument is a flag, and the default is set to +true+, then if it is specified on the commandline the value will be +false+. ## [+:required+] If set to +true+, the argument must be provided on the commandline. ## [+:multi+] If set to +true+, allows multiple occurrences of the option on the commandline. Otherwise, only a single instance of the option is allowed. (Note that this is different from taking multiple parameters. See below.) ## diff --git a/scripts/run_test.sh b/scripts/run_test.sh index c9d7bf1f27..53e6be1702 100755 --- a/scripts/run_test.sh +++ b/scripts/run_test.sh @@ -2,11 +2,19 @@ BASE_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )"/.. && pwd )" cd $BASE_DIR +export JVM_OPTS="-XX:+CMSClassUnloadingEnabled -XX:+UseConcMarkSweepGC -XX:ReservedCodeCacheSize=96m -XX:+TieredCompilation -XX:MaxPermSize=128m -Xms256m -Xmx512m -Xss2m" + +INNER_JAVA_OPTS="set javaOptions += \"-Dlog4j.configuration=file://$TRAVIS_BUILD_DIR/project/travis-log4j.properties\"" + +withCmd() { + CMD=$1 + for t in $TEST_TARGET; do echo "; project $t; set logLevel := Level.Warn; $INNER_JAVA_OPTS; ++$TRAVIS_SCALA_VERSION; $CMD"; done +} + bash -c "while true; do echo -n .; sleep 5; done" & + PROGRESS_REPORTER_PID=$! -time ./sbt ++$TRAVIS_SCALA_VERSION $TEST_TARGET/compile $TEST_TARGET/test:compile &> /dev/null +time ./sbt "$(withCmd "compile; test:compile")" &> /dev/null kill -9 $PROGRESS_REPORTER_PID -export JVM_OPTS="-XX:+CMSClassUnloadingEnabled -XX:+UseConcMarkSweepGC -XX:ReservedCodeCacheSize=96m -XX:+TieredCompilation -XX:MaxPermSize=128m -Xms256m -Xmx512m -Xss2m" - -./sbt -Dlog4j.configuration=file://$TRAVIS_BUILD_DIR/project/travis-log4j.properties ++$TRAVIS_SCALA_VERSION $TEST_TARGET/test +./sbt "$(withCmd test)" diff --git a/scripts/scald.rb b/scripts/scald.rb index a4027f0412..62dc767c2b 100755 --- a/scripts/scald.rb +++ b/scripts/scald.rb @@ -136,18 +136,20 @@ def maven_filename(jar_filename) end SCALA_VERSION= OPTS[:scalaversion] || BUILDFILE.match(/scalaVersion\s*:=\s*\"([^\"]+)\"/)[1] -SHORT_SCALA_VERSION = SCALA_VERSION.start_with?("2.10") ? "2.10" : SCALA_VERSION +SHORT_SCALA_VERSION = if SCALA_VERSION.start_with?("2.10") +"2.10" +elsif SCALA_VERSION.start_with?("2.11") + "2.11" + else + SCALA_VERSION +end SBT_HOME="#{ENV['HOME']}/.sbt" SCALA_LIB_DIR = Dir.tmpdir + "/scald.rb/scala_home/#{SCALA_VERSION}" def scala_libs(version) - if( version.start_with?("2.10") ) - ["scala-library", "scala-reflect", "scala-compiler"] - else - ["scala-library", "scala-compiler"] - end + ["scala-library", "scala-reflect", "scala-compiler"] end def find_dependencies(org, dep, version) diff --git a/version.sbt b/version.sbt index 02996e98b2..0d39db4919 100644 --- a/version.sbt +++ b/version.sbt @@ -1,2 +1 @@ - -version in ThisBuild := "0.12.0" +version in ThisBuild := "0.13.0"