diff --git a/.travis.yml b/.travis.yml index cbfd84ce51..f49cd925cb 100644 --- a/.travis.yml +++ b/.travis.yml @@ -38,6 +38,14 @@ matrix: env: BUILD="base" TEST_TARGET="scalding-core scalding-jdbc scalding-json scalding-db" script: "scripts/run_test.sh" + - scala: 2.10.6 + env: BUILD="base" TEST_TARGET="scalding-core-fabric-tests scalding-fabric-hadoop scalding-fabric-hadoop2-mr1 scalding-fabric-tez scalding-fabric-flink" + script: "scripts/run_test.sh" + + - scala: 2.11.7 + env: BUILD="base" TEST_TARGET="scalding-core-fabric-tests scalding-fabric-hadoop scalding-fabric-hadoop2-mr1 scalding-fabric-tez scalding-fabric-flink" + script: "scripts/run_test.sh" + - scala: 2.10.6 env: BUILD="base" TEST_TARGET="scalding-hadoop-test" script: "scripts/run_test.sh" @@ -80,6 +88,78 @@ matrix: - "scripts/build_assembly_no_test.sh scalding-assembly" - "scripts/test_matrix_tutorials.sh" + - scala: 2.10.6 + env: BUILD="test tutorials and matrix tutorials and repl" TEST_TARGET="scalding-repl" SCALD_FABRIC_SUFFIX="hadoop" + script: + - "scripts/run_test.sh" + - "scripts/build_assembly_no_test.sh scalding-assembly-${SCALD_FABRIC_SUFFIX}" + - "scripts/test_tutorials.sh" + - "scripts/build_assembly_no_test.sh scalding-assembly-${SCALD_FABRIC_SUFFIX}" + - "scripts/test_matrix_tutorials.sh" + + - scala: 2.11.7 + env: BUILD="test tutorials and matrix tutorials and repl" TEST_TARGET="scalding-repl" SCALD_FABRIC_SUFFIX="hadoop" + script: + - "scripts/run_test.sh" + - "scripts/build_assembly_no_test.sh scalding-assembly-${SCALD_FABRIC_SUFFIX}" + - "scripts/test_tutorials.sh" + - "scripts/build_assembly_no_test.sh scalding-assembly-${SCALD_FABRIC_SUFFIX}" + - "scripts/test_matrix_tutorials.sh" + + - scala: 2.10.6 + env: BUILD="test tutorials and matrix tutorials and repl" TEST_TARGET="scalding-repl" SCALD_FABRIC_SUFFIX="hadoop2-mr1" + script: + - "scripts/run_test.sh" + - "scripts/build_assembly_no_test.sh scalding-assembly-${SCALD_FABRIC_SUFFIX}" + - "scripts/test_tutorials.sh" + - "scripts/build_assembly_no_test.sh scalding-assembly-${SCALD_FABRIC_SUFFIX}" + - "scripts/test_matrix_tutorials.sh" + + - scala: 2.11.7 + env: BUILD="test tutorials and matrix tutorials and repl" TEST_TARGET="scalding-repl" SCALD_FABRIC_SUFFIX="hadoop2-mr1" + script: + - "scripts/run_test.sh" + - "scripts/build_assembly_no_test.sh scalding-assembly-${SCALD_FABRIC_SUFFIX}" + - "scripts/test_tutorials.sh" + - "scripts/build_assembly_no_test.sh scalding-assembly-${SCALD_FABRIC_SUFFIX}" + - "scripts/test_matrix_tutorials.sh" + + - scala: 2.10.6 + env: BUILD="test tutorials and matrix tutorials and repl" TEST_TARGET="scalding-repl" SCALD_FABRIC_SUFFIX="tez" + script: + - "scripts/run_test.sh" + - "scripts/build_assembly_no_test.sh scalding-assembly-${SCALD_FABRIC_SUFFIX}" + - "scripts/test_tutorials.sh" + - "scripts/build_assembly_no_test.sh scalding-assembly-${SCALD_FABRIC_SUFFIX}" + - "scripts/test_matrix_tutorials.sh" + + - scala: 2.11.7 + env: BUILD="test tutorials and matrix tutorials and repl" TEST_TARGET="scalding-repl" SCALD_FABRIC_SUFFIX="tez" + script: + - "scripts/run_test.sh" + - "scripts/build_assembly_no_test.sh scalding-assembly-${SCALD_FABRIC_SUFFIX}" + - "scripts/test_tutorials.sh" + - "scripts/build_assembly_no_test.sh scalding-assembly-${SCALD_FABRIC_SUFFIX}" + - "scripts/test_matrix_tutorials.sh" + + - scala: 2.10.6 + env: BUILD="test tutorials and matrix tutorials and repl" TEST_TARGET="scalding-repl" SCALD_FABRIC_SUFFIX="flink" + script: + - "scripts/run_test.sh" + - "scripts/build_assembly_no_test.sh scalding-assembly-${SCALD_FABRIC_SUFFIX}" + - "scripts/test_tutorials.sh" + - "scripts/build_assembly_no_test.sh scalding-assembly-${SCALD_FABRIC_SUFFIX}" + - "scripts/test_matrix_tutorials.sh" + + - scala: 2.11.7 + env: BUILD="test tutorials and matrix tutorials and repl" TEST_TARGET="scalding-repl" SCALD_FABRIC_SUFFIX="flink" + script: + - "scripts/run_test.sh" + - "scripts/build_assembly_no_test.sh scalding-assembly-${SCALD_FABRIC_SUFFIX}" + - "scripts/test_tutorials.sh" + - "scripts/build_assembly_no_test.sh scalding-assembly-${SCALD_FABRIC_SUFFIX}" + - "scripts/test_matrix_tutorials.sh" + - scala: 2.10.6 env: BUILD="test repl and typed tutorials" script: @@ -99,3 +179,4 @@ matrix: - "scripts/test_typed_tutorials.sh" - "scripts/build_assembly_no_test.sh execution-tutorial" - "scripts/test_execution_tutorial.sh" + diff --git a/build.sbt b/build.sbt index ede97bc1ba..0b807737a4 100644 --- a/build.sbt +++ b/build.sbt @@ -5,11 +5,12 @@ import com.typesafe.sbt.SbtScalariform._ import com.typesafe.tools.mima.plugin.MimaKeys._ import com.typesafe.tools.mima.plugin.MimaPlugin.mimaDefaultSettings import sbtassembly.Plugin._ + import scala.collection.JavaConverters._ import scalariform.formatter.preferences._ import scalding._ - import ScroogeSBT.autoImport._ +import sbt.Keys._ def scalaBinaryVersion(scalaVersion: String) = scalaVersion match { case version if version startsWith "2.10" => "2.10" @@ -24,6 +25,11 @@ val apacheCommonsVersion = "2.2" val avroVersion = "1.7.4" val bijectionVersion = "0.9.1" val cascadingAvroVersion = "2.1.2" + +val flinkVersion = "1.1.3" // 1.1.3 not compatible with cascading-flink 0.3.0 which isn't compatible with cascading 3.2-wip +val cascadingFlinkVersion = "0.4.0-SNAPSHOT" // definitely not ready for prime time while a -SNAPSHOT + +val tezVersion = "0.8.2" val chillVersion = "0.7.3" val elephantbirdVersion = "4.14" val hadoopLzoVersion = "0.4.19" @@ -40,14 +46,24 @@ val scalaCheckVersion = "1.12.2" val scalaTestVersion = "2.2.6" val scalameterVersion = "0.6" val scroogeVersion = "3.20.0" -val slf4jVersion = "1.6.6" -val thriftVersion = "0.5.0" +val slf4jVersion = "1.7.7" +val thriftVersion = "0.7.0" val junitVersion = "4.10" val junitInterfaceVersion = "0.11" val macroCompatVersion = "1.1.1" +val guavaVersion = "14.0.1" // caution: guava [16.0,) is incompatible with hadoop-commons 2.6.0 +val mockitoVersion = "1.8.5" val printDependencyClasspath = taskKey[Unit]("Prints location of the dependencies") +lazy val testLibraryDependencies = Seq( + "org.mockito" % "mockito-all" % mockitoVersion % "test", + "org.scalacheck" %% "scalacheck" % scalaCheckVersion % "test", + "org.scalatest" %% "scalatest" % scalaTestVersion % "test", + "org.slf4j" % "slf4j-log4j12" % slf4jVersion % "test", + "com.novocode" % "junit-interface" % junitInterfaceVersion % "test" +) + val sharedSettings = Project.defaultSettings ++ assemblySettings ++ scalariformSettings ++ Seq( organization := "com.twitter", @@ -63,13 +79,7 @@ val sharedSettings = Project.defaultSettings ++ assemblySettings ++ scalariformS wartremoverErrors in (Compile, compile) += Wart.OptionPartial, - libraryDependencies ++= Seq( - "org.mockito" % "mockito-all" % "1.8.5" % "test", - "org.scalacheck" %% "scalacheck" % scalaCheckVersion % "test", - "org.scalatest" %% "scalatest" % scalaTestVersion % "test", - "org.slf4j" % "slf4j-log4j12" % slf4jVersion % "test", - "com.novocode" % "junit-interface" % junitInterfaceVersion % "test" - ), + libraryDependencies ++= testLibraryDependencies, resolvers ++= Seq( "Local Maven Repository" at "file://" + Path.userHome.absolutePath + "/.m2/repository", @@ -228,6 +238,10 @@ lazy val scalding = Project( scaldingArgs, scaldingDate, scaldingCore, + scaldingFabricHadoop, + scaldingFabricHadoop2Mr1, + scaldingFabricTez, + // scaldingFabricFlink, // not yet ready for prime time scaldingCommons, scaldingAvro, scaldingParquet, @@ -246,31 +260,46 @@ lazy val scalding = Project( scaldingThriftMacros ) -lazy val scaldingAssembly = Project( - id = "scalding-assembly", - base = file("assembly"), - settings = sharedSettings -).settings( - test := {}, - publish := {}, // skip publishing for this root project. - publishLocal := {} -).aggregate( - scaldingArgs, - scaldingDate, - scaldingCore, - scaldingCommons, - scaldingAvro, - scaldingParquet, - scaldingParquetCascading, - scaldingParquetScrooge, - scaldingParquetScroogeCascading, - scaldingHRaven, - scaldingRepl, - scaldingJson, - scaldingJdbc, - maple, - scaldingSerialization -) +def scaldingAssemblyOfFabric(fabricName: Option[String], fabricProject: Project): Project = + Project( + id = "scalding-assembly" + fabricName.map("-"+_).getOrElse(""), + base = file("assembly"), + settings = sharedSettings + ).settings( + target := baseDirectory.value / ("target" + fabricName.map("-"+_).getOrElse("")), + test := {}, + publish := {}, // skip publishing for this root project. + publishLocal := {} + ).aggregate( + scaldingArgs, + scaldingDate, + scaldingCore, + scaldingFabricHadoop, + scaldingCommons, + scaldingAvro, + scaldingParquet, + scaldingParquetCascading, + scaldingParquetScrooge, + scaldingParquetScroogeCascading, + scaldingHRaven, + scaldingRepl, + scaldingJson, + scaldingJdbc, + maple, + scaldingSerialization, + fabricProject + ) + +lazy val scaldingAssembly = scaldingAssemblyOfFabric(None, defaultScaldingFabric) + +lazy val scaldingAssemblyHadoop = scaldingAssemblyOfFabric(Some("hadoop"), scaldingFabricHadoop) + +lazy val scaldingAssemblyHadoop2Mr1 = scaldingAssemblyOfFabric(Some("hadoop2-mr1"), scaldingFabricHadoop2Mr1) + +lazy val scaldingAssemblyTez = scaldingAssemblyOfFabric(Some("tez"), scaldingFabricTez) + +lazy val scaldingAssemblyFlink = scaldingAssemblyOfFabric(Some("flink"), scaldingFabricFlink) + lazy val formattingPreferences = { import scalariform.formatter.preferences._ @@ -319,11 +348,20 @@ lazy val scaldingBenchmarks = module("benchmarks").settings( parallelExecution in Test := false ).dependsOn(scaldingCore) +lazy val providedCascadingHadoopDependencies = Seq( + /* the following two dependencies are "provided", and are here to provide Hfs and Configuration: */ + "cascading" % "cascading-hadoop" % cascadingVersion % "provided", // TODO: investigate whether can get rid of this. Perhaps depend on cascading-hadoop2-io (common to all) instead ? + "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided" + exclude("com.google.guava", "guava") + ) + + lazy val scaldingCore = module("core").settings( libraryDependencies <++= (scalaVersion) { scalaVersion => Seq( "cascading" % "cascading-core" % cascadingVersion, - "cascading" % "cascading-hadoop" % cascadingVersion, - "cascading" % "cascading-local" % cascadingVersion, + "cascading" % "cascading-local" % cascadingVersion + exclude("com.google.guava", "guava"), + "com.google.guava" % "guava" % guavaVersion % "provided", "com.twitter" % "chill-hadoop" % chillVersion, "com.twitter" % "chill-java" % chillVersion, "com.twitter" %% "chill-bijection" % chillVersion, @@ -333,15 +371,86 @@ lazy val scaldingCore = module("core").settings( "com.twitter" %% "bijection-macros" % bijectionVersion, "com.twitter" %% "chill" % chillVersion, "com.twitter" %% "chill-algebird" % chillVersion, - "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided" - exclude("com.google.guava", "guava"), + "org.scala-lang" % "scala-library" % scalaVersion, "org.scala-lang" % "scala-reflect" % scalaVersion, "org.slf4j" % "slf4j-api" % slf4jVersion, "org.slf4j" % "slf4j-log4j12" % slf4jVersion % "provided") ++ + providedCascadingHadoopDependencies ++ + (if (isScala210x(scalaVersion)) Seq("org.scalamacros" %% "quasiquotes" % quasiquotesVersion) else Seq()) + }, addCompilerPlugin("org.scalamacros" % "paradise" % paradiseVersion cross CrossVersion.full) +) .dependsOn(scaldingArgs, scaldingDate, scaldingSerialization, maple) + +lazy val scaldingCoreFabricTests = module("core-fabric-tests").settings( + libraryDependencies <++= (scalaVersion) { scalaVersion => + testLibraryDependencies.map (m => m.copy(configurations = None)) ++ + providedCascadingHadoopDependencies ++ (if (isScala210x(scalaVersion)) Seq("org.scalamacros" %% "quasiquotes" % quasiquotesVersion) else Seq()) }, addCompilerPlugin("org.scalamacros" % "paradise" % paradiseVersion cross CrossVersion.full) -).dependsOn(scaldingArgs, scaldingDate, scaldingSerialization, maple) +).dependsOn(scaldingCore, + scaldingCore % "test->compile", + scaldingCore % "test->test", + scaldingCore % "compile->test" ) + + +lazy val scaldingFabricHadoop = module("fabric-hadoop").settings( + libraryDependencies <++= (scalaVersion) { scalaVersion => Seq( + "org.apache.hadoop" % "hadoop-client" % hadoopVersion + exclude("com.google.guava", "guava"), + "com.google.guava" % "guava" % guavaVersion, + "cascading" % "cascading-hadoop" % cascadingVersion) ++ + (if (isScala210x(scalaVersion)) Seq("org.scalamacros" %% "quasiquotes" % quasiquotesVersion) else Seq()) + }, addCompilerPlugin("org.scalamacros" % "paradise" % paradiseVersion cross CrossVersion.full), + + unmanagedSourceDirectories in Test += baseDirectory.value / ".." / "scalding-core-fabric-tests" / "src" / "fabric" / "scala" + +).dependsOn(scaldingCore, scaldingCoreFabricTests % "test->compile,test->test,compile->test") + + +lazy val scaldingFabricHadoop2Mr1 = module("fabric-hadoop2-mr1").settings( + libraryDependencies <++= (scalaVersion) { scalaVersion => Seq( + "org.apache.hadoop" % "hadoop-client" % hadoopVersion + exclude("com.google.guava", "guava"), + "com.google.guava" % "guava" % guavaVersion, + "cascading" % "cascading-hadoop2-mr1" % cascadingVersion) ++ + (if (isScala210x(scalaVersion)) Seq("org.scalamacros" %% "quasiquotes" % quasiquotesVersion) else Seq()) + }, addCompilerPlugin("org.scalamacros" % "paradise" % paradiseVersion cross CrossVersion.full), + unmanagedSourceDirectories in Test += baseDirectory.value / ".." / "scalding-core-fabric-tests" / "src" / "fabric" / "scala" +).dependsOn(scaldingCore, scaldingCoreFabricTests % "compile->test") + +lazy val scaldingFabricTez = module("fabric-tez").settings( + libraryDependencies <++= (scalaVersion) { scalaVersion => Seq( + "com.google.guava" % "guava" % guavaVersion) ++ + Seq( + "org.apache.hadoop" % "hadoop-client" % hadoopVersion, + "org.apache.tez" % "tez-api" % tezVersion, + "org.apache.tez" % "tez-mapreduce" % tezVersion, + "org.apache.tez" % "tez-dag" % tezVersion, + "cascading" % "cascading-hadoop2-tez" % cascadingVersion + ).map(_.exclude("com.google.guava", "guava")) ++ + (if (isScala210x(scalaVersion)) Seq("org.scalamacros" %% "quasiquotes" % quasiquotesVersion) else Seq()) + }, addCompilerPlugin("org.scalamacros" % "paradise" % paradiseVersion cross CrossVersion.full), + unmanagedSourceDirectories in Test += baseDirectory.value / ".." / "scalding-core-fabric-tests" / "src" / "fabric" / "scala" +).dependsOn(scaldingCore, scaldingCoreFabricTests % "compile->test" ) + +lazy val scaldingFabricFlink = module("fabric-flink").settings( + libraryDependencies <++= (scalaVersion) { scalaVersion => Seq( + "com.google.guava" % "guava" % guavaVersion, + "com.data-artisans" % "cascading-flink" % cascadingFlinkVersion + exclude ("org.clapper", "grizzled-slf4j_2.10") + exclude ("org.clapper", "grizzled-slf4j_2.11") + exclude ("org.clapper", "grizzled-slf4j_2.12") + exclude("org.apache.flink", "flink-clients_2.10") + exclude("org.apache.flink", "flink-clients_2.11") + exclude("org.apache.flink", "flink-clients_2.12"), + "org.apache.flink" %% "flink-clients" % flinkVersion // cascading-flink, written in java, depends on flink-clients which is written in scala. + ) ++ + (if (isScala210x(scalaVersion)) Seq("org.scalamacros" %% "quasiquotes" % quasiquotesVersion) else Seq()) + }, addCompilerPlugin("org.scalamacros" % "paradise" % paradiseVersion cross CrossVersion.full), + unmanagedSourceDirectories in Test += baseDirectory.value / ".." / "scalding-core-fabric-tests" / "src" / "fabric" / "scala" +).dependsOn(scaldingCore, scaldingCoreFabricTests % "compile->test") + +lazy val defaultScaldingFabric = scaldingFabricHadoop lazy val scaldingCommons = module("commons").settings( libraryDependencies ++= Seq( @@ -354,16 +463,15 @@ lazy val scaldingCommons = module("commons").settings( "com.twitter.elephantbird" % "elephant-bird-core" % elephantbirdVersion, "com.hadoop.gplcompression" % "hadoop-lzo" % hadoopLzoVersion, // TODO: split this out into scalding-thrift - "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided" - exclude("com.google.guava", "guava"), "org.apache.thrift" % "libthrift" % thriftVersion, // TODO: split this out into a scalding-scrooge "com.twitter" %% "scrooge-serializer" % scroogeVersion % "provided", "org.slf4j" % "slf4j-api" % slf4jVersion, "org.slf4j" % "slf4j-log4j12" % slf4jVersion % "provided", "junit" % "junit" % junitVersion % "test" - ) -).dependsOn(scaldingArgs, scaldingDate, scaldingCore, scaldingHadoopTest % "test") + ) ++ providedCascadingHadoopDependencies +).dependsOn(scaldingArgs, scaldingDate, scaldingCore, scaldingHadoopTest % "test", + defaultScaldingFabric % "provided") lazy val scaldingAvro = module("avro").settings( libraryDependencies ++= Seq( @@ -412,30 +520,30 @@ lazy val scaldingParquetCascading = module("parquet-cascading").settings( exclude("com.twitter.elephantbird", "elephant-bird-pig") exclude("com.twitter.elephantbird", "elephant-bird-core"), "org.apache.thrift" % "libthrift" % thriftVersion, - "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided" - exclude("com.google.guava", "guava"), "cascading" % "cascading-core" % cascadingVersion % "provided", - "cascading" % "cascading-hadoop" % cascadingVersion % "provided", "com.twitter.elephantbird" % "elephant-bird-core" % elephantbirdVersion % "test" - ) -).dependsOn(scaldingParquetFixtures % "test->test") + ) ++ providedCascadingHadoopDependencies +).dependsOn(scaldingParquetFixtures % "test->test", + defaultScaldingFabric % "test,provided") lazy val scaldingParquet = module("parquet").settings( libraryDependencies <++= (scalaVersion) { scalaVersion => Seq( "org.apache.parquet" % "parquet-column" % parquetVersion, "org.apache.parquet" % "parquet-hadoop" % parquetVersion, "org.slf4j" % "slf4j-api" % slf4jVersion, - "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided" - exclude("com.google.guava", "guava"), "org.scala-lang" % "scala-compiler" % scalaVersion, "org.scala-lang" % "scala-reflect" % scalaVersion, "com.twitter" %% "bijection-macros" % bijectionVersion, "com.twitter" %% "chill-bijection" % chillVersion, "com.twitter.elephantbird" % "elephant-bird-core" % elephantbirdVersion % "test", "org.typelevel" %% "macro-compat" % macroCompatVersion - ) ++ (if(isScala210x(scalaVersion)) Seq("org.scalamacros" %% "quasiquotes" % quasiquotesVersion) else Seq()) + ) ++ providedCascadingHadoopDependencies ++ + (if(isScala210x(scalaVersion)) Seq("org.scalamacros" %% "quasiquotes" % quasiquotesVersion) else Seq()) }, addCompilerPlugin("org.scalamacros" % "paradise" % paradiseVersion cross CrossVersion.full)) - .dependsOn(scaldingCore, scaldingParquetCascading, scaldingHadoopTest % "test", scaldingParquetFixtures % "test->test") + .dependsOn(scaldingCore, scaldingParquetCascading, + scaldingHadoopTest % "test", // TODO: address possible discrepancy if default fabric != hadoop + scaldingParquetFixtures % "test->test", + defaultScaldingFabric % "test,provided") lazy val scaldingParquetScroogeFixtures = module("parquet-scrooge-fixtures") .settings(ScroogeSBT.newSettings:_*) @@ -469,17 +577,15 @@ lazy val scaldingParquetScroogeCascading = module("parquet-scrooge-cascading") libraryDependencies ++= Seq( // see https://issues.apache.org/jira/browse/PARQUET-143 for exclusions "cascading" % "cascading-core" % cascadingVersion % "provided", - "cascading" % "cascading-hadoop" % cascadingVersion % "test", "org.apache.parquet" % "parquet-thrift" % parquetVersion % "test" classifier "tests" exclude("org.apache.parquet", "parquet-pig") exclude("com.twitter.elephantbird", "elephant-bird-pig") exclude("com.twitter.elephantbird", "elephant-bird-core"), "com.twitter" %% "scrooge-serializer" % scroogeVersion, - "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided" - exclude("com.google.guava", "guava"), "junit" % "junit" % junitVersion % "test" - ) -).dependsOn(scaldingParquetCascading % "compile->compile;test->test", scaldingParquetScroogeFixtures % "test->test") + ) ++ providedCascadingHadoopDependencies + ).dependsOn(scaldingParquetCascading % "compile->compile;test->test", scaldingParquetScroogeFixtures % "test->test", + defaultScaldingFabric % "provided") lazy val scaldingParquetScrooge = module("parquet-scrooge") .settings( @@ -490,13 +596,12 @@ lazy val scaldingParquetScrooge = module("parquet-scrooge") exclude("com.twitter.elephantbird", "elephant-bird-pig") exclude("com.twitter.elephantbird", "elephant-bird-core"), "com.twitter" %% "scrooge-serializer" % scroogeVersion, - "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided" - exclude("com.google.guava", "guava"), "com.twitter.elephantbird" % "elephant-bird-core" % elephantbirdVersion % "test", "com.novocode" % "junit-interface" % junitInterfaceVersion % "test", "junit" % "junit" % junitVersion % "test" - ) -).dependsOn(scaldingCore, scaldingParquetScroogeCascading, scaldingParquet % "compile->compile;test->test", scaldingParquetScroogeFixtures % "test->test") + ) ++ providedCascadingHadoopDependencies +).dependsOn(scaldingCore, scaldingParquetScroogeCascading, scaldingParquet % "compile->compile;test->test", + scaldingParquetScroogeFixtures % "test->test", defaultScaldingFabric % "provided") lazy val scaldingHRaven = module("hraven").settings( libraryDependencies ++= Seq( @@ -508,6 +613,8 @@ lazy val scaldingHRaven = module("hraven").settings( exclude("com.sun.jdmk", "jmxtools") exclude("com.sun.jmx", "jmxri") + exclude("com.google.guava", "guava") + // These transitive dependencies of hRaven cause conflicts when // running scalding-hraven/*assembly and aren't needed // for the part of the hRaven API that we use anyway @@ -516,11 +623,9 @@ lazy val scaldingHRaven = module("hraven").settings( exclude("com.twitter.common", "args") exclude("com.twitter.common", "application"), "org.apache.hbase" % "hbase" % hbaseVersion, - "org.slf4j" % "slf4j-api" % slf4jVersion, - "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided" - exclude("com.google.guava", "guava") - ) -).dependsOn(scaldingCore) + "org.slf4j" % "slf4j-api" % slf4jVersion + ) ++ providedCascadingHadoopDependencies +).dependsOn(scaldingCore, defaultScaldingFabric % "provided") // create new configuration which will hold libs otherwise marked as 'provided' // so that we can re-include them in 'run'. unfortunately, we still have to @@ -542,8 +647,7 @@ lazy val scaldingRepl = module("repl") "org.scala-lang" % "scala-reflect" % scalaVersion, "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided" exclude("com.google.guava", "guava"), - "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "unprovided" - exclude("com.google.guava", "guava"), + "com.google.guava" % "guava" % guavaVersion, "org.slf4j" % "slf4j-api" % slf4jVersion, "org.slf4j" % "slf4j-log4j12" % slf4jVersion % "provided", "org.slf4j" % "slf4j-log4j12" % slf4jVersion % "unprovided" @@ -551,7 +655,7 @@ lazy val scaldingRepl = module("repl") }, // https://gist.github.com/djspiewak/976cd8ac65e20e136f05 unmanagedSourceDirectories in Compile += (sourceDirectory in Compile).value / s"scala-${scalaBinaryVersion(scalaVersion.value)}" -).dependsOn(scaldingCore) +).dependsOn(scaldingCore, defaultScaldingFabric % "provided") // run with 'unprovided' config includes libs marked 'unprovided' in classpath .settings(inConfig(Unprovided)(Classpaths.configSettings ++ Seq( run <<= Defaults.runTask(fullClasspath, mainClass in (Runtime, run), runner in (Runtime, run)) @@ -572,34 +676,29 @@ addCompilerPlugin("org.scalamacros" % "paradise" % "2.0.1" cross CrossVersion.fu lazy val scaldingJson = module("json").settings( libraryDependencies <++= (scalaVersion) { scalaVersion => Seq( - "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided" - exclude("com.google.guava", "guava"), "com.fasterxml.jackson.module" %% "jackson-module-scala" % jacksonVersion, "org.json4s" %% "json4s-native" % json4SVersion, "com.twitter.elephantbird" % "elephant-bird-cascading3" % elephantbirdVersion % "provided" - ) + ) ++ providedCascadingHadoopDependencies } ).dependsOn(scaldingCore) lazy val scaldingJdbc = module("jdbc").settings( - libraryDependencies <++= (scalaVersion) { scalaVersion => Seq( - "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided" - exclude("com.google.guava", "guava"), + libraryDependencies ++= Seq( "cascading" % "cascading-jdbc-core" % cascadingJDBCVersion, "cascading" % "cascading-jdbc-mysql" % cascadingJDBCVersion - ) - } -).dependsOn(scaldingCore) + ) ++ providedCascadingHadoopDependencies +) .dependsOn(scaldingCore) lazy val scaldingHadoopTest = module("hadoop-test").settings( libraryDependencies <++= (scalaVersion) { scalaVersion => Seq( - "org.apache.hadoop" % "hadoop-client" % hadoopVersion + "org.apache.hadoop" % "hadoop-minicluster" % hadoopVersion exclude("com.google.guava", "guava"), - "org.apache.hadoop" % "hadoop-minicluster" % hadoopVersion, "org.apache.hadoop" % "hadoop-yarn-server-tests" % hadoopVersion classifier "tests", "org.apache.hadoop" % "hadoop-yarn-server" % hadoopVersion, "org.apache.hadoop" % "hadoop-hdfs" % hadoopVersion classifier "tests", - "org.apache.hadoop" % "hadoop-common" % hadoopVersion classifier "tests", + "org.apache.hadoop" % "hadoop-common" % hadoopVersion classifier "tests" + exclude("com.google.guava", "guava"), "org.apache.hadoop" % "hadoop-mapreduce-client-jobclient" % hadoopVersion classifier "tests", "org.scala-lang" % "scala-compiler" % scalaVersion, "com.twitter" %% "chill-algebird" % chillVersion, @@ -607,9 +706,14 @@ lazy val scaldingHadoopTest = module("hadoop-test").settings( "org.slf4j" % "slf4j-log4j12" % slf4jVersion, "org.scalacheck" %% "scalacheck" % scalaCheckVersion, "org.scalatest" %% "scalatest" % scalaTestVersion + + /* note: here, we depend on the Hadoop stack and provide a working HDFS + YARN minicluster. This doesn't + bring any assumption on the fabric ("hopefully") + */ ) } -).dependsOn(scaldingCore, scaldingSerialization) +).dependsOn(scaldingCore, scaldingSerialization, + defaultScaldingFabric % "provided") // consider always providing Hadoop2MR1 here // This one uses a different naming convention lazy val maple = Project( @@ -626,11 +730,13 @@ lazy val maple = Project( if(scalaVersion.startsWith("2.10")) false else true }, libraryDependencies <++= (scalaVersion) { scalaVersion => Seq( + /* we must not depend on defaultScaldingFabric here (loop), but we need the symbols to compile */ "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided" exclude("com.google.guava", "guava"), - "org.apache.hbase" % "hbase" % hbaseVersion % "provided", - "cascading" % "cascading-hadoop" % cascadingVersion % "provided" - ) + "cascading" % "cascading-local" % cascadingVersion % "provided", + "cascading" % "cascading-hadoop" % cascadingVersion % "provided", + + "org.apache.hbase" % "hbase" % hbaseVersion % "provided") } ) @@ -643,14 +749,11 @@ lazy val executionTutorial = Project( libraryDependencies <++= (scalaVersion) { scalaVersion => Seq( "org.scala-lang" % "scala-library" % scalaVersion, "org.scala-lang" % "scala-reflect" % scalaVersion, - "org.apache.hadoop" % "hadoop-client" % hadoopVersion - exclude("com.google.guava", "guava"), "org.slf4j" % "slf4j-api" % slf4jVersion, - "org.slf4j" % "slf4j-log4j12" % slf4jVersion, - "cascading" % "cascading-hadoop" % cascadingVersion - ) + "org.slf4j" % "slf4j-log4j12" % slf4jVersion + ) ++ providedCascadingHadoopDependencies } -).dependsOn(scaldingCore) +).dependsOn(scaldingCore, defaultScaldingFabric % "provided" ) lazy val scaldingDb = module("db").settings( libraryDependencies <++= (scalaVersion) { scalaVersion => Seq( @@ -685,17 +788,20 @@ lazy val scaldingThriftMacros = module("thrift-macros") "org.apache.thrift" % "libthrift" % thriftVersion, "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "test" exclude("com.google.guava", "guava"), - "org.apache.hadoop" % "hadoop-minicluster" % hadoopVersion % "test", + "org.apache.hadoop" % "hadoop-minicluster" % hadoopVersion % "test" + exclude("com.google.guava", "guava"), "org.apache.hadoop" % "hadoop-yarn-server-tests" % hadoopVersion classifier "tests", "org.apache.hadoop" % "hadoop-yarn-server" % hadoopVersion % "test", "org.apache.hadoop" % "hadoop-hdfs" % hadoopVersion classifier "tests", - "org.apache.hadoop" % "hadoop-common" % hadoopVersion classifier "tests", + "org.apache.hadoop" % "hadoop-common" % hadoopVersion classifier "tests" + exclude("com.google.guava", "guava"), "org.apache.hadoop" % "hadoop-mapreduce-client-jobclient" % hadoopVersion classifier "tests" ) ++ (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 % "test", + defaultScaldingFabric % "provided" , + scaldingHadoopTest % "test", /* TODO isn"t scaldingHadoopTest already bringing in all the dependencies above? */ scaldingSerialization, scaldingThriftMacrosFixtures % "test->test") diff --git a/project/plugins.sbt b/project/plugins.sbt index 209beaa1a9..7e56903520 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -19,3 +19,5 @@ addSbtPlugin("org.scoverage" % "sbt-scoverage" % "1.0.4") addSbtPlugin("org.xerial.sbt" % "sbt-sonatype" % "1.0") addSbtPlugin("pl.project13.scala" % "sbt-jmh" % "0.2.2") addSbtPlugin("org.brianmckenna" % "sbt-wartremover" % "0.14") + +addSbtPlugin("io.get-coursier" % "sbt-coursier" % "1.0.0-M14") diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/extensions/Checkpoint.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/extensions/Checkpoint.scala index 941387497c..36ff05f822 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/extensions/Checkpoint.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/extensions/Checkpoint.scala @@ -156,8 +156,8 @@ object Checkpoint { // Returns a format for the checkpoint. The format of the source is // determined by the flag --checkpoint.format, and defaults to SequenceFile. private def getFormat(checkpointName: String)(implicit args: Args, mode: Mode): String = { - val defaultFormat = mode match { - case Hdfs(_, _) | HadoopTest(_, _) => "sequencefile" + val defaultFormat = mode.storageMode match { + case _: HdfsStorageModeCommon => "sequencefile" case _ => "tsv" } CheckpointArg(checkpointName, "format").value.getOrElse(defaultFormat).toLowerCase diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoCodecSource.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoCodecSource.scala index f7b1e9b0c9..d75d0751df 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoCodecSource.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoCodecSource.scala @@ -27,8 +27,8 @@ import com.twitter.bijection.Injection object LzoCodecSource { def apply[T](paths: String*)(implicit passedInjection: Injection[T, Array[Byte]]) = new LzoCodec[T] { - val hdfsPaths = paths - val localPaths = paths + override val hdfsPaths = paths + override val localPaths = paths val boxed = Externalizer(passedInjection) override def injection = boxed.get } 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 def9bc1673..7a7db5a0b5 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 @@ -82,12 +82,12 @@ trait LzoText extends LocalTapSource with SingleMappable[String] with TypedSink[ } trait LzoTsv extends DelimitedScheme with LocalTapSource { - override def hdfsScheme = HadoopSchemeInstance((new LzoTextDelimited(fields, skipHeader, writeHeader, separator, strict, quote, types, safe)).asInstanceOf[Scheme[_, _, _, _, _]]) + override def hdfsScheme = HadoopSchemeInstance(new LzoTextDelimited(fields, skipHeader, writeHeader, separator, strict, quote, types, safe)) } 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, skipHeader, writeHeader, separator, strict, quote, types, safe)).asInstanceOf[Scheme[_, _, _, _, _]]) + override def hdfsScheme = HadoopSchemeInstance(new LzoTextDelimited(fields, skipHeader, writeHeader, separator, strict, quote, types, safe)) def mf: Manifest[T] diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/TsvWithHeader.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/TsvWithHeader.scala index e81fb19093..e41b4604fd 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/TsvWithHeader.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/TsvWithHeader.scala @@ -38,7 +38,7 @@ class TsvWithHeader(p: String, f: Fields = Fields.UNKNOWN)(implicit mode: Mode) // make it lazy so as to only do once lazy val fieldsFromHeaderFile = { - val names = readFromFile(headerPath) + val names = mode.storageMode.readFromFile(headerPath) .split("\t") .toSeq new Fields(names: _*) @@ -50,72 +50,17 @@ class TsvWithHeader(p: String, f: Fields = Fields.UNKNOWN)(implicit mode: Mode) f } - // TODO: move this method to make it a util function. - def readFromFile(filename: String)(implicit mode: Mode) = { - mode match { - case Hdfs(_, conf) => { - try { - val pt = new Path(filename) - val fs = pt.getFileSystem(conf) - fs.open(pt).readUTF - } catch { - case e: IOException => { - throw new RuntimeException(e) - } - } - } - // Local mode - case _ => { - try { - Files.toString(new File(filename), Charsets.UTF_8) - } catch { - case e: IOException => { - throw new RuntimeException(e) - } - } - } - } - } - - // TODO: move this method to make it a util function. - def writeToFile(filename: String, text: String)(implicit mode: Mode): Unit = { - mode match { - case Hdfs(_, conf) => { - try { - val pt = new Path(filename) - val fs = pt.getFileSystem(conf) - val br = new BufferedWriter(new OutputStreamWriter(fs.create(pt, true))) + @deprecated("please use mode.storageMode.readFromFile", "0.17.0") + def readFromFile(filename: String)(implicit mode: Mode): String = mode.storageMode.readFromFile(filename) - br.write(text) - br.close() - } catch { - case e: IOException => { - throw new RuntimeException(e) - } - } - } - // Local mode - case _ => { - try { - val br = new BufferedWriter( - new OutputStreamWriter(new FileOutputStream(filename), "utf-8")) - - br.write(text) - br.close() - } catch { - case e: IOException => { - throw new RuntimeException(e) - } - } - } - } - } + @deprecated("please use mode.storageMode.readFromFile", "0.17.0") + def writeToFile(filename: String, text: String)(implicit mode: Mode): Unit = mode.storageMode.writeToFile(filename, text) override def writeFrom(pipe: Pipe)(implicit flowDef: FlowDef, mode: Mode) = { val ret = super.writeFrom(pipe)(flowDef, mode) val fieldNames = for (i <- (0 until fields.size)) yield fields.get(i).asInstanceOf[String] val headerFileText = fieldNames.mkString("\t") - writeToFile(headerPath, headerFileText) + mode.storageMode.writeToFile(headerPath, headerFileText) ret } } diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/VersionedKeyValSource.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/VersionedKeyValSource.scala index 51f10ae03c..7ba7e877dc 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/VersionedKeyValSource.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/VersionedKeyValSource.scala @@ -94,67 +94,66 @@ class VersionedKeyValSource[K, V](val path: String, val sourceVersion: Option[Lo override def validateTaps(mode: Mode): Unit = { // if a version is explicitly supplied, ensure that it exists sourceVersion.foreach { version => - mode match { - case hadoopMode: HadoopMode => { - val store = source.getStore(new JobConf(hadoopMode.jobConf)) + mode.storageMode match { + case hdfs: HdfsStorageModeCommon => + val store = source.getStore(new JobConf(hdfs.jobConf)) if (!store.hasVersion(version)) { throw new InvalidSourceException( - "Version %s does not exist. Currently available versions are: %s" - .format(version, store.getAllVersions)) + s"Version ${version} does not exist. Currently available versions are: ${store.getAllVersions}") } - } case _ => throw new IllegalArgumentException( - "VersionedKeyValSource does not support mode %s. Only HadoopMode is supported" - .format(mode)) + s"VersionedKeyValSource does not support mode ${mode}. Only HadoopMode is supported") } } } def resourceExists(mode: Mode): Boolean = mode match { - case Test(buffers) => { - buffers(this) map { !_.isEmpty } getOrElse false - } - case HadoopTest(conf, buffers) => { - buffers(this) map { !_.isEmpty } getOrElse false - } - case _ => { - val conf = new JobConf(mode.asInstanceOf[HadoopMode].jobConf) + case testMode: TestMode => + testMode.buffers(this).exists(_.nonEmpty) + + case h: HadoopFamilyMode => + val conf = new JobConf(h.jobConf) source.resourceExists(conf) - } + + case _ => throw new IllegalArgumentException( + s"VersionedKeyValSource does not support mode ${mode}. Only HadoopMode is supported") } def sinkExists(mode: Mode): Boolean = sinkVersion match { case Some(version) => mode match { - case Test(buffers) => - buffers(this) map { !_.isEmpty } getOrElse false - - case HadoopTest(conf, buffers) => - buffers(this) map { !_.isEmpty } getOrElse false + case testMode: TestMode => + testMode.buffers(this).exists(_.nonEmpty) - case m: HadoopMode => + case m: HadoopFamilyMode => val conf = new JobConf(m.jobConf) val store = sink.getStore(conf) store.hasVersion(version) - case _ => sys.error(s"Unknown mode $mode") + + case _ => throw new IllegalArgumentException( + s"VersionedKeyValSource does not support mode ${mode}. Only HadoopMode is supported") } case None => false } override def createTap(readOrWrite: AccessMode)(implicit mode: Mode): Tap[_, _, _] = { import com.twitter.scalding.CastHfsTap - mode match { - case Hdfs(_strict, _config) => + + (mode, mode.storageMode) match { + case (_, _: HdfsStorageModeCommon) => readOrWrite match { case Read => CastHfsTap(source) case Write => CastHfsTap(sink) } - case _ => - TestTapFactory(this, hdfsScheme).createTap(readOrWrite) + + case (testMode: TestMode, _: LocalStorageModeCommon) => + TestTapFactory(this, fields).createLocalTap(readOrWrite: AccessMode, testMode) + + case _ => ??? // TODO: support Local mode here, and better testing. } } diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/examples/WeightedPageRank.scala b/scalding-commons/src/main/scala/com/twitter/scalding/examples/WeightedPageRank.scala index c32f07d501..46eeeee2f9 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/examples/WeightedPageRank.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/examples/WeightedPageRank.scala @@ -82,19 +82,28 @@ class WeightedPageRank(args: Args) extends Job(args) { } } + /** + * Build a source of the appropriate type based on the operation mode (see also the comment in {@link WeightedPageRankSpec}) + */ + protected def nodeSource(fileName: String): Source = + mode.storageMode match { + case _: HdfsTestStorageMode => Tsv(fileName) // special for the benefit of WeightedPageRankSpec + case _: HdfsStorageModeCommon => SequenceFile(fileName) // any other HDFS + case _ => Tsv(fileName) + } + /** * read the pregenerated nodes file <'src_id, 'dst_ids, 'weights, 'mass_prior> */ def getNodes(fileName: String) = { - mode match { - case Hdfs(_, conf) => { - SequenceFile(fileName).read - .mapTo((0, 1, 2, 3) -> ('src_id, 'dst_ids, 'weights, 'mass_prior)) { - input: (Int, Array[Int], Array[Float], Double) => input - } + nodeSource(fileName) match { + case source: SequenceFile => { // or any type that directly support Array[Int] as a field type + source.read.mapTo((0, 1, 2, 3) -> ('src_id, 'dst_ids, 'weights, 'mass_prior)) { + input: (Int, Array[Int], Array[Float], Double) => input + } } - case _ => { - Tsv(fileName).read + case source: DelimitedScheme => { + source.read .mapTo((0, 1, 2, 3) -> ('src_id, 'dst_ids, 'weights, 'mass_prior)) { input: (Int, String, String, Double) => { @@ -116,6 +125,7 @@ class WeightedPageRank(args: Args) extends Job(args) { } } } + case _ => ??? // TODO: implement other types (or add recognition in the cases above) } } diff --git a/scalding-commons/src/test/scala/com/twitter/scalding/WeightedPageRankTest.scala b/scalding-commons/src/test/scala/com/twitter/scalding/WeightedPageRankTest.scala index d215b03d77..c2919a8e80 100644 --- a/scalding-commons/src/test/scala/com/twitter/scalding/WeightedPageRankTest.scala +++ b/scalding-commons/src/test/scala/com/twitter/scalding/WeightedPageRankTest.scala @@ -20,12 +20,19 @@ import org.scalatest.{ Matchers, WordSpec } class WeightedPageRankSpec extends WordSpec with Matchers { "Weighted PageRank job" should { 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))) + /* NOTE: in Hadoop mode, WeightedPageRank NORMALLY operates over SequenceFiles, not Tsv. However, we cannot + access the mode at this time (as it's decided by the "useHadoop = true" flag below), so we have to pick a + source type. We choose Tsv, and we expect WeightedPageRank to notice that despite working in Hadoop mode, we + use a Tsv source + */ + .source(Tsv("./numnodes"), List((3))) .source(Tsv("./pagerank_0"), List((1, 0.086), (2, 0.192), (3, 0.722))) .typedSink(TypedTsv[Double]("./totaldiff")) { ob => 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 da9bd0290e..d09e382ff0 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 @@ -22,7 +22,7 @@ import com.twitter.scalding.commons.datastores.VersionedStore import com.twitter.scalding.typed.IterablePipe import com.twitter.bijection.Injection import com.google.common.io.Files -import org.apache.hadoop.mapred.{SequenceFileInputFormat, JobConf} +import org.apache.hadoop.mapred.{ SequenceFileInputFormat, JobConf } import java.io.File // Use the scalacheck generators @@ -157,7 +157,7 @@ class VersionedKeyValSourceTest extends WordSpec with Matchers { private def validateVersion(path: String, version: Option[Long] = None) = { val store = VersionedKeyValSource(path = path, sourceVersion = version) val conf: JobConf = new JobConf() - store.validateTaps(Hdfs(strict = false, conf)) + store.validateTaps(Mode(Args(Seq("--autoCluster", "--tool.partialok")), conf)) // also validate the paths for the version validateVersionPaths(path, version, store, conf) diff --git a/scalding-core/src/test/scala/com/twitter/scalding/CascadeTest.scala b/scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/CascadeTest.scala similarity index 100% rename from scalding-core/src/test/scala/com/twitter/scalding/CascadeTest.scala rename to scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/CascadeTest.scala diff --git a/scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/CoreTestFabric.scala b/scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/CoreTestFabric.scala new file mode 100644 index 0000000000..a51388275c --- /dev/null +++ b/scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/CoreTestFabric.scala @@ -0,0 +1,636 @@ +/* +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. +*/ + +/* split from the original CoreTest.scala */ +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 java.lang.{ Integer => JInt } + + +class NumberJoinerJob(args: Args) extends Job(args) { + val in0 = TypedTsv[(Int, Int)]("input0").read.rename((0, 1) -> ('x0, 'y0)) + val in1 = Tsv("input1").read.mapTo((0, 1) -> ('x1, 'y1)) { input: (Long, Long) => input } + in0.joinWithSmaller('x0 -> 'x1, in1) + .write(Tsv("output")) +} + +class NumberJoinTest extends WordSpec with Matchers { + import Dsl._ + "A NumberJoinerJob" should { + //Set up the job: + "not throw when joining longs with ints" in { + 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 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 + .finish() + } + } +} + +class SpillingJob(args: Args) extends Job(args) { + TypedTsv[(Int, Int)]("input").read.rename((0, 1) -> ('n, 'v)) + .groupBy('n) { group => + group.spillThreshold(3).sum[Int]('v).size + }.write(Tsv("output")) +} + +class SpillingTest extends WordSpec with Matchers { + import Dsl._ + "A SpillingJob" should { + val src = (0 to 9).map(_ -> 1) ++ List(0 -> 4) + val result = src.groupBy(_._1) + .mapValues { v => (v.map(_._2).sum, v.size) } + .map { case (a, (b, c)) => (a, b, c) } + .toSet + + //Set up the job: + "work when number of keys exceeds spill threshold" in { + JobTest(new SpillingJob(_)) + .source(TypedTsv[(Int, Int)]("input"), src) + .sink[(Int, Int, Int)](Tsv("output")) { outBuf => + outBuf.toSet shouldBe result + }.run + .runHadoop + .finish() + } + } +} + +class TinyJoinJob(args: Args) extends Job(args) { + val p1 = Tsv(args("input1")) + .read + .mapTo((0, 1) -> ('k1, 'v1)) { v: (String, Int) => v } + val p2 = Tsv(args("input2")) + .read + .mapTo((0, 1) -> ('k2, 'v2)) { v: (String, Int) => v } + p1.joinWithTiny('k1 -> 'k2, p2) + .project('k1, 'v1, 'v2) + .write(Tsv(args("output"))) +} + +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)) + var idx = 0 + JobTest(new TinyJoinJob(_)) + .arg("input1", "fakeInput1") + .arg("input2", "fakeInput2") + .arg("output", "fakeOutput") + .source(Tsv("fakeInput1"), input1) + .source(Tsv("fakeInput2"), input2) + .sink[(String, Int, Int)](Tsv("fakeOutput")) { outBuf => + val actualOutput = outBuf.map { + case (k: String, v1: Int, v2: Int) => + (k, (v1, v2)) + }.toMap + (idx + ": join tuples with the same key") in { + actualOutput shouldBe correctOutput + } + idx += 1 + } + .run + .runHadoop + .finish() + } +} + +class TinyThenSmallJoin(args: Args) extends Job(args) { + val pipe0 = Tsv("in0", ('x0, 'y0)).read + val pipe1 = Tsv("in1", ('x1, 'y1)).read + val pipe2 = Tsv("in2", ('x2, 'y2)).read + + pipe0.joinWithTiny('x0 -> 'x1, pipe1) + .joinWithSmaller('x0 -> 'x2, pipe2) + .map(('y0, 'y1, 'y2) -> ('y0, 'y1, 'y2)) { v: (TC, TC, TC) => + (v._1.n, v._2.n, v._3.n) + } + .project('x0, 'y0, 'x1, 'y1, 'x2, 'y2) + .write(Tsv("out")) +} + +case class TC(val n: Int) + +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)) + 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 => + (idx + ": join tuples with the same key") in { + outBuf.toList.sorted shouldBe correct + } + idx += 1 + } + .run + .runHadoop + .finish() + } +} + +class LeftJoinJob(args: Args) extends Job(args) { + val p1 = Tsv(args("input1")) + .mapTo((0, 1) -> ('k1, 'v1)) { v: (String, Int) => v } + val p2 = Tsv(args("input2")) + .mapTo((0, 1) -> ('k2, 'v2)) { v: (String, Int) => v } + p1.leftJoinWithSmaller('k1 -> 'k2, p2) + .project('k1, 'v1, 'v2) + // Null sent to TSV will not be read in properly + .map('v2 -> 'v2) { v: AnyRef => Option(v).map { _.toString }.getOrElse("NULL") } + .write(Tsv(args("output"))) +} + +class LeftJoinTest extends 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")) + var idx = 0 + JobTest(new LeftJoinJob(_)) + .arg("input1", "fakeInput1") + .arg("input2", "fakeInput2") + .arg("output", "fakeOutput") + .source(Tsv("fakeInput1"), input1) + .source(Tsv("fakeInput2"), input2) + .sink[(String, Int, JInt)](Tsv("fakeOutput")) { outBuf => + val actualOutput = outBuf.map { input: (String, Int, AnyRef) => + println(input) + val (k, v1, v2) = input + (k, (v1, v2)) + }.toMap + (idx + ": join tuples with the same key") in { + correctOutput shouldBe actualOutput + } + idx += 1 + } + .run + .runHadoop + .finish() + } +} + +class LeftJoinWithLargerJob(args: Args) extends Job(args) { + val p1 = Tsv(args("input1")) + .mapTo((0, 1) -> ('k1, 'v1)) { v: (String, Int) => v } + val p2 = Tsv(args("input2")) + .mapTo((0, 1) -> ('k2, 'v2)) { v: (String, Int) => v } + // Note i am specifying the joiner explicitly since this did not work properly before (leftJoinWithLarger always worked) + p1.joinWithLarger('k1 -> 'k2, p2, new cascading.pipe.joiner.LeftJoin) + .project('k1, 'v1, 'v2) + // Null sent to TSV will not be read in properly + .map('v2 -> 'v2) { v: AnyRef => Option(v).map { _.toString }.getOrElse("NULL") } + .write(Tsv(args("output"))) +} + +class 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")) + var idx = 0 + JobTest(new LeftJoinWithLargerJob(_)) + .arg("input1", "fakeInput1") + .arg("input2", "fakeInput2") + .arg("output", "fakeOutput") + .source(Tsv("fakeInput1"), input1) + .source(Tsv("fakeInput2"), input2) + .sink[(String, Int, JInt)](Tsv("fakeOutput")) { outBuf => + val actualOutput = outBuf.map { input: (String, Int, AnyRef) => + println(input) + val (k, v1, v2) = input + (k, (v1, v2)) + }.toMap + s"$idx: join tuples with the same key" in { + correctOutput shouldBe actualOutput + } + idx += 1 + } + .run + .runHadoop + .finish() + } +} + +class ForceReducersJob(args: Args) extends Job(args) { + TextLine("in").read + .rename((0, 1) -> ('num, 'line)) + .flatMap('line -> 'words){ l: String => l.split(" ") } + .groupBy('num){ _.toList[String]('words -> 'wordList).forceToReducers } + .map('wordList -> 'wordList){ w: List[String] => w.mkString(" ") } + .project('num, 'wordList) + .write(Tsv("out")) +} + +class ForceReducersTest extends WordSpec with Matchers { + "A ForceReducersJob" should { + var idx = 0 + JobTest(new ForceReducersJob(_)) + .source(TextLine("in"), List("0" -> "single test", "1" -> "single result")) + .sink[(Int, String)](Tsv("out")) { outBuf => + (idx + ": must get the result right") in { + //need to convert to sets because order + outBuf(0)._2.split(" ").toSet shouldBe Set("single", "test") + outBuf(1)._2.split(" ").toSet shouldBe Set("single", "result") + } + idx += 1 + } + .run + .runHadoop + .finish() + } +} + + +class CrossJob(args: Args) extends Job(args) { + val p1 = Tsv(args("in1")).read + .mapTo((0, 1) -> ('x, 'y)) { tup: (Int, Int) => tup } + val p2 = Tsv(args("in2")).read + .mapTo(0 -> 'z) { (z: Int) => z } + p1.crossWithTiny(p2).write(Tsv(args("out"))) +} + +class CrossTest extends WordSpec with Matchers { + "A CrossJob" should { + 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 => + (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 + .finish() + } +} + +class GroupAllCrossJob(args: Args) extends Job(args) { + val p1 = Tsv(args("in1")).read + .mapTo((0, 1) -> ('x, 'y)) { tup: (Int, Int) => tup } + .groupAll { _.max('x) } + .map('x -> 'x) { x: Int => List(x) } + + val p2 = Tsv(args("in2")).read + .mapTo(0 -> 'z) { (z: Int) => z } + p2.crossWithTiny(p1) + .map('x -> 'x) { l: List[Int] => l.size } + .project('x, 'z) + .write(Tsv(args("out"))) +} + +class GroupAllCrossTest extends WordSpec with Matchers { + "A GroupAllCrossJob" should { + var idx = 0 + JobTest(new GroupAllCrossJob(_)) + .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)](Tsv("fakeOut")) { outBuf => + (idx + ": must look exactly right") in { + outBuf should have size 2 + outBuf.toSet shouldBe Set((1, 4), (1, 5)) + } + idx += 1 + } + .run + .runHadoop + .finish() + } +} + +class SmallCrossJob(args: Args) extends Job(args) { + val p1 = Tsv(args("in1")).read + .mapTo((0, 1) -> ('x, 'y)) { tup: (Int, Int) => tup } + val p2 = Tsv(args("in2")).read + .mapTo(0 -> 'z) { (z: Int) => z } + p1.crossWithSmaller(p2).write(Tsv(args("out"))) +} + +class SmallCrossTest extends WordSpec with Matchers { + "A SmallCrossJob" should { + 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 => + (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 + .finish() + } +} + + +class ScanJob(args: Args) extends Job(args) { + Tsv("in", ('x, 'y, 'z)) + .groupBy('x) { + _.sortBy('y) + .scanLeft('y -> 'ys)(0) { (oldV: Int, newV: Int) => oldV + newV } + } + .project('x, 'ys, 'z) + .map('z -> 'z) { z: Int => z } //Make sure the null z is converted to an int + .write(Tsv("out")) +} + +class ScanTest extends WordSpec with Matchers { + import Dsl._ + + "A ScanJob" should { + 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)) + (idx + ": have a working scanLeft") in { + outBuf.toList shouldBe correct + } + idx += 1 + } + .run + .runHadoop + .finish() + } +} + +class IterableSourceJob(args: Args) extends Job(args) { + val list = List((1, 2, 3), (4, 5, 6), (3, 8, 9)) + val iter = IterableSource(list, ('x, 'y, 'z)) + Tsv("in", ('x, 'w)) + .joinWithSmaller('x -> 'x, iter) + .write(Tsv("out")) + + Tsv("in", ('x, 'w)) + .joinWithTiny('x -> 'x, iter) + .write(Tsv("tiny")) + //Now without fields and using the implicit: + Tsv("in", ('x, 'w)) + .joinWithTiny('x -> 0, list).write(Tsv("imp")) +} + +class IterableSourceTest extends WordSpec with Matchers with FieldConversions { + val input = List((1, 10), (2, 20), (3, 30)) + "A IterableSourceJob" should { + var idx = 0 + JobTest(new IterableSourceJob(_)) + .source(Tsv("in", ('x, 'w)), input) + .sink[(Int, Int, Int, Int)](Tsv("out")) { outBuf => + 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 => + 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 => + 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 + .finish() + } +} + +// TODO make a Product serializer that clean $outer parameters +case class V(v: Int) +class InnerCaseJob(args: Args) extends Job(args) { + val res = TypedTsv[Int]("input") + .mapTo(('xx, 'vx)) { x => (x * x, V(x)) } + .groupBy('xx) { _.head('vx) } + .map('vx -> 'x) { v: V => v.v } + .project('x, 'xx) + .write(Tsv("output")) +} + +class InnerCaseTest extends WordSpec with Matchers { + import Dsl._ + + val input = List(Tuple1(1), Tuple1(2), Tuple1(2), Tuple1(4)) + "An InnerCaseJob" should { + JobTest(new InnerCaseJob(_)) + .source(TypedTsv[Int]("input"), input) + .sink[(Int, Int)](Tsv("output")) { outBuf => + "Correctly handle inner case classes" in { + outBuf.toSet shouldBe Set((1, 1), (2, 4), (4, 16)) + } + } + .runHadoop + .finish() + } +} + + +class ForceToDiskJob(args: Args) extends Job(args) { + val x = Tsv("in", ('x, 'y)) + .read + .filter('x) { x: Int => x > 0 } + .rename('x -> 'x1) + Tsv("in", ('x, 'y)) + .read + .joinWithTiny('y -> 'y, x.forceToDisk) + .project('x, 'x1, 'y) + .write(Tsv("out")) +} + +class ForceToDiskTest extends WordSpec with Matchers { + import Dsl._ + + "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 => + (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 shouldBe correct + } + idx += 1 + } + .run + .runHadoop + .finish() + } +} + + +class GroupAllToListTestJob(args: Args) extends Job(args) { + TypedTsv[(Long, String, Double)]("input") + .mapTo('a, 'b) { case (id, k, v) => (id, Map(k -> v)) } + .groupBy('a) { _.sum[Map[String, Double]]('b) } + .groupAll { + _.toList[(Long, Map[String, Double])](('a, 'b) -> 'abList) + } + .map('abList -> 'abMap) { + list: List[(Long, Map[String, Double])] => list.toMap + } + .project('abMap) + .map('abMap -> 'abMap) { x: AnyRef => x.toString } + .write(Tsv("output")) +} + +class GroupAllToListTest extends WordSpec with Matchers { + import Dsl._ + + "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)) + JobTest(new GroupAllToListTestJob(_)) + .source(TypedTsv[(Long, String, Double)]("input"), input) + .sink[String](Tsv("output")) { outBuf => + "must properly aggregate stuff into a single map" in { + outBuf should have size 1 + outBuf(0) shouldBe output.toString + } + } + .runHadoop + .finish() + } +} + + +class ToListGroupAllToListTestJob(args: Args) extends Job(args) { + TypedTsv[(Long, String)]("input") + .mapTo('b, 'c) { case (k, v) => (k, v) } + .groupBy('c) { _.toList[Long]('b -> 'bList) } + .groupAll { + _.toList[(String, List[Long])](('c, 'bList) -> 'cbList) + } + .project('cbList) + .write(Tsv("output")) +} + +class ToListGroupAllToListSpec extends WordSpec with Matchers { + import Dsl._ + + val expected = List(("us", List(1)), ("jp", List(3, 2)), ("gb", List(3, 1))) + + "A ToListGroupAllToListTestJob" should { + JobTest(new ToListGroupAllToListTestJob(_)) + .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 should have size 1 + outBuf.head shouldBe (expected.toString) + println(outBuf.head) + } + } + .runHadoop + .finish() + + JobTest(new ToListGroupAllToListTestJob(_)) + .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 should have size 1 + outBuf.head shouldBe expected + println(outBuf.head) + } + } + .run + .finish() + } +} + +// TODO: HangingTest is very flaky now because we enabled multi-thread testing. Need to be fixed later. +/* +class HangingJob(args : Args) extends Job(args) { + val x = Tsv("in", ('x,'y)) + .read + .filter('x, 'y) { t: (Int, Int) => + val (x, y) = t + timeout(Millisecs(2)) { + if (y % 2 == 1) Thread.sleep(1000) + x > 0 + } getOrElse false + } + .write(Tsv("out")) +} + +class HangingTest extends Specification { + import Dsl._ + noDetailedDiffs() + + "A HangingJob" should { + val input = (1 to 100).flatMap { i => List((-1, i), (1, i)) }.toList + JobTest(new HangingJob(_)) + .source(Tsv("in",('x,'y)), input) + .sink[(Int,Int)](Tsv("out")) { outBuf => + "run correctly when task times out" in { + //outBuf.size must_== 100 + //val correct = (1 to 100).map { i => (1, i) } + outBuf.size must_== 50 + val correct = (1 to 50).map { i => (1, i*2) } + outBuf.toList.sorted must_== correct + } + } + .run + .runHadoop + .finish() + } +} +*/ + diff --git a/scalding-core/src/test/scala/com/twitter/scalding/ExecutionTest.scala b/scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/ExecutionTest.scala similarity index 97% rename from scalding-core/src/test/scala/com/twitter/scalding/ExecutionTest.scala rename to scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/ExecutionTest.scala index b7560e6f51..c00c5044ae 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/ExecutionTest.scala +++ b/scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/ExecutionTest.scala @@ -225,16 +225,16 @@ class ExecutionTest extends WordSpec with Matchers { "ExecutionApp" should { 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") should contain ("100") + "parse args correctly (local)" in { + val (conf, mode) = parser.config(Array("-Dmapred.reduce.tasks=100", "--local")) + 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") should contain ("110") - conf1.getArgs.boolean("test") shouldBe true - hconf.get("mapred.reduce.tasks") shouldBe "110" + mode shouldBe a[LocalMode] } + + /* Hadoop-specific code moved to ExecutionHadoopExtraTest */ + } "An ExecutionJob" should { "run correctly" in { diff --git a/scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/FileSourceHadoopExtraTest.scala b/scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/FileSourceHadoopExtraTest.scala new file mode 100644 index 0000000000..1b8ca49c0a --- /dev/null +++ b/scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/FileSourceHadoopExtraTest.scala @@ -0,0 +1,42 @@ +package com.twitter.scalding + +import org.apache.hadoop.conf.Configuration +import org.scalatest.{ Matchers, WordSpec } + +class FileSourceHadoopExtraTest /* logically extends FileSourceTest */ extends WordSpec with Matchers { + import Dsl._ + + def makeClusterMode(strictSources: Boolean) = + Mode(Args(Seq("--autoCluster") ++ (if (strictSources) Seq[String]() else Seq("--tool.partialok"))), new Configuration) + + "invalid source input" should { + "Throw in validateTaps in strict mode" in { + val e = intercept[InvalidSourceException] { + TestInvalidFileSource.validateTaps(makeClusterMode(true)) + } + assert(e.getMessage.endsWith("Data is missing from one or more paths in: List(invalid_hdfs_path)")) + } + + "Throw in validateTaps in non-strict mode" in { + val e = intercept[InvalidSourceException] { + TestInvalidFileSource.validateTaps(makeClusterMode(false)) + } + assert(e.getMessage.endsWith("No good paths in: List(invalid_hdfs_path)")) + } + + "Throw in toIterator because no data is present in strict mode" in { + val e = intercept[InvalidSourceException] { + TestInvalidFileSource.toIterator(Config.default, makeClusterMode(true)) + } + assert(e.getMessage.endsWith("Data is missing from one or more paths in: List(invalid_hdfs_path)")) + } + + "Throw in toIterator because no data is present in non-strict mode" in { + val e = intercept[InvalidSourceException] { + TestInvalidFileSource.toIterator(Config.default, makeClusterMode(false)) + } + assert(e.getMessage.endsWith("No good paths in: List(invalid_hdfs_path)")) + } + } + +} diff --git a/scalding-core/src/test/scala/com/twitter/scalding/PartitionSourceTest.scala b/scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/PartitionSourceTest.scala similarity index 68% rename from scalding-core/src/test/scala/com/twitter/scalding/PartitionSourceTest.scala rename to scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/PartitionSourceTest.scala index 4c771e75fe..c567346a82 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/PartitionSourceTest.scala +++ b/scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/PartitionSourceTest.scala @@ -16,18 +16,18 @@ limitations under the License. package com.twitter.scalding -import java.io.File -import scala.io.{ Source => ScalaSource } +/* (temporarily?) moved from scalding-core due to chicken-and-egg dependency situation */ -import org.scalatest.{ Matchers, WordSpec } +import java.io.File import cascading.tap.SinkMode -import cascading.tuple.Fields -import cascading.tuple.TupleEntry -import cascading.util.Util import cascading.tap.partition.Partition +import cascading.tuple.{ Fields, TupleEntry } +import cascading.util.Util +import com.twitter.scalding.{ PartitionedTsv => StandardPartitionedTsv } +import org.scalatest.{ Matchers, WordSpec } -import com.twitter.scalding.{ PartitionedTsv => StandardPartitionedTsv, _ } +import scala.io.{ Source => ScalaSource } object PartitionSourceTestHelpers { import Dsl._ @@ -53,7 +53,8 @@ object PartitionSourceTestHelpers { class DelimitedPartitionTestJob(args: Args) extends Job(args) { import PartitionSourceTestHelpers._ try { - Tsv("input", ('col1, 'col2)).read.write(DelimitedPartitionedTsv) + Tsv("input", ('col1, 'col2)).read + .write(DelimitedPartitionedTsv) } catch { case e: Exception => e.printStackTrace() } @@ -94,20 +95,23 @@ class DelimitedPartitionSourceTest extends WordSpec with Matchers { JobTest(buildJob(_)) .source(Tsv("input", ('col1, 'col2)), input) - .runHadoop + .runWithMinicluster .finish() - val testMode = job.mode.asInstanceOf[HadoopTest] + job.mode match { + case testMode: HadoopFamilyTestMode => - val directory = new File(testMode.getWritePathFor(DelimitedPartitionedTsv)) + val directory = new File(testMode.getWritePathFor(DelimitedPartitionedTsv)) - directory.listFiles().map({ _.getName() }).toSet shouldBe 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")) + val aSource = ScalaSource.fromFile(new File(directory, "A/part-00000-00000")) + val bSource = ScalaSource.fromFile(new File(directory, "B/part-00000-00001")) - aSource.getLines.toSeq shouldBe Seq("A\t1", "A\t2") - bSource.getLines.toSeq shouldBe Seq("B\t3") + aSource.getLines.toSeq shouldBe Seq("A\t1", "A\t2") + bSource.getLines.toSeq shouldBe Seq("B\t3") + case _ => ??? + } } } } @@ -128,20 +132,22 @@ class CustomPartitionSourceTest extends WordSpec with Matchers { JobTest(buildJob(_)) .source(Tsv("input", ('col1, 'col2, 'col3)), input) - .runHadoop + .runWithMinicluster .finish() - val testMode = job.mode.asInstanceOf[HadoopTest] - - val directory = new File(testMode.getWritePathFor(CustomPartitionedTsv)) + job.mode match { + case testMode: HadoopFamilyTestMode => + val directory = new File(testMode.getWritePathFor(CustomPartitionedTsv)) - directory.listFiles().map({ _.getName() }).toSet shouldBe 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")) + 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.toSeq shouldBe Seq("A\tx\t1", "A\tx\t2") - bSource.getLines.toSeq shouldBe Seq("B\ty\t3") + aSource.getLines.toSeq shouldBe Seq("A\tx\t1", "A\tx\t2") + bSource.getLines.toSeq shouldBe Seq("B\ty\t3") + case _ => ??? + } } } } @@ -163,20 +169,24 @@ class PartialPartitionSourceTest extends WordSpec with Matchers { JobTest(buildJob(_)) .source(Tsv("input", ('col1, 'col2, 'col3)), input) - .runHadoop + .runWithMinicluster .finish() - val testMode = job.mode.asInstanceOf[HadoopTest] - - val directory = new File(testMode.getWritePathFor(PartialPartitionedTsv)) + job.mode match { + case testMode: HadoopFamilyTestMode => + val directory = new File(testMode.getWritePathFor(PartialPartitionedTsv)) - directory.listFiles().map({ _.getName() }).toSet shouldBe 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")) + 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.toSeq shouldBe Seq("A\t1", "A\t2") - bSource.getLines.toSeq shouldBe Seq("B\t3") + aSource.getLines.toSeq shouldBe Seq("A\t1", "A\t2") + bSource.getLines.toSeq shouldBe Seq("B\t3") + case _ => ??? + } } } } diff --git a/scalding-core/src/test/scala/com/twitter/scalding/ReduceOperationsTest.scala b/scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/ReduceOperationsTest.scala similarity index 88% rename from scalding-core/src/test/scala/com/twitter/scalding/ReduceOperationsTest.scala rename to scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/ReduceOperationsTest.scala index 9e9e5e323b..0da071bdc4 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/ReduceOperationsTest.scala +++ b/scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/ReduceOperationsTest.scala @@ -27,7 +27,8 @@ class SortWithTakeJob(args: Args) extends Job(args) { } .map('top_items -> 'top_items) { //used to test that types are correct - topItems: List[(Long, Double)] => topItems + topItems: List[(Long, Double)] => topItems.toString /* TSV can't encode complex types + fake TSV for in-memory Local tests can get away with this, but this'll fail on a real fabric test */ } .project('key, 'top_items) .write(Tsv("output0")) @@ -87,19 +88,21 @@ class ApproximateUniqueCountJob(args: Args) extends Job(args) { } } -class ReduceOperationsTest extends WordSpec with Matchers { + + +class ListsLongDoubleUtil 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(new SortWithTakeJob(_)) .source(Tsv("input0", ('key, 'item_id, 'score)), inputData) - .sink[(String, List[(Long, Double)])](Tsv("output0")) { buf => + .sink[(String, String /* representing List[(Long, Double)] */)](Tsv("output0")) { buf => "grouped list" in { - val whatWeWant: Map[String, String] = Map( + val whatWeWant = Map( "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 + val whatWeGet = buf.toMap whatWeGet.get("a").getOrElse("apples") shouldBe (whatWeWant.get("a").getOrElse("oranges")) whatWeGet.get("b").getOrElse("apples") shouldBe (whatWeWant.get("b").getOrElse("oranges")) } @@ -110,12 +113,12 @@ class ReduceOperationsTest extends WordSpec with Matchers { "A sortedTake job" should { JobTest(new SortedTakeJob(_)) .source(Tsv("input0", ('key, 'item_id, 'score)), inputData) - .sink[(String, List[(Long, Double)])](Tsv("output0")) { buf => + .sink[(String, String /* representing List[(Long, Double)] */)](Tsv("output0")) { buf => "grouped list" in { val whatWeWant: Map[String, String] = Map( "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 + val whatWeGet = buf.toMap whatWeGet.get("a").getOrElse("apples") shouldBe (whatWeWant.get("a").getOrElse("oranges")) whatWeGet.get("b").getOrElse("apples") shouldBe (whatWeWant.get("b").getOrElse("oranges")) } @@ -127,12 +130,12 @@ class ReduceOperationsTest extends WordSpec with Matchers { "A sortedReverseTake job" should { JobTest(new SortedReverseTakeJob(_)) .source(Tsv("input0", ('key, 'item_id, 'score)), inputData) - .sink[(String, List[(Long, Double)])](Tsv("output0")) { buf => + .sink[(String, String /* representing List[(Long, Double)] */)](Tsv("output0")) { buf => "grouped list" in { val whatWeWant: Map[String, String] = Map( "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 + val whatWeGet = buf.toMap whatWeGet.get("a").getOrElse("apples") shouldBe (whatWeWant.get("a").getOrElse("oranges")) whatWeGet.get("b").getOrElse("apples") shouldBe (whatWeWant.get("b").getOrElse("oranges")) } diff --git a/scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/TestFileUtil.scala b/scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/TestFileUtil.scala new file mode 100644 index 0000000000..02c4f09a3c --- /dev/null +++ b/scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/TestFileUtil.scala @@ -0,0 +1,46 @@ +package com.twitter.scalding + +import java.io.File + +/** + * Miscellaneous bits of code related to checking whether the expected + * output files are within a directory, etc. + */ +object TestFileUtil { + import scala.language.implicitConversions + + case class RichDirectory(dir: File) { + import RichDirectory._ + lazy val allFiles = dir.listFiles.toSet + + lazy val rawFileNames = allFiles.map(_.getName) + + lazy val fileNameSet = rawFileNames.filterNot(isFileNameOfCrcFile) + + def fileNameSetExSuccess = fileNameSet - "_SUCCESS" + + /* The naming convention of the parts is a fabric-specific implementation detail. However, + * they tend to start with part- */ + def partFiles = fileNameSet.filter(_.startsWith("part-")) + + def list = dir.list.toSet + def listFiles = allFiles + + implicit def backToFile = dir + override def toString = dir.toString + } + implicit def toRichDirectory(dir: File) = RichDirectory(dir) + + object RichDirectory { + def apply(dirname: String): RichDirectory = RichDirectory(new File(dirname)) + def apply(parent: File, dirname: String): RichDirectory = RichDirectory(new File(parent, dirname)) + def apply(parent: RichDirectory, dirname: String): RichDirectory = RichDirectory(new File(parent.dir, dirname)) + + private val CrcPattern = "^[.](.*?)[.]crc$".r + def isFileNameOfCrcFile(filename: String) = { + val m = CrcPattern.findFirstMatchIn(filename) + !m.isEmpty + } + } +} + diff --git a/scalding-core/src/test/scala/com/twitter/scalding/TypedPipeTest.scala b/scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/TypedPipeTest.scala similarity index 98% rename from scalding-core/src/test/scala/com/twitter/scalding/TypedPipeTest.scala rename to scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/TypedPipeTest.scala index 39aebfb2a6..ad970981ff 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/TypedPipeTest.scala +++ b/scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/TypedPipeTest.scala @@ -16,8 +16,9 @@ limitations under the License. package com.twitter.scalding import org.scalatest.{ FunSuite, Matchers, WordSpec } +import java.lang.SuppressWarnings -import com.twitter.scalding.source.TypedText +import com.twitter.scalding.source.{ TypedSep, TypedText } // Use the scalacheck generators import org.scalacheck.Gen import scala.collection.mutable.Buffer @@ -392,7 +393,7 @@ class TypedPipeWithOnCompleteTest extends WordSpec with Matchers { JobTest(new TypedWithOnCompleteJob(_)) .source(TypedText.tsv[String]("input"), inputText.split("\\s+").map(Tuple1(_))) .counter("onCompleteMapper") { cnt => "have onComplete called on mapper" in { assert(cnt == 1) } } - .counter("onCompleteReducer") { cnt => "have onComplete called on reducer" in { assert(cnt == 1) } } + .counter("onCompleteReducer") { cnt => "have onComplete called on reducer" in { assert(cnt == 1) } } /* FIXME TEZ: this one fails because we don't (YET?) control the number of Gathers per node, equiv. of .withReducers */ .sink[String](TypedText.tsv[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 @@ -847,8 +848,8 @@ trait TypedComplexHashAndMergeJobBase { def tdXe: TypedPipe[String] } -class TypedComplexHashAndMergeJob(args: Args , - fieldsToMerge: Seq[(String, TypedComplexHashAndMergeJobBase => TypedPipe[String])]) +class TypedComplexHashAndMergeJob(args: Args, + fieldsToMerge: Seq[(String, TypedComplexHashAndMergeJobBase => TypedPipe[String])]) extends Job(args) with TypedComplexHashAndMergeJobBase { override def name: String = super.name + " (" + fieldsToMerge.map(_._1).mkString(" ++ ") + ")" @@ -874,7 +875,7 @@ class TypedComplexHashAndMergeTest extends WordSpec with Matchers { ("a", _.ta), ("a∩b", _.taXb), ("c", _.tc), - ("d", _.td), + ("d", _.td), ("d∩e", _.tdXe)) val selection = fields.permutations.take(3) // Take'em all if you need to prove all permutations work equally (kind of slow, and internally we do use commutativity) @@ -931,8 +932,8 @@ class TypedTwistedHashAndMergeJob(args: Args) extends Job(args) { (y.head, y.tail) }).group - val twistAB: TypedPipe[String] = twistA.hashJoin(twistB).values.map { case (a,b) => a + "≡" + b } - val twistBA: TypedPipe[String] = twistB.hashJoin(twistA).values.map { case (a,b) => a + "≢" + b } + val twistAB: TypedPipe[String] = twistA.hashJoin(twistB).values.map { case (a, b) => a + "≡" + b } + val twistBA: TypedPipe[String] = twistB.hashJoin(twistA).values.map { case (a, b) => a + "≢" + b } (taXbXe ++ tdXeXb ++ twistAB ++ twistBA) .write(TypedText.tsv[String]("output")) @@ -941,7 +942,7 @@ class TypedTwistedHashAndMergeJob(args: Args) extends Job(args) { class TypedTwistedHashAndMergeTest extends WordSpec with Matchers { import Dsl._ - s"A TypedTwistedHashAndMergeTest" should { + "A TypedTwistedHashAndMergeTest" should { var idx = 0 JobTest(new TypedTwistedHashAndMergeJob(_: Args)) @@ -1533,6 +1534,8 @@ class TypedSketchJoinJob(args: Args) extends Job(args) { } class TypedSketchLeftJoinJob(args: Args) extends Job(args) { + /* FIXME TEZ: this seems to give trouble to the TezRuleRegistry, in rule RemoveMalformedHashJoinNodeTransformer */ + val zero = TypedPipe.from(TypedText.tsv[(Int, Int)]("input0")) val one = TypedPipe.from(TypedText.tsv[(Int, Int)]("input1")) diff --git a/scalding-core/src/test/scala/com/twitter/scalding/mathematics/Matrix2Test.scala b/scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/mathematics/Matrix2Test.scala similarity index 100% rename from scalding-core/src/test/scala/com/twitter/scalding/mathematics/Matrix2Test.scala rename to scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/mathematics/Matrix2Test.scala diff --git a/scalding-core/src/test/scala/com/twitter/scalding/typed/BijectedSourceSinkTest.scala b/scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/typed/BijectedSourceSinkTest.scala similarity index 100% rename from scalding-core/src/test/scala/com/twitter/scalding/typed/BijectedSourceSinkTest.scala rename to scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/typed/BijectedSourceSinkTest.scala diff --git a/scalding-core/src/test/scala/com/twitter/scalding/typed/InAnotherPackage.scala b/scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/typed/InAnotherPackage.scala similarity index 87% rename from scalding-core/src/test/scala/com/twitter/scalding/typed/InAnotherPackage.scala rename to scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/typed/InAnotherPackage.scala index 108884acce..54e589432e 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/typed/InAnotherPackage.scala +++ b/scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/typed/InAnotherPackage.scala @@ -5,6 +5,8 @@ import scala.concurrent.{ ExecutionContext => SExecutionContext, _ } import SExecutionContext.Implicits.global object InAnotherPackage { + /* this object is used in NoStackLineNumberTest */ + def buildF: Future[TypedPipe[(Int, Int)]] = { Future { TypedPipe.from(List(1, 2, 3, 4, 555, 3)) diff --git a/scalding-core/src/test/scala/com/twitter/scalding/typed/NoStackLineNumberTest.scala b/scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/typed/NoStackLineNumberTest.scala similarity index 86% rename from scalding-core/src/test/scala/com/twitter/scalding/typed/NoStackLineNumberTest.scala rename to scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/typed/NoStackLineNumberTest.scala index 4db59794a4..64d519468c 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/typed/NoStackLineNumberTest.scala +++ b/scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/typed/NoStackLineNumberTest.scala @@ -15,23 +15,24 @@ limitations under the License. */ package com.twitter.scalding.typed -import org.scalatest.WordSpec - -import com.twitter.scalding._ -import scala.concurrent.{ ExecutionContext => SExecutionContext, _ } -import SExecutionContext.Implicits.global -import scala.concurrent.duration.{ Duration => SDuration } +/* moved from scalding-core due to a chicken-and-egg situation */ import cascading.flow.FlowDef +import com.twitter.scalding._ import org.apache.hadoop.conf.Configuration +import org.scalatest.WordSpec + +import scala.concurrent.ExecutionContext.Implicits.global +import scala.concurrent.duration.{ Duration => SDuration } +import scala.concurrent.{ ExecutionContext => SExecutionContext, _ } +import Dsl._ class NoStackLineNumberTest extends WordSpec { "No Stack Shouldn't block getting line number info" should { "actually get the no stack info" in { - import Dsl._ implicit val fd = new FlowDef - implicit val m = new Hdfs(false, new Configuration) + implicit val m = Mode.test("anyCluster-test", new Configuration, (_: Source) => None) val pipeFut = com.twitter.example.scalding.typed.InAnotherPackage.buildF.map { tp => tp.toPipe('a, 'b) diff --git a/scalding-core/src/test/scala/com/twitter/scalding/typed/PartitionedDelimitedSourceTest.scala b/scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/typed/PartitionedDelimitedSourceTest.scala similarity index 57% rename from scalding-core/src/test/scala/com/twitter/scalding/typed/PartitionedDelimitedSourceTest.scala rename to scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/typed/PartitionedDelimitedSourceTest.scala index 9a262717a8..61470273c8 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/typed/PartitionedDelimitedSourceTest.scala +++ b/scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/typed/PartitionedDelimitedSourceTest.scala @@ -16,20 +16,23 @@ package com.twitter.scalding.typed import java.io.File -import scala.io.{ Source => ScalaSource } - +import com.twitter.scalding._ +import com.twitter.scalding.source.TypedText import org.scalatest.{ Matchers, WordSpec } -import com.twitter.scalding._ +import scala.io.{ Source => ScalaSource } import TDsl._ +/* Moved from scalding-core due to fabric-specific code */ + object PartitionedDelimitedTestSources { val singlePartition = PartitionedCsv[String, (String, String)]("out", "%s") } class PartitionedDelimitedWriteJob(args: Args) extends Job(args) { import PartitionedDelimitedTestSources._ - TypedCsv[(String, String, String)]("in") + + TypedText.csv[(String, String, String)]("in") .map { case (v1, v2, v3) => (v1, (v2, v3)) } .write(singlePartition) } @@ -49,21 +52,37 @@ class PartitionedDelimitedTest extends WordSpec with Matchers { } JobTest(buildJob(_)) - .source(TypedCsv[(String, String, String)]("in"), input) - .runHadoop + .source(TypedText.csv[(String, String, String)]("in"), input) + .runWithMinicluster .finish() - val testMode = job.mode.asInstanceOf[HadoopTest] + job.mode match { + case testMode: HadoopFamilyTestMode => + import TestFileUtil._ + val directory = RichDirectory(testMode.getWritePathFor(singlePartition)) + + // println(s"looking at ${directory}") + + directory.list shouldBe Set("A", "B") // this proves the partition strategy WAS applied. - val directory = new File(testMode.getWritePathFor(singlePartition)) + val aDir = RichDirectory(directory, "A") + val aFiles = aDir.fileNameSetExSuccess + aFiles.size shouldBe 1 + aDir.partFiles.size shouldBe 1 - directory.listFiles().map({ _.getName() }).toSet shouldBe Set("A", "B") + val bDir = RichDirectory(directory, "B") + val bFiles = bDir.fileNameSetExSuccess + bFiles.size shouldBe 1 + bDir.partFiles.size shouldBe 1 - val aSource = ScalaSource.fromFile(new File(directory, "A/part-00000-00000")) - val bSource = ScalaSource.fromFile(new File(directory, "B/part-00000-00001")) + val aSource = ScalaSource.fromFile(new File(aDir.dir, aDir.partFiles.head)) + val bSource = ScalaSource.fromFile(new File(bDir.dir, bDir.partFiles.head)) + + aSource.getLines.toList shouldBe Seq("X,1", "Y,2") + bSource.getLines.toList shouldBe Seq("Z,3") + case _ => ??? + } - 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-fabric-tests/src/fabric/scala/com/twitter/scalding/typed/PartitionedTextLineTest.scala similarity index 94% rename from scalding-core/src/test/scala/com/twitter/scalding/typed/PartitionedTextLineTest.scala rename to scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/typed/PartitionedTextLineTest.scala index cc5967c941..9e7802d08b 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/typed/PartitionedTextLineTest.scala +++ b/scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/typed/PartitionedTextLineTest.scala @@ -16,14 +16,14 @@ package com.twitter.scalding.typed import java.io.File -import scala.io.{ Source => ScalaSource } - -import org.scalatest.{ Matchers, WordSpec } - import com.twitter.scalding._ +import org.scalatest.{ Matchers, WordSpec } +import scala.io.{ Source => ScalaSource } import TDsl._ +/* moved here due to chicken-and-egg situation with scalding-core */ + object PartitionedTextLineTestSources { val singlePartition = PartitionedTextLine[String]("out", "%s") val multiplePartition = PartitionedTextLine[(String, String)]("out", "%s/%s") @@ -60,7 +60,7 @@ class PartitionedTextLineTest extends WordSpec with Matchers { .runHadoop .finish() - val testMode = job.mode.asInstanceOf[HadoopTest] + val testMode = job.mode.asInstanceOf[HadoopFamilyTestMode] val directory = new File(testMode.getWritePathFor(singlePartition)) println(directory) @@ -88,7 +88,7 @@ class PartitionedTextLineTest extends WordSpec with Matchers { .runHadoop .finish() - val testMode = job.mode.asInstanceOf[HadoopTest] + val testMode = job.mode.asInstanceOf[HadoopFamilyTestMode] val directory = new File(testMode.getWritePathFor(multiplePartition)) println(directory) diff --git a/scalding-core/src/test/scala/com/twitter/scalding/WrappedJoinerTest.scala b/scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/typed/WrappedJoinerTest.scala similarity index 92% rename from scalding-core/src/test/scala/com/twitter/scalding/WrappedJoinerTest.scala rename to scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/typed/WrappedJoinerTest.scala index 94e76edff3..be432265f3 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/WrappedJoinerTest.scala +++ b/scalding-core-fabric-tests/src/fabric/scala/com/twitter/scalding/typed/WrappedJoinerTest.scala @@ -1,13 +1,14 @@ -package com.twitter.scalding +package com.twitter.scalding.typed + +import java.util.{ Iterator => JIterator } import cascading.flow.FlowException import cascading.pipe.CoGroup -import cascading.pipe.joiner.{ JoinerClosure, InnerJoin } +import cascading.pipe.joiner.{ InnerJoin, JoinerClosure } import cascading.tuple.Tuple +import com.twitter.scalding._ import org.scalatest.{ Matchers, WordSpec } -import java.util.{ Iterator => JIterator } - class CheckFlowProcessJoiner(uniqueID: UniqueID) extends InnerJoin { override def getIterator(joinerClosure: JoinerClosure): JIterator[Tuple] = { val flowProcess = RuntimeStats.getFlowProcessForUniqueId(uniqueID) @@ -46,7 +47,7 @@ class WrappedJoinerTest extends WordSpec with Matchers { .sink[(Int, String)](Tsv("output")) { outBuf => // The job will fail with an exception if the FlowProcess is unavailable. } - .runHadoop + .runWithMinicluster .finish() } @@ -58,7 +59,7 @@ class WrappedJoinerTest extends WordSpec with Matchers { .sink[(Int, String)](Tsv("output")) { outBuf => // The job will fail with an exception if the FlowProcess is unavailable. } - .runHadoop + .runWithMinicluster .finish() fail("The test Job without WrappedJoiner should fail.") diff --git a/scalding-core-fabric-tests/src/test/scala/com/twitter/scalding/NoticeSpec.scala b/scalding-core-fabric-tests/src/test/scala/com/twitter/scalding/NoticeSpec.scala new file mode 100644 index 0000000000..e797bf5aca --- /dev/null +++ b/scalding-core-fabric-tests/src/test/scala/com/twitter/scalding/NoticeSpec.scala @@ -0,0 +1,12 @@ +package com.twitter.scalding + +import org.scalatest.{ WordSpec, Matchers } + +class NoticeSpec extends WordSpec with Matchers { + "This package" should { + "not actually run tests but provide support for other subprojects" in { + true === true + } + } + +} \ No newline at end of file diff --git a/scalding-core/src/main/java/com/twitter/scalding/InvalidSourceException.java b/scalding-core/src/main/java/com/twitter/scalding/InvalidSourceException.java new file mode 100644 index 0000000000..e8df4d4618 --- /dev/null +++ b/scalding-core/src/main/java/com/twitter/scalding/InvalidSourceException.java @@ -0,0 +1,15 @@ +package com.twitter.scalding; + +/** + * thrown when validateTaps fails + * + * Defined in Java as SLS 5.3.1 prevents us from selecting which inherited ctor we defer to. + */ +public class InvalidSourceException extends RuntimeException { + public InvalidSourceException(String message) { + super(message); + } + public InvalidSourceException(String message, Throwable throwable) { + super(message, throwable); + } +} diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala index efd8236777..6e6c0b9fff 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala @@ -449,11 +449,7 @@ object Config { /** * Merge Config.default with Hadoop config from the mode (if in Hadoop mode) */ - def defaultFrom(mode: Mode): Config = - default ++ (mode match { - case m: HadoopMode => Config.fromHadoop(m.jobConf) - IoSerializationsKey - case _ => empty - }) + def defaultFrom(mode: Mode): Config = default ++ mode.executionMode.defaultConfig def apply(m: Map[String, String]): Config = new Config { def toMap = m } /* diff --git a/scalding-core/src/main/scala/com/twitter/scalding/ExecutionContext.scala b/scalding-core/src/main/scala/com/twitter/scalding/ExecutionContext.scala index 74699a4929..566e7db852 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/ExecutionContext.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/ExecutionContext.scala @@ -39,17 +39,6 @@ trait ExecutionContext { import ExecutionContext._ - private def getIdentifierOpt(descriptions: Seq[String]): Option[String] = { - if (descriptions.nonEmpty) Some(descriptions.distinct.mkString(", ")) else None - } - - private def updateStepConfigWithDescriptions(step: BaseFlowStep[JobConf]): Unit = { - val conf = step.getConfig - getIdentifierOpt(ExecutionContext.getDesc(step)).foreach(descriptionString => { - conf.set(Config.StepDescriptions, descriptionString) - }) - } - final def buildFlow: Try[Flow[_]] = // For some horrible reason, using Try( ) instead of the below gets me stuck: // [error] @@ -71,53 +60,14 @@ trait ExecutionContext { // identify the flowDef val configWithId = config.addUniqueId(UniqueID.getIDFor(flowDef)) - val flow = mode.newFlowConnector(configWithId).connect(flowDef) + val flow = mode.newFlow(configWithId, flowDef) + if (config.getRequireOrderedSerialization) { // This will throw, but be caught by the outer try if // we have groupby/cogroupby not using OrderedSerializations CascadingBinaryComparator.checkForOrderedSerialization(flow).get } - flow match { - case hadoopFlow: HadoopFlow => - val flowSteps = hadoopFlow.getFlowSteps.asScala - flowSteps.foreach { - case baseFlowStep: BaseFlowStep[JobConf] => - updateStepConfigWithDescriptions(baseFlowStep) - } - case _ => // descriptions not yet supported in other modes - } - - // if any reducer estimators have been set, register the step strategy - // which instantiates and runs them - mode match { - case _: HadoopMode => - val reducerEstimatorStrategy: Seq[FlowStepStrategy[JobConf]] = config.get(Config.ReducerEstimators).toList.map(_ => ReducerEstimatorStepStrategy) - - val otherStrategies: Seq[FlowStepStrategy[JobConf]] = config.getFlowStepStrategies.map { - case Success(fn) => fn(mode, configWithId) - case Failure(e) => throw new Exception("Failed to decode flow step strategy when submitting job", e) - } - - val optionalFinalStrategy = FlowStepStrategies().sumOption(reducerEstimatorStrategy ++ otherStrategies) - - optionalFinalStrategy.foreach { strategy => - flow.setFlowStepStrategy(strategy) - } - - config.getFlowListeners.foreach { - case Success(fn) => flow.addListener(fn(mode, configWithId)) - case Failure(e) => throw new Exception("Failed to decode flow listener", e) - } - - config.getFlowStepListeners.foreach { - case Success(fn) => flow.addStepListener(fn(mode, configWithId)) - case Failure(e) => new Exception("Failed to decode flow step listener when submitting job", e) - } - - case _ => () - } - Success(flow) } catch { case err: Throwable => Failure(err) 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 b8dcb8c595..bc46e8024f 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala @@ -16,85 +16,157 @@ limitations under the License. package com.twitter.scalding import java.io.{ File, InputStream, OutputStream } -import java.util.{ UUID, Properties } - -import cascading.scheme.Scheme -import cascading.scheme.local.{ TextLine => CLTextLine, TextDelimited => CLTextDelimited } -import cascading.scheme.hadoop.{ - TextLine => CHTextLine, - TextDelimited => CHTextDelimited, - SequenceFile => CHSequenceFile -} +import java.util.{ Properties, UUID } + +import cascading.tuple.{ Tuple => CTuple } +import cascading.scheme.{ NullScheme, Scheme } +import cascading.scheme.local.{ TextDelimited => CLTextDelimited, TextLine => CLTextLine } +import cascading.scheme.hadoop.{ SequenceFile => CHSequenceFile, TextDelimited => CHTextDelimited, TextLine => CHTextLine } import cascading.tap.hadoop.Hfs import cascading.tap.MultiSourceTap import cascading.tap.SinkMode import cascading.tap.Tap import cascading.tap.local.FileTap -import cascading.tuple.Fields - +import cascading.tuple.{ Fields, Tuple } import com.etsy.cascading.tap.local.LocalTap -import com.twitter.algebird.{ Semigroup, MapAlgebra } - +import com.twitter.algebird.{ MapAlgebra, Semigroup } import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{ FileStatus, PathFilter, Path } +import org.apache.hadoop.fs.{ FileStatus, Path, PathFilter } import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapred.OutputCollector import org.apache.hadoop.mapred.RecordReader -import scala.util.{ Try, Success, Failure } +import scala.util.{ Failure, Success, Try } -/** - * A base class for sources that take a scheme trait. - */ -abstract class SchemedSource extends Source { +trait LocalSchemedSource { + protected def localUnsupported = throw ModeException("Cascading local mode not supported for: " + toString) /** The scheme to use if the source is local. */ - def localScheme: Scheme[Properties, InputStream, OutputStream, _, _] = - throw ModeException("Cascading local mode not supported for: " + toString) + def localScheme: LocalSchemeInstance.SchemeType = localUnsupported + + /** The path to use if the source is local. */ + def localPaths: Iterable[String] = localUnsupported + + // By default, we write to the last path for local paths + def localWritePath: String = localPaths.last +} + +trait HdfsSchemedSource { + protected def hdfsUnsupported = throw ModeException("Cascading HDFS storage mode not supported for: " + toString) /** The scheme to use if the source is on hdfs. */ - def hdfsScheme: Scheme[JobConf, RecordReader[_, _], OutputCollector[_, _], _, _] = - throw ModeException("Cascading Hadoop mode not supported for: " + toString) + def hdfsScheme: HadoopSchemeInstance.SchemeType = hdfsUnsupported + + /** The path to use if the source is local. */ + def hdfsPaths: Iterable[String] = hdfsUnsupported + + // By default, we write to the last path for hdfs paths + def hdfsWritePath: String = hdfsPaths.last +} + +/** + * A base class for sources that take a scheme trait. + */ +abstract class SchemedSource extends Source with LocalSchemedSource with HdfsSchemedSource { + // /** The path to use if the source is on hdfs. */ + //def hdfsPath: String = + // throw ModeException("Cascading Hadoop mode not supported for: " + toString) // The mode to use for output taps determining how conflicts with existing output are handled. val sinkMode: SinkMode = SinkMode.REPLACE + + override def createTap(readOrWrite: AccessMode)(implicit mode: Mode): Tap[_, _, _] = + mode.storageMode.createTap(this, readOrWrite, mode, sinkMode) + + override def validateTaps(mode: Mode): Unit = + mode.storageMode.validateTap(this) } -trait HfsTapProvider { - def createHfsTap(scheme: Scheme[JobConf, RecordReader[_, _], OutputCollector[_, _], _, _], - path: String, - sinkMode: SinkMode): Hfs = +trait HfsTapProvider extends HdfsSchemedSource { + + protected def createHfsTap(scheme: Scheme[JobConf, RecordReader[_, _], OutputCollector[_, _], _, _], + path: String, sinkMode: SinkMode): Hfs = new Hfs( Hadoop2SchemeInstance(scheme), path, sinkMode) -} -private[scalding] object CastFileTap { - // The scala compiler has problems with the generics in Cascading - def apply(tap: FileTap): Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]] = - tap.asInstanceOf[Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]]] -} + def createHdfsWriteTap(path: String, sinkMode: SinkMode): Tap[_, _, _] = createHfsTap(hdfsScheme, path, sinkMode).asInstanceOf[Tap[_, _, _]] + final def createHdfsWriteTap(sinkMode: SinkMode): Tap[_, _, _] = createHdfsWriteTap(hdfsWritePath, sinkMode) -/** - * A trait which provides a method to create a local tap. - */ -trait LocalSourceOverride extends SchemedSource { - /** A path to use for the local tap. */ - def localPaths: Iterable[String] + /** + * Determines if a path is 'valid' for this source. In strict mode all paths must be valid. + * In non-strict mode, all invalid paths will be filtered out. + * + * Subclasses can override this to validate paths. + * + * The default implementation is a quick sanity check to look for missing or empty directories. + * It is necessary but not sufficient -- there are cases where this will return true but there is + * in fact missing data. + * + * TODO: consider writing a more in-depth version of this method in [[TimePathedSource]] that looks for + * TODO: missing days / hours etc. + */ + protected def pathIsGood(globPattern: String, conf: Configuration) = FileSource.globHasNonHiddenPaths(globPattern, conf) - // By default, we write to the last path for local paths - def localWritePath: String = localPaths.last + // This is only called when Mode.sourceStrictness is true + protected def hdfsReadPathsAreGood(conf: Configuration) = { + hdfsPaths.forall { pathIsGood(_, conf) } + } + + /* + * Get all the set of valid paths based on source strictness. + */ + protected def goodHdfsPaths(strictSources: Boolean, conf: Configuration): Iterable[String] = { + if (strictSources) { + //we check later that all the paths are good + hdfsPaths + } else { + // If there are no matching paths, this is still an error, we need at least something: + hdfsPaths.filter{ pathIsGood(_, conf) } + } + } + + def createHdfsReadTap(strictSources: Boolean, conf: Configuration, mode: Mode, sinkMode: SinkMode): Tap[_, _, _] = { + val taps = + goodHdfsPaths(strictSources, conf) + .iterator + .map { path => CastHfsTap(createHfsTap(hdfsScheme, path, sinkMode)) } + .toList + + taps match { + case Nil => new IterableSource[Any](Nil).createTap(Read)(mode) + case one :: Nil => one + case many => new ScaldingMultiSourceTap(many) + } + } + + def validateHdfsTap(strictSources: Boolean, conf: Configuration): Unit = { + if (strictSources && (!hdfsReadPathsAreGood(conf))) { + throw new InvalidSourceException( + "[" + this.toString + "] Data is missing from one or more paths in: " + + hdfsPaths.toString) + } else if (!hdfsPaths.exists { pathIsGood(_, conf) }) { + //Check that there is at least one good path: + throw new InvalidSourceException( + "[" + this.toString + "] No good paths in: " + hdfsPaths.toString) + } + } +} + +trait LocalTapProvider extends LocalSchemedSource { + protected final def createLocalFileTap(scheme: Scheme[Properties, InputStream, OutputStream, _, _], + path: String, sinkMode: SinkMode): FileTap = new FileTap(scheme, path, sinkMode) /** * Creates a local tap. * * @param sinkMode The mode for handling output conflicts. - * @returns A tap. + * @return A tap. */ - def createLocalTap(sinkMode: SinkMode): Tap[JobConf, _, _] = { + def createLocalReadTap(sinkMode: SinkMode): Tap[_, _, _] = { val taps = localPaths.map { p: String => - CastFileTap(new FileTap(localScheme, p, sinkMode)) + CastFileTap(createLocalFileTap(localScheme, p, sinkMode)) }.toList taps match { @@ -103,6 +175,33 @@ trait LocalSourceOverride extends SchemedSource { case many => new ScaldingMultiSourceTap(many) } } + + def createLocalWriteTap(path: String, sinkMode: SinkMode): Tap[_, _, _] = createLocalFileTap(localScheme, path, sinkMode) + final def createLocalWriteTap(sinkMode: SinkMode): Tap[_, _, _] = createLocalWriteTap(localWritePath, sinkMode) + + def validateLocalTap(strictSources: Boolean): Unit = { + val files = localPaths.map{ p => new java.io.File(p) } + if (strictSources && !files.forall(_.exists)) { + throw new InvalidSourceException( + "[" + this.toString + s"] Data is missing from: ${localPaths.filterNot { p => new java.io.File(p).exists }}") + } else if (!files.exists(_.exists)) { + throw new InvalidSourceException( + "[" + this.toString + "] No good paths in: " + localPaths.toString) + } + } +} + +private[scalding] object CastFileTap { + // The scala compiler has problems with the generics in Cascading + def apply(tap: FileTap): Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]] = + tap.asInstanceOf[Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]]] +} + +/** + * A trait which provides a method to create a local tap. + */ +trait LocalSourceOverride extends SchemedSource with LocalTapProvider { + } object HiddenFileFilter extends PathFilter { @@ -181,130 +280,16 @@ object FileSource { /** * This is a base class for File-based sources */ -abstract class FileSource extends SchemedSource with LocalSourceOverride with HfsTapProvider { - - /** - * Determines if a path is 'valid' for this source. In strict mode all paths must be valid. - * In non-strict mode, all invalid paths will be filtered out. - * - * Subclasses can override this to validate paths. - * - * The default implementation is a quick sanity check to look for missing or empty directories. - * It is necessary but not sufficient -- there are cases where this will return true but there is - * in fact missing data. - * - * TODO: consider writing a more in-depth version of this method in [[TimePathedSource]] that looks for - * TODO: missing days / hours etc. - */ - protected def pathIsGood(globPattern: String, conf: Configuration) = FileSource.globHasNonHiddenPaths(globPattern, conf) - - def hdfsPaths: Iterable[String] - // By default, we write to the LAST path returned by hdfsPaths - def hdfsWritePath: String = hdfsPaths.last - - override def createTap(readOrWrite: AccessMode)(implicit mode: Mode): Tap[_, _, _] = { - mode match { - // TODO support strict in Local - case Local(_) => { - readOrWrite match { - case Read => createLocalTap(sinkMode) - case Write => new FileTap(localScheme, localWritePath, sinkMode) - } - } - case hdfsMode @ Hdfs(_, _) => readOrWrite match { - case Read => createHdfsReadTap(hdfsMode) - case Write => CastHfsTap(createHfsTap(hdfsScheme, hdfsWritePath, sinkMode)) - } - case _ => { - 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]] - } - } - - 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) - } - } - } - } - - // This is only called when Mode.sourceStrictness is true - protected def hdfsReadPathsAreGood(conf: Configuration) = { - hdfsPaths.forall { pathIsGood(_, conf) } - } +abstract class FileSource extends SchemedSource with LocalSourceOverride /*with LocalTapProvider*/ with HfsTapProvider { - /* - * This throws InvalidSourceException if: - * 1) we are in sourceStrictness mode and all sources are not present. - * 2) we are not in the above, but some source has no input whatsoever - * TODO this only does something for HDFS now. Maybe we should do the same for LocalMode - */ - override def validateTaps(mode: Mode): Unit = { - mode match { - case Hdfs(strict, conf) => { - if (strict && (!hdfsReadPathsAreGood(conf))) { - throw new InvalidSourceException( - "[" + this.toString + "] Data is missing from one or more paths in: " + - hdfsPaths.toString) - } else if (!hdfsPaths.exists { pathIsGood(_, conf) }) { - //Check that there is at least one good path: - throw new InvalidSourceException( - "[" + this.toString + "] No good paths in: " + hdfsPaths.toString) - } - } - - case Local(strict) => { - val files = localPaths.map{ p => new java.io.File(p) } - if (strict && !files.forall(_.exists)) { - throw new InvalidSourceException( - "[" + this.toString + s"] Data is missing from: ${localPaths.filterNot { p => new java.io.File(p).exists }}") - } else if (!files.exists(_.exists)) { - throw new InvalidSourceException( - "[" + this.toString + "] No good paths in: " + hdfsPaths.toString) - } - } - case _ => () - } - } - - /* - * Get all the set of valid paths based on source strictness. - */ - protected def goodHdfsPaths(hdfsMode: Hdfs): Iterable[String] = { - hdfsMode match { - //we check later that all the paths are good - case Hdfs(true, _) => hdfsPaths - // If there are no matching paths, this is still an error, we need at least something: - case Hdfs(false, conf) => hdfsPaths.filter{ pathIsGood(_, conf) } - } - } - - protected def createHdfsReadTap(hdfsMode: Hdfs): Tap[JobConf, _, _] = { - val taps = - goodHdfsPaths(hdfsMode) - .iterator - .map { path => CastHfsTap(createHfsTap(hdfsScheme, path, sinkMode)) } - .toList - - taps match { - case Nil => new IterableSource[Any](Nil).createTap(Read)(hdfsMode).asInstanceOf[Tap[JobConf, _, _]] - case one :: Nil => one - case many => new ScaldingMultiSourceTap(many) - } - } } class ScaldingMultiSourceTap(taps: Seq[Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]]]) extends MultiSourceTap[Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]], JobConf, RecordReader[_, _]](taps: _*) { private final val randomId = UUID.randomUUID.toString + override def getIdentifier() = randomId + override def hashCode: Int = randomId.hashCode } @@ -317,12 +302,24 @@ trait TextSourceScheme extends SchemedSource { val textEncoding: String = CHTextLine.DEFAULT_CHARSET override def localScheme = new CLTextLine(new Fields("offset", "line"), Fields.ALL, textEncoding) + override def hdfsScheme = HadoopSchemeInstance(new CHTextLine(CHTextLine.DEFAULT_SOURCE_FIELDS, textEncoding)) } -trait TextLineScheme extends TextSourceScheme with SingleMappable[String] { +trait TextLineScheme extends TextSourceScheme with SingleMappable[String] with TypedSink[String] { + //In textline, 0 is the byte position, the actual text string is in column 1 override def sourceFields = Dsl.intFields(Seq(1)) + + override def setter[U <: String]: TupleSetter[U] = new TupleSetter[U] { + override def apply(arg: U) = { + val tup = CTuple.size(1) + tup.set(0, arg) + tup + } + override def arity = 1 + } + override def sinkFields = sourceFields } /** @@ -348,7 +345,7 @@ trait DelimitedScheme extends SchemedSource { val safe = true //These should not be changed: - override def localScheme = new CLTextDelimited(fields, skipHeader, writeHeader, separator, strict, quote, types, safe) + override def localScheme = LocalSchemeInstance(new CLTextDelimited(fields, skipHeader, writeHeader, separator, strict, quote, types, safe)) override def hdfsScheme = { assert( @@ -361,6 +358,7 @@ trait DelimitedScheme extends SchemedSource { 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)) } @@ -384,12 +382,12 @@ trait SuccessFileSource extends FileSource { // we need to do some filtering on goodHdfsPaths to remove // empty dirs that we consider "good" but don't want to ask hadoop's FileInputFormat to read. - override protected def goodHdfsPaths(hdfsMode: Hdfs): Iterable[String] = { + override protected def goodHdfsPaths(strictSources: Boolean, conf: Configuration): Iterable[String] = { super - .goodHdfsPaths(hdfsMode) + .goodHdfsPaths(strictSources, conf) // some paths deemed "good" may actually be empty, and hadoop's FileInputFormat // doesn't like that. So we filter them away here. - .filter { p => FileSource.globHasNonHiddenPaths(p, hdfsMode.conf) } + .filter { p => FileSource.globHasNonHiddenPaths(p, conf) } } } @@ -398,7 +396,11 @@ trait SuccessFileSource extends FileSource { * Put another way, this runs a Hadoop tap outside of Hadoop in the Cascading local mode */ trait LocalTapSource extends LocalSourceOverride { - override def createLocalTap(sinkMode: SinkMode): Tap[JobConf, _, _] = { + + override def localScheme: Scheme[Properties, InputStream, OutputStream, _, _] = + new NullScheme[Properties, InputStream, OutputStream, AnyRef, AnyRef](hdfsScheme.getSourceFields, hdfsScheme.getSinkFields) + + override def createLocalReadTap(sinkMode: SinkMode): Tap[_, _, _] = { val taps = localPaths.map { p => new LocalTap(p, Hadoop2SchemeInstance(hdfsScheme), sinkMode).asInstanceOf[Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]]] }.toSeq @@ -413,14 +415,17 @@ trait LocalTapSource extends LocalSourceOverride { abstract class FixedPathSource(path: String*) extends FileSource { override def localPaths: Iterable[String] = path.toList + override def hdfsPaths: Iterable[String] = path.toList // `toString` is used by equals in JobTest, which causes // problems due to unstable collection type of `path` override def toString = getClass.getName + path.mkString("(", ",", ")") + override def hdfsWritePath: String = stripTrailing(super.hdfsWritePath) override def hashCode = toString.hashCode + override def equals(that: Any): Boolean = (that != null) && (that.toString == toString) /** @@ -538,3 +543,4 @@ case class MultipleDelimitedFiles(f: Fields, p: String*) extends FixedPathSource(p: _*) with DelimitedScheme { override val fields = f } + diff --git a/scalding-core/src/main/scala/com/twitter/scalding/IterableSource.scala b/scalding-core/src/main/scala/com/twitter/scalding/IterableSource.scala index 9a575bd9dd..b0a1d98f88 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/IterableSource.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/IterableSource.scala @@ -54,13 +54,7 @@ case class IterableSource[+T](@transient iter: Iterable[T], inFields: Fields = F if (readOrWrite == Write) { sys.error("IterableSource is a Read-only Source") } - mode match { - case Local(_) => new MemoryTap[InputStream, OutputStream](new NullScheme(fields, fields), asBuffer) - case Test(_) => new MemoryTap[InputStream, OutputStream](new NullScheme(fields, fields), asBuffer) - case Hdfs(_, _) => hdfsTap - case HadoopTest(_, _) => hdfsTap - case _ => throw ModeException("Unsupported mode for IterableSource: " + mode.toString) - } + mode.storageMode.createMemoryTap(readOrWrite, fields, asBuffer) } /** 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 1e4e37b0dd..23593c2af1 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Job.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Job.scala @@ -178,10 +178,7 @@ class Job(val args: Args) extends FieldConversions with java.io.Serializable { AppProps.addApplicationFramework(null, String.format("scalding:%s", scaldingVersion)) - val modeConf = mode match { - case h: HadoopMode => Config.fromHadoop(h.jobConf) - case _ => Config.empty - } + val modeConf = mode.executionMode.defaultConfig val init = base ++ modeConf @@ -513,7 +510,7 @@ trait CounterVerification extends Job { def verifyCountersInTest: Boolean = true override def listeners: List[FlowListener] = { - if (this.mode.isInstanceOf[TestMode] && !this.verifyCountersInTest) { + if (this.mode.isTesting && !this.verifyCountersInTest) { super.listeners } else { super.listeners :+ new StatsFlowListener(this.verifyCounters) 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 689f6cef48..7899edde51 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/JobTest.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/JobTest.scala @@ -21,6 +21,7 @@ import scala.annotation.tailrec import cascading.tuple.Tuple import cascading.tuple.TupleEntry import cascading.stats.CascadingStats +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapred.JobConf import scala.util.Try @@ -159,12 +160,18 @@ class JobTest(cons: (Args) => Job) { this } - def runHadoop = { + @deprecated("please use runWithMinicluster instead", "2016-10-20") + def runHadoop = runWithMinicluster + + def runWithMinicluster = { runJob(initJob(true), true) this } - def runHadoopWithConf(conf: JobConf) = { + @deprecated("please use runWithMiniclusterAndConf instead", "2016-10-20") + def runHadoopWithConf(conf: JobConf) = runWithMiniclusterAndConf(conf) + + def runWithMiniclusterAndConf(conf: Configuration) = { runJob(initJob(true, Some(conf)), true) this } @@ -178,20 +185,24 @@ class JobTest(cons: (Args) => Job) { } // Registers test files, initializes the global mode, and creates a job. - private def initJob(useHadoop: Boolean, job: Option[JobConf] = None): Job = { + private def initJob(useHadoop: Boolean, job: Option[Configuration] = None): Job = { // Create a global mode to use for testing. - val testMode: TestMode = - if (useHadoop) { - val conf = job.getOrElse(new JobConf) + val testMode: TestMode = { + val conf = job.getOrElse(new Configuration) + val modeName: String = if (useHadoop) { // Set the polling to a lower value to speed up tests: conf.set("jobclient.completion.poll.interval", "100") conf.set("cascading.flow.job.pollinginterval", "5") // Work around for local hadoop race conf.set("mapred.local.dir", "/tmp/hadoop/%s/mapred/local".format(java.util.UUID.randomUUID)) - HadoopTest(conf, sourceMap) + + "anyCluster-test" // pick up any suitable mode depending on classpath. TODO: repalce useHadoop:Boolean with a better-controlled variable. } else { - Test(sourceMap) + "local-test" } + Mode.test(modeName, conf, sourceMap) + } + testMode.registerTestFiles(fileSet) val args = new Args(argsMap) @@ -199,6 +210,12 @@ class JobTest(cons: (Args) => Job) { cons(Mode.putMode(testMode, args)) } + protected def jobMode(job: Job): TestMode = + job.mode match { + case m: TestMode => m + case _ => throw new IllegalArgumentException("The Job is not running under a TestMode. Has this job been created from this initJob() method?") // shouldn't happen + } + @tailrec private final def runJob(job: Job, runNext: Boolean): Unit = { // Disable automatic cascading update @@ -222,17 +239,7 @@ class JobTest(cons: (Args) => Job) { next match { case Some(nextjob) => runJob(nextjob, runNext) case None => { - job.mode match { - case hadoopTest @ HadoopTest(_, _) => { - /* NOTE: `HadoopTest.finalize` depends on `sinkSet` matching the set of - * "keys" in the `sourceMap`. Do not change the following line unless - * you also modify the `finalize` function accordingly. - */ - // The sinks are written to disk, we need to clean them up: - sinkSet.foreach{ hadoopTest.finalize(_) } - } - case _ => () - } + jobMode(job).finalize(sinkSet) // Now it is time to check the test conditions: callbacks.foreach { cb => cb() } statsCallbacks.foreach { cb => cb(job.scaldingCascadingStats.get) } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Mode.scala b/scalding-core/src/main/scala/com/twitter/scalding/Mode.scala index 944c1a7d64..bae085d5c3 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Mode.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Mode.scala @@ -15,20 +15,28 @@ limitations under the License. */ package com.twitter.scalding -import java.io.File -import java.util.{ UUID, Properties } +import java.io._ +import java.lang.reflect.Constructor +import java.nio.charset.Charset +import java.nio.file.{ Files, Paths } +import java.util.{ Properties, UUID } import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{ FileSystem, Path } -import org.apache.hadoop.mapred.JobConf - -import cascading.flow.{ FlowProcess, FlowConnector, FlowDef, Flow } +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapred.{ JobConf, OutputCollector, RecordReader } +import cascading.flow._ import cascading.flow.local.LocalFlowConnector import cascading.flow.local.LocalFlowProcess +import cascading.flow.planner.BaseFlowStep import cascading.property.AppProps -import cascading.tap.Tap -import cascading.tuple.Tuple -import cascading.tuple.TupleEntryIterator +import cascading.scheme.NullScheme +import cascading.tap.{ SinkMode, Tap } +import cascading.tuple.{ Fields, Tuple, TupleEntryIterator } +import com.twitter.maple.tap.MemorySourceTap +import com.twitter.scalding.StorageMode.TemporarySource +import com.twitter.scalding.filecache.{ CachedFile, UncachedFile } +import com.twitter.scalding.reducer_estimation.ReducerEstimatorStepStrategy +import com.twitter.scalding.typed.MemorySink import scala.annotation.tailrec import scala.collection.JavaConverters._ @@ -36,20 +44,24 @@ import scala.collection.mutable.Buffer import scala.collection.mutable.{ Map => MMap } import scala.collection.mutable.{ Set => MSet } import scala.util.{ Failure, Success } - import org.slf4j.LoggerFactory +import scala.annotation.meta.param +import scala.collection.{ Map, mutable } + case class ModeException(message: String) extends RuntimeException(message) case class ModeLoadException(message: String, origin: ClassNotFoundException) extends RuntimeException(origin) object Mode { + private val LOG = LoggerFactory.getLogger(this.getClass) + /** * This is a Args and a Mode together. It is used purely as * a work-around for the fact that Job only accepts an Args object, * but needs a Mode inside. */ - private class ArgsWithMode(argsMap: Map[String, List[String]], val mode: Mode) extends Args(argsMap) { + private class ArgsWithMode(argsMap: scala.Predef.Map[String, List[String]], val mode: Mode) extends Args(argsMap) { override def +(keyvals: (String, Iterable[String])): Args = new ArgsWithMode(super.+(keyvals).m, mode) } @@ -63,72 +75,258 @@ object Mode { case _ => None } - val CascadingFlowConnectorClassKey = "cascading.flow.connector.class" - val CascadingFlowProcessClassKey = "cascading.flow.process.class" + val KnownModesMap = Seq( + /* The modes are referenced by class *names* to avoid tightly coupling all the jars together. This class, Mode, + must be loadable whether one of these is missing or not (it is unlikely all can be loaded at the same time) + */ + "local" -> "com.twitter.scalding.Local", + "hadoop2-mr1" -> "com.twitter.scalding.Hadoop2Mr1Mode", + "hadoop2-tez" -> "com.twitter.scalding.TezMode", "tez" -> "com.twitter.scalding.TezMode", + "hadoop1" -> "com.twitter.scalding.LegacyHadoopMode", + "hdfs" -> "com.twitter.scalding.LegacyHadoopMode", + "flink" -> "com.twitter.scalding.FlinkMode") + + val KnownTestModesMap = Seq( + "local-test" -> "com.twitter.scalding.Test", + "hadoop1-test" -> "com.twitter.scalding.HadoopTest", + "hadoop2-mr1-test" -> "com.twitter.scalding.Hadoop2Mr1TestMode", + "tez-test" -> "com.twitter.scalding.TezTestMode", + "flink-test" -> "com.twitter.scalding.FlinkTestMode") + // TODO: define hadoop2-mr1 (easy), tez and flink (less easy) classes. + + private def getModeConstructor[M <: Mode](clazzName: String, types: Class[_]*) = + try { + val clazz: Class[M] = Class.forName(clazzName).asInstanceOf[Class[M]] + val ctor = clazz.getConstructor(types: _*) + Some(ctor) + } catch { + case ncd: ClassNotFoundException => None + } - val DefaultHadoopFlowConnector = "cascading.flow.hadoop.HadoopFlowConnector" - val DefaultHadoopFlowProcess = "cascading.flow.hadoop.HadoopFlowProcess" + private def filterKnownModes[M <: Mode](clazzNames: Iterable[(String, String)], types: Class[_]*): Map[String, Constructor[M]] = + clazzNames.map { + case (modeName, clazzName) => + (modeName, getModeConstructor[M](clazzName, types: _*)) + }.collect { case (modeName, Some(ctor)) => (modeName, ctor) }.toMap - val DefaultHadoop2Mr1FlowConnector = "cascading.flow.hadoop2.Hadoop2MR1FlowConnector" - val DefaultHadoop2Mr1FlowProcess = "cascading.flow.hadoop.HadoopFlowProcess" // no Hadoop2MR1FlowProcess as of Cascading 3.0.0-wip-75? + private val RegularModeTypes = Seq(classOf[Boolean], classOf[Configuration]) - val DefaultHadoop2TezFlowConnector = "cascading.flow.tez.Hadoop2TezFlowConnector" - val DefaultHadoop2TezFlowProcess = "cascading.flow.tez.Hadoop2TezFlowProcess" + private lazy val ReallyAvailableModes = filterKnownModes[Mode](KnownModesMap, RegularModeTypes: _*) + + lazy private val KnownModesDict = KnownModesMap.toMap + val KnownModeNames = KnownModesMap.map(_._1) // same order as KnownModesMap (priorities) + + private val TestModeTypes = Seq(classOf[Configuration], classOf[(Source) => Option[Buffer[Tuple]]]) + private lazy val ReallyAvailableTestModes = filterKnownModes[TestMode](KnownTestModesMap, TestModeTypes: _*) + + def test(modeName: String, config: Configuration, bufferMap: (Source) => Option[Buffer[Tuple]]): TestMode = { + + val modeCtor = KnownTestModesMap + .filter { case (flag, _) => (flag == modeName) || ((modeName == "anyCluster-test") && (flag != "local-test")) } + .map { case (flag, _) => ReallyAvailableTestModes.get(flag) } + .collect { case Some(ctor) => ctor } + .headOption + + construct[TestMode](modeCtor, config, bufferMap) + } // This should be passed ALL the args supplied after the job name def apply(args: Args, config: Configuration): Mode = { val strictSources = args.boolean("tool.partialok") == false if (!strictSources) { - // TODO we should do smarter logging here - println("[Scalding:INFO] using --tool.partialok. Missing log data won't cause errors.") + LOG.info("using --tool.partialok. Missing log data won't cause errors.") } - if (args.boolean("local")) - Local(strictSources) - else if (args.boolean("hdfs")) /* FIXME: should we start printing deprecation warnings ? It's okay to set manually c.f.*.class though */ - Hdfs(strictSources, config) - else if (args.boolean("hadoop1")) { - config.set(CascadingFlowConnectorClassKey, DefaultHadoopFlowConnector) - config.set(CascadingFlowProcessClassKey, DefaultHadoopFlowProcess) - Hdfs(strictSources, config) - } else if (args.boolean("hadoop2-mr1")) { - config.set(CascadingFlowConnectorClassKey, DefaultHadoop2Mr1FlowConnector) - config.set(CascadingFlowProcessClassKey, DefaultHadoop2Mr1FlowProcess) - Hdfs(strictSources, config) - } else if (args.boolean("hadoop2-tez")) { - config.set(CascadingFlowConnectorClassKey, DefaultHadoop2TezFlowConnector) - config.set(CascadingFlowProcessClassKey, DefaultHadoop2TezFlowProcess) - Hdfs(strictSources, config) - } else - throw ArgsException("[ERROR] Mode must be one of --local, --hadoop1, --hadoop2-mr1, --hadoop2-tez or --hdfs, you provided none") + lazy val autoMode = if (args.boolean("autoCluster")) { + KnownModeNames.toStream + .filterNot(_ == "local") + .map(ReallyAvailableModes.get).collect { case Some(ctor) => ctor }.headOption + } else None + + val ctorFinder = (clazzName: String) => getModeConstructor[Mode](clazzName, RegularModeTypes: _*) + + lazy val requestedMode = KnownModeNames.find(args.boolean) // scanned in the order of KnownModesMap + lazy val requestedModeCtor = requestedMode.flatMap(KnownModesDict.get).flatMap(ctorFinder) + + lazy val explicitModeClazzName = args.optional("modeClass") // use this to supply a custom Mode class from the command line + lazy val explicitModeClazzCtor = explicitModeClazzName.flatMap(ctorFinder) + + val modeCtor = explicitModeClazzCtor.orElse(autoMode).orElse(requestedModeCtor) + + construct(modeCtor, boolean2Boolean(strictSources): AnyRef, config) } + + private def construct[M <: Mode](modeCtor: Option[Constructor[M]], args: AnyRef*): M = { + modeCtor match { + case Some(ctor) => + try { + ctor.newInstance(args: _*) + } catch { + case ncd: ClassNotFoundException => { + throw new ModeLoadException("Failed to load Scalding Mode class (missing runtime jar?) " + ctor, ncd) + } + } + case None => + throw ArgsException("[ERROR] Mode must be one of --local, --hadoop1, --hadoop2-mr1, --hadoop2-tez or --hdfs, you provided none or none was found in the ClassPath") + } + } + } -trait Mode extends java.io.Serializable { - /* +trait ExecutionMode extends java.io.Serializable { + /** * Using a new FlowProcess, which is only suitable for reading outside * of a map/reduce job, open a given tap and return the TupleEntryIterator */ def openForRead(config: Config, tap: Tap[_, _, _]): TupleEntryIterator + /** Create a new FlowConnector for this cascading planner */ + def newFlowConnector(props: Config): FlowConnector + + /** + * Create a new Flow using the default FlowConnector, and ensure any fabric-specific flow configuration is complete. + */ + def newFlow(config: Config, flowDef: FlowDef): Flow[_] = { + val conf2 = setupCounterCreation(config) + + val flowcon = newFlowConnector(conf2) + val flow = flowcon.connect(flowDef) + flow + } + + /** + * @return true if a checkpoint is required before a hash join on the given platform + */ + private[scalding] def getHashJoinAutoForceRight: Boolean = false + + private[scalding] def defaultConfig: Config = Config.empty + + /** + * @return a Config which includes the class name to use for creating Statistics Counters. + * Please see the comment in CounterImpl for details (this is really fabric implementation details) + */ + private[scalding] def setupCounterCreation(fp: Config): Config = + fp + (CounterImpl.CounterImplClass -> classOf[GenericFlowPCounterImpl].getCanonicalName) + +} + +trait StorageMode extends java.io.Serializable { + import StorageMode._ + + /** + * @return true if the file exists on the current filesystem. + */ + def fileExists(filename: String): Boolean + + def temporaryTypedSource[T]: TemporarySource[T] + + /** + * Create a Tap for the purpose set out in {{readOrWrite}}, using the scheme provided by {{schemedSource}} + */ + def createTap(schemedSource: SchemedSource, readOrWrite: AccessMode, mode: Mode, sinkMode: SinkMode): Tap[_, _, _] + + /** + * Verify whether sources are valid, given the applicable source strictness mode + * + * @throws InvalidSourceException if sources are missing + */ + def validateTap(schemedSource: SchemedSource): Unit + + /** + * Create a memory-only tap for the purpose of spooling some tuples from memory. + * On Local modes, it is also possible to write to a memory tap. + */ + def createMemoryTap(readOrWrite: AccessMode, fields: Fields, asBuffer: mutable.Buffer[Tuple]): Tap[_, _, _] + + /** Create a write-only tap which simply discards the results */ + def createNullTap: Tap[_, _, _] + + // access this through the DistributedCacheFile object + private[scalding] def addCachedFile(file: UncachedFile): CachedFile + + /** Read the content of a file into a string, UTF-8 encoding is assumed */ + def readFromFile(filename: String): String + /** Read the content of a file into a string, UTF-8 encoding is assumed */ + def writeToFile(filename: String, text: String): Unit +} + +object StorageMode { + trait TemporarySource[T] { + def sink(conf: Config): TypedSink[T] + def downstreamPipe(conf: Config): TypedPipe[T] + } +} + +trait Mode extends java.io.Serializable { + def storageMode: StorageMode + def executionMode: ExecutionMode + + def strictSources: Boolean + def name: String + + // Override this if you define new testing modes + def isTesting: Boolean = false + + /** + * Using a new FlowProcess, which is only suitable for reading outside + * of a map/reduce job, open a given tap and return the TupleEntryIterator + */ + final def openForRead(config: Config, tap: Tap[_, _, _]): TupleEntryIterator = executionMode.openForRead(config, tap) + + /** Create a new FlowConnector for this cascading planner */ + protected // will probably have to end up just "@deprecated" + final def newFlowConnector(config: Config): FlowConnector = newFlowConnector(config) + + /** Create a new Flow for this cascading planner */ + def newFlow(config: Config, flowDef: FlowDef): Flow[_] = executionMode.newFlow(config, flowDef) + + /** + * @return true if the file exists on the current filesystem. + */ + final def fileExists(filename: String): Boolean = storageMode.fileExists(filename) + @deprecated("A Config is needed, especially if any kryo serialization has been used", "0.12.0") final def openForRead(tap: Tap[_, _, _]): TupleEntryIterator = openForRead(Config.defaultFrom(this), tap) +} - // Returns true if the file exists on the current filesystem. - def fileExists(filename: String): Boolean - /** Create a new FlowConnector for this cascading planner */ - def newFlowConnector(props: Config): FlowConnector +/** + * Any Mode that runs using local resources only + */ +trait LocalMode extends Mode { +} + +/** + * Any Mode that runs over a cluster (as opposed to Local) is a ClusterMode. + */ +trait ClusterMode extends Mode { } -trait HadoopMode extends Mode { +/** + * Any ClusterMode whose main configuration is org.apache.hadoop.conf.Configuration is a HadoopFamilyMode + */ +trait HadoopFamilyMode extends ClusterMode { + def jobConf: Configuration +} +/** + * The "HadoopMode" is actually an alias for "a mode running on a fabric that ultimately runs using an execution + * engine compatible with some of the Hadoop technology stack (may or may not include Hadoop 1.x, YARN, etc.) + */ +trait HadoopExecutionModeBase[ConfigType <: Configuration] + extends ExecutionMode { def jobConf: Configuration + def mode: Mode + + override def getHashJoinAutoForceRight: Boolean = { + val config = Config.fromHadoop(jobConf) + config.getHashJoinAutoForceRight + } - override def newFlowConnector(conf: Config) = { + protected def enrichConfig(conf: Config): Map[AnyRef, AnyRef] = { val asMap = conf.toMap.toMap[AnyRef, AnyRef] // linter:ignore val jarKey = AppProps.APP_JAR_CLASS - val finalMap = conf.getCascadingAppJar match { + conf.getCascadingAppJar match { case Some(Success(cls)) => asMap + (jarKey -> cls) case Some(Failure(err)) => // This may or may not cause the job to fail at submission, let's punt till then @@ -140,46 +338,94 @@ trait HadoopMode extends Mode { asMap - jarKey case None => asMap } + } - val flowConnectorClass = jobConf.get(Mode.CascadingFlowConnectorClassKey, Mode.DefaultHadoopFlowConnector) + protected def newFlowConnector(rawConf: java.util.Map[AnyRef, AnyRef]): FlowConnector + override def newFlowConnector(conf: Config) = newFlowConnector(enrichConfig(conf).asJava) - try { - val clazz = Class.forName(flowConnectorClass) - val ctor = clazz.getConstructor(classOf[java.util.Map[_, _]]) - ctor.newInstance(finalMap.asJava).asInstanceOf[FlowConnector] - } catch { - case ncd: ClassNotFoundException => { - throw new ModeLoadException("Failed to load Cascading flow connector class " + flowConnectorClass, ncd) - } - } - } + protected def newFlowProcess(conf: ConfigType): FlowProcess[ConfigType] + + protected def defaultConfiguration: ConfigType // TODO unlike newFlowConnector, this does not look at the Job.config override def openForRead(config: Config, tap: Tap[_, _, _]) = { - val htap = tap.asInstanceOf[Tap[JobConf, _, _]] - val conf = new JobConf(true) // initialize the default config + val htap = tap.asInstanceOf[Tap[ConfigType, _, _]] // copy over Config - config.toMap.foreach{ case (k, v) => conf.set(k, v) } - - val flowProcessClass = jobConf.get(Mode.CascadingFlowProcessClassKey, Mode.DefaultHadoopFlowProcess) - - val fp = try { - val clazz = Class.forName(flowProcessClass) - val ctor = clazz.getConstructor(classOf[JobConf]) - ctor.newInstance(conf).asInstanceOf[FlowProcess[JobConf]] - } catch { - case ncd: ClassNotFoundException => { - throw new ModeLoadException("Failed to load Cascading flow process class " + flowProcessClass, ncd) - } - } + val conf = defaultConfiguration + config.toMap.foreach { case (k, v) => conf.set(k, v) } + val fp = newFlowProcess(conf) htap.retrieveSourceFields(fp) htap.sourceConfInit(fp, conf) htap.openForRead(fp) } + + override def defaultConfig: Config = Config.fromHadoop(jobConf) - Config.IoSerializationsKey + + override def newFlow(config: Config, flowDef: FlowDef): Flow[_] = { + val flow = super.newFlow(config, flowDef) + + applyDescriptionsOnSteps(flow) + applyReducerEstimationStrategies(flow, config) + /* NOTE: it is not yet known whether the descriptions and the Reducer Estimation Strategies are *relevant* for + non-MR engines (Tez, Flink). + TODO: If they are, the actual way to convey these results *to* the fabric are not yet plugged in. + */ + + flow + } + + private def getIdentifierOpt(descriptions: Seq[String]): Option[String] = { + if (descriptions.nonEmpty) Some(descriptions.distinct.mkString(", ")) else None + } + + private def updateStepConfigWithDescriptions(step: BaseFlowStep[ConfigType]): Unit = { + val conf = step.getConfig + getIdentifierOpt(ExecutionContext.getDesc(step)).foreach(descriptionString => { + conf.set(Config.StepDescriptions, descriptionString) + }) + } + + def applyDescriptionsOnSteps(flow: Flow[_]): Unit = { + val flowSteps = flow.getFlowSteps.asScala + flowSteps.foreach { + case baseFlowStep: BaseFlowStep[ConfigType @unchecked] => + updateStepConfigWithDescriptions(baseFlowStep) + } + } + + def applyReducerEstimationStrategies(flow: Flow[_], config: Config): Unit = { + + // if any reducer estimators have been set, register the step strategy + // which instantiates and runs them + + val reducerEstimatorStrategy: Seq[FlowStepStrategy[JobConf]] = config.get(Config.ReducerEstimators).toList.map(_ => ReducerEstimatorStepStrategy) + + val otherStrategies: Seq[FlowStepStrategy[JobConf]] = config.getFlowStepStrategies.map { + case Success(fn) => fn(mode, config) + case Failure(e) => throw new Exception("Failed to decode flow step strategy when submitting job", e) + } + + val optionalFinalStrategy = FlowStepStrategies().sumOption(reducerEstimatorStrategy ++ otherStrategies) + + optionalFinalStrategy.foreach { strategy => + flow.setFlowStepStrategy(strategy) + } + + config.getFlowListeners.foreach { + case Success(fn) => flow.addListener(fn(mode, config)) + case Failure(e) => throw new Exception("Failed to decode flow listener", e) + } + + config.getFlowStepListeners.foreach { + case Success(fn) => flow.addStepListener(fn(mode, config)) + case Failure(e) => new Exception("Failed to decode flow step listener when submitting job", e) + } + } + } -trait CascadingLocal extends Mode { +class CascadingLocalExecutionMode extends ExecutionMode { override def newFlowConnector(conf: Config) = new LocalFlowConnector(conf.toMap.toMap[AnyRef, AnyRef].asJava) // linter:ignore @@ -194,32 +440,245 @@ trait CascadingLocal extends Mode { } } -// Mix-in trait for test modes; overrides fileExists to allow the registration -// of mock filenames for testing. -trait TestMode extends Mode { +trait TestStorageMode extends StorageMode { + import StorageMode._ + private var fileSet = Set[String]() - def registerTestFiles(files: Set[String]) = fileSet = files + def registerTestFiles(files: Iterable[String]) = fileSet = files.toSet override def fileExists(filename: String): Boolean = fileSet.contains(filename) } -case class Hdfs(strict: Boolean, @transient conf: Configuration) extends HadoopMode { - override def jobConf = conf +trait LocalStorageModeCommon extends StorageMode { + def createMemoryTap(createMemoryTap: AccessMode, fields: Fields, tupleBuffer: mutable.Buffer[Tuple]): Tap[_, _, _] = + new MemoryTap[InputStream, OutputStream](new NullScheme(fields, fields), tupleBuffer) + + def createNullTap: Tap[_, _, _] = new NullTap[Properties, InputStream, OutputStream, Any, Any] + + def addCachedFile(file: UncachedFile): CachedFile = file.addLocal + + def readFromFile(filename: String): String = + try { + new String(Files.readAllBytes(Paths.get(filename)), Charset.forName("UTF-8")) + } catch { + case e: IOException => + throw new RuntimeException(e) + } + + def writeToFile(filename: String, text: String): Unit = + try { + val br = new BufferedWriter(new OutputStreamWriter(new FileOutputStream(filename), Charset.forName("UTF-8"))) + br.write(text) + br.close() + } catch { + case e: IOException => throw new RuntimeException(e) + } + + def temporaryTypedSource[T]: TemporarySource[T] = new TemporarySource[T] { + lazy val inMemoryDest = new MemorySink[T] + + override def sink(conf: Config): TypedSink[T] = inMemoryDest + + override def downstreamPipe(conf: Config): TypedPipe[T] = TypedPipe.from[T](inMemoryDest.readResults) + } + +} + +class LocalStorageMode(strictSources: Boolean) extends TestStorageMode with LocalStorageModeCommon { + override def fileExists(filename: String): Boolean = new File(filename).exists + + def createTap(schemedSource: SchemedSource, readOrWrite: AccessMode, mode: Mode, sinkMode: SinkMode): Tap[_, _, _] = { + schemedSource match { + case ltp: LocalTapProvider => + readOrWrite match { + case Read => ltp.createLocalReadTap(sinkMode) + case Write => ltp.createLocalWriteTap(sinkMode) + } + case _ => throw new ModeException("Cascading Local storage mode not supported for: " + schemedSource.toString) + } + } + + def validateTap(schemedSource: SchemedSource): Unit = + schemedSource match { + case ltp: LocalTapProvider => ltp.validateLocalTap(strictSources) + case _ => throw new ModeException("Cascading Local storage mode not supported for: " + schemedSource.toString) + } + +} + +trait HdfsStorageModeCommon extends StorageMode { + def jobConf: Configuration + + def createMemoryTap(readOrWrite: AccessMode, fields: Fields, tupleBuffer: mutable.Buffer[Tuple]): Tap[_, _, _] = + if (readOrWrite == Read) + new MemorySourceTap(tupleBuffer.asJava, fields) + else + throw new UnsupportedOperationException(s"on non-Local storage mode, cannot build MemoryTap for ${readOrWrite} operation") + + def temporaryTypedSource[T]: TemporarySource[T] = new TemporarySource[T] { + val cachedRandomUUID = java.util.UUID.randomUUID + + def hadoopTypedSource(conf: Config): TypedSource[T] with TypedSink[T] = { + // come up with unique temporary filename, use the config here + // TODO: refactor into TemporarySequenceFile class + val tmpDir = conf.get("hadoop.tmp.dir") + .orElse(conf.get("cascading.tmp.dir")) + .getOrElse("/tmp") + + val tmpSeq = tmpDir + "/scalding/snapshot-" + cachedRandomUUID + ".seq" + source.TypedSequenceFile[T](tmpSeq) + } + + override def sink(conf: Config): TypedSink[T] = hadoopTypedSource(conf: Config) + + override def downstreamPipe(conf: Config): TypedPipe[T] = TypedPipe.from[T](hadoopTypedSource(conf)) + } + + def createNullTap: Tap[_, _, _] = new NullTap[JobConf, RecordReader[_, _], OutputCollector[_, _], Any, Any] + + def addCachedFile(file: UncachedFile): CachedFile = file.addHdfs(jobConf) + + def readFromFile(filename: String): String = + try { + val pt = new Path(filename) + val fs = pt.getFileSystem(jobConf) + fs.open(pt).readUTF + } catch { + case e: IOException => + throw new RuntimeException(e) + } + + def writeToFile(filename: String, text: String): Unit = + try { + val pt = new Path(filename) + val fs = pt.getFileSystem(jobConf) + val br = new BufferedWriter(new OutputStreamWriter(fs.create(pt, true))) + + br.write(text) + br.close() + } catch { + case e: IOException => throw new RuntimeException(e) + } + +} + +class HdfsTestStorageMode(strictSources: Boolean, @transient override val jobConf: Configuration, @transient pathAllocator: Source => String) + extends StorageMode with TestStorageMode with HdfsStorageModeCommon { + + override def createTap(schemedSource: SchemedSource, readOrWrite: AccessMode, + mode: Mode, sinkMode: SinkMode): Tap[_, _, _] = + (mode, schemedSource) match { + case (testMode: TestMode, tapProvider: HfsTapProvider) => + TestTapFactory(schemedSource, schemedSource.hdfsScheme, sinkMode).createHdfsTap(readOrWrite, testMode, pathAllocator, tapProvider) + + case (_: TestMode, _) => + throw new UnsupportedOperationException("HdfsTestStorageMode cannot create test tap: source doesn't provide Hfs taps") + + case _ => + throw new UnsupportedOperationException("HdfsTestStorageMode cannot create test tap in a non-testing mode") + } + + def validateTap(schemedSource: SchemedSource): Unit = () // no path validation in test mode + +} + +class LocalTestStorageMode extends TestStorageMode with LocalStorageModeCommon { + + override def createTap(schemedSource: SchemedSource, readOrWrite: AccessMode, mode: Mode, sinkMode: SinkMode): Tap[_, _, _] = + (mode, readOrWrite) match { + case (testMode: TestMode, Read) => + TestTapFactory(schemedSource, schemedSource.localScheme.getSourceFields, sinkMode).createLocalTap(readOrWrite, testMode) + case (testMode: TestMode, Write) => + TestTapFactory(schemedSource, schemedSource.localScheme.getSinkFields, sinkMode).createLocalTap(readOrWrite, testMode) + case _ => + throw new UnsupportedOperationException("LocalTestStorageMode cannot create test tap in a non-testing mode") + } + + def validateTap(schemedSource: SchemedSource): Unit = () // no path validation in test mode +} + +/** + * Any + */ +class HdfsStorageMode(strictSources: Boolean, @transient override val jobConf: Configuration) extends StorageMode with HdfsStorageModeCommon { + import StorageMode._ override def fileExists(filename: String): Boolean = { val path = new Path(filename) path.getFileSystem(jobConf).exists(path) } + + def createTap(schemedSource: SchemedSource, readOrWrite: AccessMode, mode: Mode, sinkMode: SinkMode): Tap[_, _, _] = { + schemedSource match { + case htp: HfsTapProvider => + readOrWrite match { + case Read => htp.createHdfsReadTap(strictSources, jobConf, mode, sinkMode) + case Write => CastHfsTap(htp.createHdfsWriteTap(sinkMode)) + } + case _ => throw new ModeException("Cascading HDFS storage mode not supported for: " + schemedSource.toString) + } + } + + def validateTap(schemedSource: SchemedSource): Unit = + schemedSource match { + case htp: HfsTapProvider => htp.validateHdfsTap(strictSources, jobConf) + case _ => throw new ModeException("Cascading HDFS storage mode not supported for: " + schemedSource.toString) + } } -case class HadoopTest(@transient conf: Configuration, - @transient buffers: Source => Option[Buffer[Tuple]]) - extends HadoopMode with TestMode { +case class Local(strictSources: Boolean) extends LocalMode { + // this auxiliary ctor is provided for compatibility with the dynamic load made by Mode.apply + def this(strictSources: Boolean, @(transient @param) dummyJobConf: Configuration) = this(strictSources) + + val name = "local" + + override val storageMode: StorageMode = new LocalStorageMode(strictSources) + override val executionMode: ExecutionMode = new CascadingLocalExecutionMode +} + +//object Local { +// def apply(strictSources: Boolean): Local = new Local(strictSources, new Configuration) +//} + +/** + * Memory only testing for unit tests + */ + +trait TestMode extends Mode { + override def storageMode: TestStorageMode + override def isTesting = true + def buffers: Source => Option[Buffer[Tuple]] + + def registerTestFiles(files: Iterable[String]): Unit = storageMode.registerTestFiles(files) + + /** + * Perform any closing activity required on a set of sinks, if any + * + * Note: this is typically used for test purposes. + * + * @param sinks a list of sinks to visit and on which to perform post-job activity + */ + def finalize(sinks: Iterable[Source]): Unit = () +} + +case class Test(override val buffers: (Source) => Option[Buffer[Tuple]]) extends LocalMode with TestMode { + + /* this is the ctor that is found via reflection by @{link Mode.test} */ + def this(unused: Configuration, buffers: (Source) => Option[Buffer[Tuple]]) = { + this(buffers) + } + + val strictSources = false + val name = "local-test" + + override val storageMode: TestStorageMode = new LocalTestStorageMode() + override val executionMode: ExecutionMode = new CascadingLocalExecutionMode +} + +trait HadoopFamilyTestMode extends HadoopFamilyMode with TestMode { // This is a map from source.toString to disk path private val writePaths = MMap[Source, String]() private val allPaths = MSet[String]() - override def jobConf = conf - @tailrec private def allocateNewPath(prefix: String, idx: Int): String = { val candidate = prefix + idx.toString @@ -234,14 +693,22 @@ case class HadoopTest(@transient conf: Configuration, } private val thisTestID = UUID.randomUUID - private val basePath = "/tmp/scalding/%s/".format(thisTestID) - // Looks up a local path to write the given source to + private lazy val basePath = { + val r = s"/tmp/scalding/${name}-${thisTestID}/" + new File(r).mkdirs() + r + } + + /** Looks up a local path to write the given source to */ def getWritePathFor(src: Source): String = { val rndIdx = new java.util.Random().nextInt(1 << 30) - writePaths.getOrElseUpdate(src, allocateNewPath(basePath + src.getClass.getName, rndIdx)) + writePaths.getOrElseUpdate(src, allocateNewPath(basePath + + src.toString + .replaceAll("[/\\\\\\$ .\\(\\),;:\t\\[\\]\\+'\"]", "_") + .replaceAll("_+", "_") + "_", rndIdx)) } - def finalize(src: Source): Unit = { + private def finalizeSink(src: Source): Unit = { /* The following `_.get` is only safe if `src` belongs to the source map. * This invariant is preserved by the `JobTest.sink` and `JobTest.runJob` * functions, and those functions have been documented accordingly to @@ -262,13 +729,11 @@ case class HadoopTest(@transient conf: Configuration, new File(path).delete() writePaths -= src } -} -case class Local(strictSources: Boolean) extends CascadingLocal { - override def fileExists(filename: String): Boolean = new File(filename).exists -} + override def finalize(sinks: Iterable[Source]): Unit = { + sinks.foreach(finalizeSink) + // since all files finalized already, this should be empty now. + new File(basePath).delete + } -/** - * Memory only testing for unit tests - */ -case class Test(buffers: (Source) => Option[Buffer[Tuple]]) extends TestMode with CascadingLocal +} \ No newline at end of file diff --git a/scalding-core/src/main/scala/com/twitter/scalding/PartitionSource.scala b/scalding-core/src/main/scala/com/twitter/scalding/PartitionSource.scala index 88030f98d0..e406b32ea0 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/PartitionSource.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/PartitionSource.scala @@ -15,6 +15,7 @@ limitations under the License. */ package com.twitter.scalding +import cascading.scheme.Scheme import cascading.tap.hadoop.Hfs import cascading.tap.hadoop.{ PartitionTap => HPartitionTap } import cascading.tap.local.FileTap @@ -24,11 +25,13 @@ import cascading.tap.partition.Partition import cascading.tap.SinkMode import cascading.tap.Tap import cascading.tuple.Fields +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.mapred.{ JobConf, OutputCollector, RecordReader } /** * This is a base class for partition-based output sources */ -abstract class PartitionSource(val openWritesThreshold: Option[Int] = None) extends SchemedSource with HfsTapProvider { +abstract class PartitionSource(val openWritesThreshold: Option[Int] = None) extends SchemedSource with HfsTapProvider with LocalTapProvider { // The root path of the partitioned output. def basePath: String @@ -40,32 +43,31 @@ abstract class PartitionSource(val openWritesThreshold: Option[Int] = None) exte * * @param readOrWrite Describes if this source is being read from or written to. * @param mode The mode of the job. (implicit) - * - * @returns A cascading PartitionTap. + * @return A cascading PartitionTap. */ override def createTap(readOrWrite: AccessMode)(implicit mode: Mode): Tap[_, _, _] = { - readOrWrite match { - case Read => throw new InvalidSourceException("Using PartitionSource for input not yet implemented") - case Write => { - mode match { - case Local(_) => { - val localTap = new FileTap(localScheme, basePath, sinkMode) - openWritesThreshold match { - case Some(threshold) => new LPartitionTap(localTap, partition, threshold) - case None => new LPartitionTap(localTap, partition) - } - } - case hdfsMode @ Hdfs(_, _) => { - val hfsTap = createHfsTap(hdfsScheme, basePath, sinkMode) - getHPartitionTap(hfsTap) - } - case hdfsTest @ HadoopTest(_, _) => { - val hfsTap = createHfsTap(hdfsScheme, hdfsTest.getWritePathFor(this), sinkMode) - getHPartitionTap(hfsTap) - } - case _ => TestTapFactory(this, hdfsScheme).createTap(readOrWrite) - } - } + mode.storageMode.createTap(this, readOrWrite, mode, sinkMode) + } + + override def createHdfsReadTap(strictSources: Boolean, conf: Configuration, mode: Mode, sinkMode: SinkMode): Tap[_, _, _] = + throw new InvalidSourceException("Using PartitionSource for input not yet implemented", new NotImplementedError) + override def createLocalReadTap(sinkMode: SinkMode): Tap[_, _, _] = + throw new InvalidSourceException("Using PartitionSource for input not yet implemented", new NotImplementedError) + + override def createHdfsWriteTap(path: String, sinkMode: SinkMode): Tap[_, _, _] = { + val hfsTap = createHfsTap(hdfsScheme, path, sinkMode) + openWritesThreshold match { + case Some(threshold) => new HPartitionTap(hfsTap, partition, threshold) + case None => new HPartitionTap(hfsTap, partition) + } + } + + override def localWritePath: String = basePath + override def createLocalWriteTap(path: String, sinkMode: SinkMode): Tap[_, _, _] = { + val localTap = super.createLocalFileTap(localScheme, path, sinkMode) + openWritesThreshold match { + case Some(threshold) => new LPartitionTap(localTap, partition, threshold) + case None => new LPartitionTap(localTap, partition) } } @@ -79,32 +81,27 @@ abstract class PartitionSource(val openWritesThreshold: Option[Int] = None) exte throw new InvalidSourceException("basePath cannot be null for PartitionTap") } } - - private[this] def getHPartitionTap(hfsTap: Hfs): HPartitionTap = { - openWritesThreshold match { - case Some(threshold) => new HPartitionTap(hfsTap, partition, threshold) - case None => new HPartitionTap(hfsTap, partition) - } - } } /** * An implementation of TSV output, split over a partition tap. - * - * tsvFields lets users explicitly specify which fields they want to see in - * the TSV (allows user to discard path fields). - * - * apply assumes user wants a DelimitedPartition (the only - * strategy bundled with Cascading). - * - * @param basePath The root path for the output. - * @param delimiter The path delimiter, defaults to / to create sub-directory bins. - * @param pathFields The set of fields to apply to the path. - * @param writeHeader Flag to indicate that the header should be written to the file. - * @param tsvFields The set of fields to include in the TSV output. - * @param sinkMode How to handle conflicts with existing output. */ object PartitionedTsv { + /** + * + * tsvFields lets users explicitly specify which fields they want to see in + * the TSV (allows user to discard path fields). + * + * apply assumes user wants a DelimitedPartition (the only + * strategy bundled with Cascading). + * + * @param basePath The root path for the output. + * @param delimiter The path delimiter, defaults to / to create sub-directory bins. + * @param pathFields The set of fields to apply to the path. + * @param writeHeader Flag to indicate that the header should be written to the file. + * @param tsvFields The set of fields to include in the TSV output. + * @param sinkMode How to handle conflicts with existing output. + */ def apply( basePath: String, delimiter: String = "/", @@ -135,17 +132,20 @@ case class PartitionedTsv( /** * An implementation of SequenceFile output, split over a partition tap. - * - * apply assumes user wants a DelimitedPartition (the only - * strategy bundled with Cascading). - * - * @param basePath The root path for the output. - * @param delimiter The path delimiter, defaults to / to create sub-directory bins. - * @param pathFields The set of fields to apply to the path. - * @param sequenceFields The set of fields to use for the sequence file. - * @param sinkMode How to handle conflicts with existing output. */ object PartitionedSequenceFile { + /** + * An implementation of SequenceFile output, split over a partition tap. + * + * apply assumes user wants a DelimitedPartition (the only + * strategy bundled with Cascading). + * + * @param basePath The root path for the output. + * @param delimiter The path delimiter, defaults to / to create sub-directory bins. + * @param pathFields The set of fields to apply to the path. + * @param sequenceFields The set of fields to use for the sequence file. + * @param sinkMode How to handle conflicts with existing output. + */ def apply( basePath: String, delimiter: String = "/", 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 ce49a8697c..3a44ba1c6e 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Source.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Source.scala @@ -37,11 +37,6 @@ import org.apache.hadoop.mapred.RecordReader import scala.collection.JavaConverters._ -/** - * thrown when validateTaps fails - */ -class InvalidSourceException(message: String) extends RuntimeException(message) - /** * InvalidSourceTap used in createTap method when we want to defer * the failures to validateTaps method. @@ -108,20 +103,33 @@ case object Write extends AccessMode // declaration leaves some type parameters underspecified. Fill in the type // parameters with wildcards so the Scala compiler doesn't complain. +object LocalSchemeInstance { + type SchemeType = Scheme[Properties, InputStream, OutputStream, _, _] + + def apply(scheme: Scheme[_, _, _, _, _]) = + scheme.asInstanceOf[SchemeType] +} + object HadoopSchemeInstance { + type SchemeType = Scheme[JobConf, RecordReader[_, _], OutputCollector[_, _], _, _] def apply(scheme: Scheme[_, _, _, _, _]) = - scheme.asInstanceOf[Scheme[JobConf, RecordReader[_, _], OutputCollector[_, _], _, _]] + scheme.asInstanceOf[SchemeType] } object Hadoop2SchemeInstance { + type SchemeType = Scheme[Configuration, RecordReader[_, _], OutputCollector[_, _], _, _] + def apply(scheme: Scheme[_, _, _, _, _]) = - scheme.asInstanceOf[Scheme[Configuration, RecordReader[_, _], OutputCollector[_, _], _, _]] + scheme.asInstanceOf[SchemeType] } object CastHfsTap { // The scala compiler has problems with the generics in Cascading def apply(tap: Hfs): Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]] = tap.asInstanceOf[Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]]] + + def apply(tap: Tap[_, _, _]): Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]] = + tap.asInstanceOf[Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]]] } /** @@ -288,17 +296,13 @@ class NullTap[Config, Input, Output, SourceContext, SinkContext] } trait BaseNullSource extends Source { - override def createTap(readOrWrite: AccessMode)(implicit mode: Mode): Tap[_, _, _] = { + override def createTap(readOrWrite: AccessMode)(implicit mode: Mode): Tap[_, _, _] = readOrWrite match { - case Read => throw new Exception("not supported, reading from null") - case Write => mode match { - case Hdfs(_, _) => new NullTap[JobConf, RecordReader[_, _], OutputCollector[_, _], Any, Any] - case Local(_) => new NullTap[Properties, InputStream, OutputStream, Any, Any] - case Test(_) => new NullTap[Properties, InputStream, OutputStream, Any, Any] - } + case Read => throw new UnsupportedOperationException("not supported, reading from null") + case Write => mode.storageMode.createNullTap } - } } + /** * A source outputs nothing. It is used to drive execution of a task for side effect only. */ diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Stats.scala b/scalding-core/src/main/scala/com/twitter/scalding/Stats.scala index 6893e9533c..daccb1401e 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Stats.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Stats.scala @@ -1,14 +1,18 @@ package com.twitter.scalding -import cascading.flow.{ Flow, FlowListener, FlowDef, FlowProcess } -import cascading.flow.hadoop.HadoopFlowProcess +import java.lang.reflect.Constructor + +import cascading.flow.{ Flow, FlowDef, FlowListener, FlowProcess } import cascading.stats.CascadingStats import java.util.concurrent.ConcurrentHashMap + import org.slf4j.{ Logger, LoggerFactory } + import scala.collection.JavaConverters._ import scala.collection.mutable import scala.ref.WeakReference import scala.util.Try +import scala.reflect.runtime.universe._ /* * This can be a bit tricky to use, but it is important that incBy and inc @@ -48,24 +52,76 @@ object StatKey { } private[scalding] object CounterImpl { - def apply(fp: FlowProcess[_], statKey: StatKey): CounterImpl = - fp match { - case hFP: HadoopFlowProcess => HadoopFlowPCounterImpl(hFP, statKey) - case _ => GenericFlowPCounterImpl(fp, statKey) + val CounterImplClass = "scalding.stats.counter.impl.class" + + /* + Note: knowing which counter implementation class to use is a fabric-dependent question. Prior to Cascading 3, the + question of "which fabric" was easy: it was either Local or Hadoop, and both types were available wherever Scalding + is loaded. + + We cannot have access to the fabric's Mode / ExecutionMode implementation, because instantiating counters happens + at clusterside runtime, not planning time[*]. The FlowProcess[_] detailed type typically also depends on the fabric, + with exceptions: Legacy Hadoop (1.x) and Hadoop2-MR1 use the same FlowProcess implementation class name, + HadoopFlowProcess. + Anyway, we can't use pattern-matching on the specific FlowProcess[_] type, as this would require scalding-core to + know about specific fabric implementation details, which it can no longer do (post-Cascading 3). + + Solution found for now: + 1. the fabric-specific ExecutionMode is responsible for setting the class name of the appropriate counter + implementation in a config key (named CounterImplClass above) + 2. clusterside, CounterImpl$ retrieves this class name and uses reflection to instanciate the correct counter + implementation. + + + [*] even if Mode was accessible at runtime, it'd require a pervasive addition of (possibly implicit) parameters + whenever a join between pipes is possible, which would cause significant changes to the API + + */ + def apply(fp: FlowProcess[_], statKey: StatKey): CounterImpl = { + /* TODO: comment this stuff. */ + val klassName = Option(fp.getStringProperty(CounterImplClass)).getOrElse(sys.error(CounterImplClass + " property is missing")) + + val memo = scala.collection.mutable.Map[String, Constructor[CounterImpl]]() + val ctor = memo.synchronized { + memo.getOrElse(klassName, + { + val klass = Class.forName(klassName).asInstanceOf[Class[CounterImpl]] + val ctor = klass.getConstructor(classOf[FlowProcess[_]], classOf[StatKey]) + memo.put(klassName, ctor) + ctor + }) } + ctor.newInstance(fp, statKey) + } + + private[scalding] def upcast[T <: FlowProcess[_]](fp: FlowProcess[_])(implicit ev: TypeTag[T]): T = fp match { + case hfp: T @unchecked if (ev == typeTag[T]) => hfp // see below + case _ => throw new IllegalArgumentException(s"Provided flow process instance ${fp} should have been of type ${ev}") + + /* note: we jump through the additioanl hoop of passing implicit ev:TypeTag[T] and verifying the equality as + due to the JVM's type erasure, we can't tell the difference between a T and a FlowProcess[_] at runtime. */ + } + } -sealed private[scalding] trait CounterImpl { +private[scalding] trait CounterImpl { def increment(amount: Long): Unit } private[scalding] case class GenericFlowPCounterImpl(fp: FlowProcess[_], statKey: StatKey) extends CounterImpl { - override def increment(amount: Long): Unit = fp.increment(statKey.group, statKey.counter, amount) -} + /* Note: most other CounterImpl implementations will need to provide an alternate constructor matching the above + signature. Suggested definition: -private[scalding] case class HadoopFlowPCounterImpl(fp: HadoopFlowProcess, statKey: StatKey) extends CounterImpl { - private[this] val cntr = fp.getReporter().getCounter(statKey.group, statKey.counter) - override def increment(amount: Long): Unit = cntr.increment(amount) + private[scalding] case class MyFlowPCounterImpl(fp: MyFlowProcess, statKey: StatKey) extends CounterImpl { + def this(fp: FlowProcess[_], statKey: StatKey) { // this alternate ctor is the one that will actually be used at runtime + this(CounterImpl.upcast[MyFlowProcess](fp), statKey) + } + override def increment(amount: Long): Unit = ??? + } + + */ + + override def increment(amount: Long): Unit = fp.increment(statKey.group, statKey.counter, amount) } object Stat { diff --git a/scalding-core/src/main/scala/com/twitter/scalding/TestTapFactory.scala b/scalding-core/src/main/scala/com/twitter/scalding/TestTapFactory.scala index 92d7698745..beef7be208 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/TestTapFactory.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/TestTapFactory.scala @@ -51,59 +51,62 @@ object TestTapFactory extends Serializable { new TestTapFactory(src, sinkMode) { override def hdfsScheme = Some(scheme) } } -class TestTapFactory(src: Source, sinkMode: SinkMode) extends Serializable with HfsTapProvider { +class TestTapFactory(src: Source, sinkMode: SinkMode) extends Serializable /* with HfsTapProvider */ { def sourceFields: Fields = - hdfsScheme.map { _.getSourceFields }.getOrElse(sys.error("No sourceFields defined")) + hdfsScheme.map { + _.getSourceFields + }.getOrElse(sys.error("No sourceFields defined")) def sinkFields: Fields = - hdfsScheme.map { _.getSinkFields }.getOrElse(sys.error("No sinkFields defined")) + hdfsScheme.map { + _.getSinkFields + }.getOrElse(sys.error("No sinkFields defined")) def hdfsScheme: Option[Scheme[JobConf, RecordReader[_, _], OutputCollector[_, _], _, _]] = None - def createTap(readOrWrite: AccessMode)(implicit mode: Mode): Tap[_, _, _] = { - mode match { - case Test(buffers) => { - /* - * There MUST have already been a registered sink or source in the Test mode. - * to access this. You must explicitly name each of your test sources in your - * JobTest. - */ - require( - buffers(src).isDefined, - TestTapFactory.sourceNotFoundError.format(src)) - val buffer = + def createLocalTap(readOrWrite: AccessMode, mode: TestMode): Tap[_, _, _] = { + /* + * There MUST have already been a registered sink or source in the Test mode. + * to access this. You must explicitly name each of your test sources in your + * JobTest. + */ + val buffer = mode.buffers(src).fold( + throw new IllegalArgumentException(s"Failed to create tap for: ${src}, with error: requirement failed: " + TestTapFactory.sourceNotFoundError.format(src))) { + case sourceBuffer => if (readOrWrite == Write) { - val buf = buffers(src).get + val buf = sourceBuffer //Make sure we wipe it out: buf.clear() buf } else { // if the source is also used as a sink, we don't want its contents to get modified - buffers(src).get.clone() + sourceBuffer.clone() } - new MemoryTap[InputStream, OutputStream]( - new NullScheme(sourceFields, sinkFields), - buffer) } - case hdfsTest @ HadoopTest(conf, buffers) => - readOrWrite match { - case Read => { - val bufOpt = buffers(src) - if (bufOpt.isDefined) { - val buffer = bufOpt.get - val fields = sourceFields - (new MemorySourceTap(buffer.toList.asJava, fields)).asInstanceOf[Tap[JobConf, _, _]] - } else { - CastHfsTap(createHfsTap(hdfsScheme.get, hdfsTest.getWritePathFor(src), sinkMode)) - } - } - case Write => { - val path = hdfsTest.getWritePathFor(src) - CastHfsTap(createHfsTap(hdfsScheme.get, path, sinkMode)) - } + + new MemoryTap[InputStream, OutputStream]( + new NullScheme(sourceFields, sinkFields), + buffer) + } + + def createHdfsTap(readOrWrite: AccessMode, mode: TestMode, pathAllocator: Source => String, + tapProvider: HfsTapProvider): Tap[_, _, _] = { + readOrWrite match { + case Read => { + hdfsScheme.fold(throw new IllegalStateException("no available HDFS Scheme")) { scheme => + + val bufOpt = mode.buffers(src) + bufOpt.map(buffer => { + val buffer = bufOpt.get + val fields = sourceFields + (new MemorySourceTap(buffer.toList.asJava, fields)).asInstanceOf[Tap[JobConf, _, _]] + }) + .getOrElse(tapProvider.createHdfsWriteTap(pathAllocator(src), sinkMode)) } - case _ => { - throw new RuntimeException("TestTapFactory doesn't support mode: " + mode.toString) + } + + case Write => { + tapProvider.createHdfsWriteTap(pathAllocator(src), sinkMode) } } } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/TimePathedSource.scala b/scalding-core/src/main/scala/com/twitter/scalding/TimePathedSource.scala index 749f7fb4bd..44931569f1 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/TimePathedSource.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/TimePathedSource.scala @@ -146,11 +146,15 @@ abstract class MostRecentGoodSource(p: String, dr: DateRange, t: TimeZone) override def toString = "MostRecentGoodSource(" + p + ", " + dr + ", " + t + ")" - override protected def goodHdfsPaths(hdfsMode: Hdfs) = getPathStatuses(hdfsMode.jobConf) - .toList - .reverse - .find(_._2) - .map(_._1) + /* + * Get all the set of valid paths based on source strictness. + */ + override protected def goodHdfsPaths(strictSources: Boolean, conf: Configuration): Iterable[String] = + getPathStatuses(conf) + .toList + .reverse + .find(_._2) + .map(_._1) override def hdfsReadPathsAreGood(conf: Configuration) = getPathStatuses(conf) .exists(_._2) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/filecache/DistributedCacheFile.scala b/scalding-core/src/main/scala/com/twitter/scalding/filecache/DistributedCacheFile.scala index 1aae44fdd8..e060ab2b8f 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/filecache/DistributedCacheFile.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/filecache/DistributedCacheFile.scala @@ -87,14 +87,9 @@ final case class UncachedFile private[scalding] (source: Either[String, URI]) { import DistributedCacheFile._ def add()(implicit mode: Mode): CachedFile = - mode match { - case Hdfs(_, conf) => addHdfs(conf) - case HadoopTest(conf, _) => addHdfs(conf) - case (Local(_) | Test(_)) => addLocal() - case _ => throw new RuntimeException("unhandled mode: %s".format(mode)) - } - - private[this] def addLocal(): CachedFile = { + mode.storageMode.addCachedFile(this) + + private[scalding] def addLocal(): CachedFile = { val path = source match { case Left(strPath) => strPath @@ -104,7 +99,7 @@ final case class UncachedFile private[scalding] (source: Either[String, URI]) { LocallyCachedFile(path) } - private[this] def addHdfs(conf: Configuration): CachedFile = { + private[scalding] def addHdfs(conf: Configuration): CachedFile = { HDistributedCache.createSymlink(conf) def makeQualifiedStr(path: String, conf: Configuration): URI = diff --git a/scalding-core/src/main/scala/com/twitter/scalding/macros/impl/TupleSetterImpl.scala b/scalding-core/src/main/scala/com/twitter/scalding/macros/impl/TupleSetterImpl.scala index 6065a3833a..437d70a4c9 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/macros/impl/TupleSetterImpl.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/macros/impl/TupleSetterImpl.scala @@ -39,8 +39,11 @@ object TupleSetterImpl { val tupTerm = newTermName(c.fresh("tup")) val (finalIdx, set) = CaseClassBasedSetterImpl(c)(tupTerm, allowUnknownTypes, TupleFieldSetter) + // We generate with @SuppressWarnings(...OptionPartial) as IF we have option types, we'll get + // a { if (foo.isDefined) do(foo.get) else other } structure (justified as this is perf-sensitive code) val res = q""" new _root_.com.twitter.scalding.TupleSetter[$T] with _root_.com.twitter.bijection.macros.MacroGenerated { + @_root_.java.lang.SuppressWarnings(_root_.scala.Array("org.brianmckenna.wartremover.warts.OptionPartial")) override def apply(t: $T): _root_.cascading.tuple.Tuple = { val $tupTerm = _root_.cascading.tuple.Tuple.size($finalIdx) $set diff --git a/scalding-core/src/main/scala/com/twitter/scalding/source/CodecSource.scala b/scalding-core/src/main/scala/com/twitter/scalding/source/CodecSource.scala index 394d7b7d10..1bfd130bf7 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/source/CodecSource.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/source/CodecSource.scala @@ -46,7 +46,7 @@ object CodecSource { def apply[T](paths: String*)(implicit codec: Injection[T, Array[Byte]]) = new CodecSource[T](paths) } -class CodecSource[T] private (val hdfsPaths: Seq[String], val maxFailures: Int = 0)(implicit @transient injection: Injection[T, Array[Byte]]) +class CodecSource[T] private (override val hdfsPaths: Seq[String], val maxFailures: Int = 0)(implicit @transient injection: Injection[T, Array[Byte]]) extends FileSource with Mappable[T] with LocalTapSource { @@ -56,7 +56,7 @@ class CodecSource[T] private (val hdfsPaths: Seq[String], val maxFailures: Int = lazy val field = new Fields(fieldSym.name) val injectionBox = Externalizer(injection andThen BytesWritableCodec.get) - def localPaths = hdfsPaths + override def localPaths = hdfsPaths override def converter[U >: T] = TupleConverter.asSuperConverter[T, U](TupleConverter.singleConverter[T]) override def hdfsScheme = diff --git a/scalding-core/src/main/scala/com/twitter/scalding/source/TypedText.scala b/scalding-core/src/main/scala/com/twitter/scalding/source/TypedText.scala index 6d3ceb05b1..b286c61c5c 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/source/TypedText.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/source/TypedText.scala @@ -80,14 +80,15 @@ trait TypedTextDelimited[T] extends SchemedSource with Mappable[T] with TypedSin override def sinkFields = typeDescriptor.fields override def sourceFields = typeDescriptor.fields - override def localScheme = - new CLTextDelimited(typeDescriptor.fields, false, false, separator.str, strict, null /* quote */ , - typeDescriptor.fields.getTypesClasses, safe) - - override def hdfsScheme = - HadoopSchemeInstance(new CHTextDelimited(typeDescriptor.fields, null /* compression */ , false, false, - separator.str, strict, null /* quote */ , - typeDescriptor.fields.getTypesClasses, safe).asInstanceOf[Scheme[_, _, _, _, _]]) + override def localScheme: LocalSchemeInstance.SchemeType = + LocalSchemeInstance(new CLTextDelimited(typeDescriptor.fields, false, false, separator.str, strict, null /* quote */ , + typeDescriptor.fields.getTypesClasses, safe)) + + override def hdfsScheme: HadoopSchemeInstance.SchemeType = + HadoopSchemeInstance( + new CHTextDelimited(typeDescriptor.fields, null /* compression */ , false, false, + separator.str, strict, null /* quote */ , + typeDescriptor.fields.getTypesClasses, safe).asInstanceOf[Scheme[_, _, _, _, _]]) } class TimePathTypedText[T](sep: TypedSep, path: String)(implicit dr: DateRange, td: TypeDescriptor[T]) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/HashJoinable.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/HashJoinable.scala index 4a9e3e2f32..74df24ba38 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/HashJoinable.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/HashJoinable.scala @@ -71,7 +71,7 @@ trait HashJoinable[K, +V] extends CoGroupable[K, V] with KeyedPipe[K] { // if the user has turned off auto force right, we fall back to the old behavior and //just return the mapped pipe - if (!getHashJoinAutoForceRight(mode) || isSafeToSkipForceToDisk(mappedPipe)) mappedPipe + if (!mode.executionMode.getHashJoinAutoForceRight || isSafeToSkipForceToDisk(mappedPipe)) mappedPipe else mappedPipe.forceToDisk } @@ -119,12 +119,4 @@ trait HashJoinable[K, +V] extends CoGroupable[K, V] with KeyedPipe[K] { case _ => false } - private def getHashJoinAutoForceRight(mode: Mode): Boolean = { - mode match { - case h: HadoopMode => - val config = Config.fromHadoop(h.jobConf) - config.getHashJoinAutoForceRight - case _ => false //default to false - } - } } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/MemorySink.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/MemorySink.scala index af19568b43..0cef0f2dac 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/MemorySink.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/MemorySink.scala @@ -32,20 +32,17 @@ import cascading.tuple.Tuple */ class MemorySink[T] extends TypedSink[T] { private[this] val buf = Buffer[Tuple]() - private[this] val name: String = UUID.randomUUID.toString + private[this] val name: String = s"MemorySink{${UUID.randomUUID}}" // takes a copy as of NOW. Don't call this before the job has run def readResults: Iterable[T] = buf.iterator.map(_.getObject(0).asInstanceOf[T]).toList def setter[U <: T] = TupleSetter.asSubSetter(TupleSetter.singleSetter[T]) - def writeFrom(pipe: Pipe)(implicit flowDef: FlowDef, mode: Mode): Pipe = - mode match { - case cl: CascadingLocal => - val tap = new MemoryTap(new NullScheme(sinkFields, sinkFields), buf) - flowDef.addSink(name, tap) - flowDef.addTail(new Pipe(name, pipe)) - pipe - case _ => sys.error("MemorySink only usable with cascading local") - } + def writeFrom(pipe: Pipe)(implicit flowDef: FlowDef, mode: Mode): Pipe = { + val tap = mode.storageMode.createMemoryTap(Write, sinkFields, buf) // will reject service non non-local modes + flowDef.addSink(name, tap) + flowDef.addTail(new Pipe(name, pipe)) + pipe + } } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/PartitionSchemed.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/PartitionSchemed.scala index 1f57ccbc9e..398569ec73 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/PartitionSchemed.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/PartitionSchemed.scala @@ -15,11 +15,41 @@ package com.twitter.scalding package typed -import cascading.tap.hadoop.PartitionTap +import cascading.tap.hadoop.{ PartitionTap => HdfsPartitionTap } import cascading.tap.local.{ FileTap, PartitionTap => LocalPartitionTap } import cascading.tap.{ SinkMode, Tap } import cascading.tuple.Fields +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.mapred.JobConf +trait PartitionSchemedOps extends SchemedSource with HfsTapProvider with LocalTapProvider { + def partitionFields: Fields + def template: String + + protected def makePartition = new TemplatePartition(partitionFields, template) + + override def createHdfsReadTap(strictSources: Boolean, conf: Configuration, mode: Mode, sinkMode: SinkMode): Tap[_, _, _] = + { + val hfsTap = createHfsTap(hdfsScheme, hdfsWritePath, SinkMode.REPLACE) + new HdfsPartitionTap(hfsTap, makePartition, SinkMode.UPDATE) + } + + override def createLocalReadTap(sinkMode: SinkMode): Tap[_, _, _] = + { + val localTap = super.createLocalFileTap(localScheme, localWritePath, SinkMode.REPLACE) + new LocalPartitionTap(localTap, makePartition, SinkMode.UPDATE) + } + + override def createHdfsWriteTap(path: String, sinkMode: SinkMode): Tap[_, _, _] = { + val hfsTap = createHfsTap(hdfsScheme, path, SinkMode.REPLACE) + new HdfsPartitionTap(hfsTap, makePartition, SinkMode.UPDATE) + } + + override def createLocalWriteTap(path: String, sinkMode: SinkMode): Tap[_, _, _] = { + val localTap = super.createLocalFileTap(localScheme, path, SinkMode.REPLACE) + new LocalPartitionTap(localTap, makePartition, SinkMode.UPDATE) + } +} /** * Trait to assist with creating partitioned sources. * @@ -27,7 +57,7 @@ import cascading.tuple.Fields * Note that for both of them the sink fields need to be set to only include the actual fields * that should be written to file and not the partition fields. */ -trait PartitionSchemed[P, T] extends SchemedSource with TypedSink[(P, T)] with Mappable[(P, T)] with HfsTapProvider { +trait PartitionSchemed[P, T] extends PartitionSchemedOps with TypedSink[(P, T)] with Mappable[(P, T)] { def path: String def template: String def valueSetter: TupleSetter[T] @@ -57,25 +87,4 @@ trait PartitionSchemed[P, T] extends SchemedSource with TypedSink[(P, T)] with M /** Flatten a pair of `P` and `T` into a cascading tuple.*/ override def setter[U <: (P, T)] = PartitionUtil.setter[P, T, U](valueSetter, partitionSetter) - - /** Creates the taps for local and hdfs mode.*/ - override def createTap(readOrWrite: AccessMode)(implicit mode: Mode): Tap[_, _, _] = - mode match { - case Local(_) => { - val fileTap = new FileTap(localScheme, path, SinkMode.REPLACE) - new LocalPartitionTap(fileTap, new TemplatePartition(partitionFields, template), SinkMode.UPDATE) - .asInstanceOf[Tap[_, _, _]] - } - case Hdfs(_, _) => { - val hfs = createHfsTap(hdfsScheme, path, SinkMode.REPLACE) - new PartitionTap(hfs, new TemplatePartition(partitionFields, template), SinkMode.UPDATE) - .asInstanceOf[Tap[_, _, _]] - } - case hdfsTest @ HadoopTest(_, _) => { - val hfs = createHfsTap(hdfsScheme, hdfsTest.getWritePathFor(this), SinkMode.REPLACE) - new PartitionTap(hfs, new TemplatePartition(partitionFields, template), SinkMode.UPDATE) - .asInstanceOf[Tap[_, _, _]] - } - case _ => TestTapFactory(this, hdfsScheme).createTap(readOrWrite) - } } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/PartitionedTextLine.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/PartitionedTextLine.scala index 648d08ae29..f5063161db 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/PartitionedTextLine.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/PartitionedTextLine.scala @@ -54,11 +54,11 @@ import cascading.tuple.{ Fields, Tuple, TupleEntry } */ case class PartitionedTextLine[P]( path: String, template: String, encoding: String = TextLine.DEFAULT_CHARSET)(implicit val valueSetter: TupleSetter[String], val valueConverter: TupleConverter[(Long, String)], - val partitionSetter: TupleSetter[P], val partitionConverter: TupleConverter[P]) extends SchemedSource with TypedSink[(P, String)] with Mappable[(P, (Long, String))] with HfsTapProvider - with java.io.Serializable { + val partitionSetter: TupleSetter[P], val partitionConverter: TupleConverter[P]) + extends PartitionSchemedOps with TypedSink[(P, String)] with Mappable[(P, (Long, String))] with java.io.Serializable { // The partition fields, offset by the value arity. - val partitionFields = + override val partitionFields = PartitionUtil.toFields(valueSetter.arity, valueSetter.arity + partitionSetter.arity) // Create the underlying scheme and explicitly set the sink fields to be only the specified fields @@ -89,27 +89,6 @@ case class PartitionedTextLine[P]( override def sinkFields: Fields = PartitionUtil.toFields(0, valueSetter.arity + partitionSetter.arity) - /** Creates the taps for local and hdfs mode.*/ - override def createTap(readOrWrite: AccessMode)(implicit mode: Mode): Tap[_, _, _] = - mode match { - case Local(_) => { - val fileTap = new FileTap(localScheme, path, SinkMode.REPLACE) - new LocalPartitionTap(fileTap, new TemplatePartition(partitionFields, template), SinkMode.UPDATE) - .asInstanceOf[Tap[_, _, _]] - } - case Hdfs(_, _) => { - val hfs = createHfsTap(hdfsScheme, path, SinkMode.REPLACE) - new PartitionTap(hfs, new TemplatePartition(partitionFields, template), SinkMode.UPDATE) - .asInstanceOf[Tap[_, _, _]] - } - case hdfsTest @ HadoopTest(_, _) => { - val hfs = createHfsTap(hdfsScheme, hdfsTest.getWritePathFor(this), SinkMode.REPLACE) - new PartitionTap(hfs, new TemplatePartition(partitionFields, template), SinkMode.UPDATE) - .asInstanceOf[Tap[_, _, _]] - } - case _ => TestTapFactory(this, hdfsScheme).createTap(readOrWrite) - } - /** * Combine both the partition and value converter to extract the data from a flat cascading tuple * into a pair of `P` and `(offset, line)`. 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 561afa7c1d..d9195a16b2 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 @@ -15,14 +15,15 @@ limitations under the License. */ package com.twitter.scalding.typed -import java.io.{ OutputStream, InputStream, Serializable } -import java.util.Random +import java.io.{ InputStream, OutputStream, Serializable } +import java.util.{ Random, UUID } import cascading.flow.FlowDef -import cascading.pipe.{ Checkpoint, Each, Pipe, Merge } +import cascading.pipe.{ Checkpoint, Each, Merge, Pipe } import cascading.tap.Tap import cascading.tuple.{ Fields, TupleEntry } import com.twitter.algebird.{ Aggregator, Batched, Monoid, Semigroup } +import com.twitter.scalding.StorageMode.TemporarySource import com.twitter.scalding.TupleConverter.{ TupleEntryConverter, singleConverter, tuple2Converter } import com.twitter.scalding.TupleSetter.{ singleSetter, tup2Setter } import com.twitter.scalding._ @@ -500,39 +501,39 @@ trait TypedPipe[+T] extends Serializable { * and then opens it after complete so that you can continue from that point */ def forceToDiskExecution: Execution[TypedPipe[T]] = { - val cachedRandomUUID = java.util.UUID.randomUUID - lazy val inMemoryDest = new MemorySink[T] + val tempSourceUuid = UUID.randomUUID - def hadoopTypedSource(conf: Config): TypedSource[T] with TypedSink[T] = { - // come up with unique temporary filename, use the config here - // TODO: refactor into TemporarySequenceFile class - val tmpDir = conf.get("hadoop.tmp.dir") - .orElse(conf.get("cascading.tmp.dir")) - .getOrElse("/tmp") + val thisPipe = this + class ForceToDiskExecution { + var temporarySource: Option[TemporarySource[T]] = None - val tmpSeq = tmpDir + "/scalding/snapshot-" + cachedRandomUUID + ".seq" - source.TypedSequenceFile[T](tmpSeq) - - } - val writeFn = { (conf: Config, mode: Mode) => - mode match { - case _: CascadingLocal => // Local or Test mode - (this, inMemoryDest) - case _: HadoopMode => - (this, hadoopTypedSource(conf)) + def writeFn(conf: Config, mode: Mode) = { + val ts = mode.storageMode.temporaryTypedSource[T] + temporarySource = Some(ts) + (thisPipe, ts.sink(conf)) } - } - val readFn = { (conf: Config, mode: Mode) => - mode match { - case _: CascadingLocal => // Local or Test mode - TypedPipe.from(inMemoryDest.readResults) - case _: HadoopMode => - TypedPipe.from(hadoopTypedSource(conf)) + def readFn(conf: Config, mode: Mode) = { + temporarySource.fold( + throw new IllegalStateException("readFn() called before writeFn()"))(ts => ts.downstreamPipe((conf))) } + + def toExecution = Execution.write(writeFn _, readFn _) + + /* Major assumption: when readFn gets called, the (conf, mode) arguments are the same than + when writeFn got called. + + This is critical to getting the same instance of TemporarySource[T] (of which we really cannot know the + concrete type, which is the Mode's private business) from writeFn to readFn. + + The TypedPipeDiffTest test is a good canary for breakage here. + + TODO: open question. Are there any threading issues wrt to the chronology of calling writeFn then readFn? Should + "temporarySource" be protected? + */ } - Execution.write(writeFn, readFn) + (new ForceToDiskExecution).toExecution } /** @@ -1117,8 +1118,10 @@ case class WithDescriptionTypedPipe[T](typedPipe: TypedPipe[T], description: Str val pipe = typedPipe.toPipe[U](fieldNames)(flowDef, mode, setter) RichPipe.setPipeDescriptions(pipe, List(description)) } + override def cross[U](tiny: TypedPipe[U]): TypedPipe[(T, U)] = WithDescriptionTypedPipe(typedPipe.cross(tiny), description) + override def flatMap[U](f: T => TraversableOnce[U]): TypedPipe[U] = WithDescriptionTypedPipe(typedPipe.flatMap(f), description) } @@ -1130,8 +1133,11 @@ case class WithDescriptionTypedPipe[T](typedPipe: TypedPipe[T], description: Str */ 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)) } 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 c829c7673e..e59930bbe0 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/CoreTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/CoreTest.scala @@ -24,64 +24,6 @@ import com.twitter.scalding.source.DailySuffixTsv import java.lang.{ Integer => JInt } -class NumberJoinerJob(args: Args) extends Job(args) { - val in0 = TypedTsv[(Int, Int)]("input0").read.rename((0, 1) -> ('x0, 'y0)) - val in1 = Tsv("input1").read.mapTo((0, 1) -> ('x1, 'y1)) { input: (Long, Long) => input } - in0.joinWithSmaller('x0 -> 'x1, in1) - .write(Tsv("output")) -} - -class NumberJoinTest extends WordSpec with Matchers { - import Dsl._ - "A NumberJoinerJob" should { - //Set up the job: - "not throw when joining longs with ints" in { - 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 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 - .finish() - } - } -} - -class SpillingJob(args: Args) extends Job(args) { - TypedTsv[(Int, Int)]("input").read.rename((0, 1) -> ('n, 'v)) - .groupBy('n) { group => - group.spillThreshold(3).sum[Int]('v).size - }.write(Tsv("output")) -} - -class SpillingTest extends WordSpec with Matchers { - import Dsl._ - "A SpillingJob" should { - val src = (0 to 9).map(_ -> 1) ++ List(0 -> 4) - val result = src.groupBy(_._1) - .mapValues { v => (v.map(_._2).sum, v.size) } - .map { case (a, (b, c)) => (a, b, c) } - .toSet - - //Set up the job: - "work when number of keys exceeds spill threshold" in { - JobTest(new SpillingJob(_)) - .source(TypedTsv[(Int, Int)]("input"), src) - .sink[(Int, Int, Int)](Tsv("output")) { outBuf => - outBuf.toSet shouldBe result - }.run - .runHadoop - .finish() - } - } -} - object GroupRandomlyJob { val NumShards = 10 } @@ -336,46 +278,6 @@ class CollidingKeyJoinTest extends WordSpec with Matchers { } } -class TinyJoinJob(args: Args) extends Job(args) { - val p1 = Tsv(args("input1")) - .read - .mapTo((0, 1) -> ('k1, 'v1)) { v: (String, Int) => v } - val p2 = Tsv(args("input2")) - .read - .mapTo((0, 1) -> ('k2, 'v2)) { v: (String, Int) => v } - p1.joinWithTiny('k1 -> 'k2, p2) - .project('k1, 'v1, 'v2) - .write(Tsv(args("output"))) -} - -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)) - var idx = 0 - JobTest(new TinyJoinJob(_)) - .arg("input1", "fakeInput1") - .arg("input2", "fakeInput2") - .arg("output", "fakeOutput") - .source(Tsv("fakeInput1"), input1) - .source(Tsv("fakeInput2"), input2) - .sink[(String, Int, Int)](Tsv("fakeOutput")) { outBuf => - val actualOutput = outBuf.map { - case (k: String, v1: Int, v2: Int) => - (k, (v1, v2)) - }.toMap - (idx + ": join tuples with the same key") in { - actualOutput shouldBe correctOutput - } - idx += 1 - } - .run - .runHadoop - .finish() - } -} - class TinyCollisionJoinJob(args: Args) extends Job(args) { val p1 = Tsv(args("input1")) .read @@ -413,131 +315,6 @@ class TinyCollisionJoinTest extends WordSpec with Matchers { } } -class TinyThenSmallJoin(args: Args) extends Job(args) { - val pipe0 = Tsv("in0", ('x0, 'y0)).read - val pipe1 = Tsv("in1", ('x1, 'y1)).read - val pipe2 = Tsv("in2", ('x2, 'y2)).read - - pipe0.joinWithTiny('x0 -> 'x1, pipe1) - .joinWithSmaller('x0 -> 'x2, pipe2) - .map(('y0, 'y1, 'y2) -> ('y0, 'y1, 'y2)) { v: (TC, TC, TC) => - (v._1.n, v._2.n, v._3.n) - } - .project('x0, 'y0, 'x1, 'y1, 'x2, 'y2) - .write(Tsv("out")) -} - -case class TC(val n: Int) - -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)) - 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 => - (idx + ": join tuples with the same key") in { - outBuf.toList.sorted shouldBe correct - } - idx += 1 - } - .run - .runHadoop - .finish() - } -} - -class LeftJoinJob(args: Args) extends Job(args) { - val p1 = Tsv(args("input1")) - .mapTo((0, 1) -> ('k1, 'v1)) { v: (String, Int) => v } - val p2 = Tsv(args("input2")) - .mapTo((0, 1) -> ('k2, 'v2)) { v: (String, Int) => v } - p1.leftJoinWithSmaller('k1 -> 'k2, p2) - .project('k1, 'v1, 'v2) - // Null sent to TSV will not be read in properly - .map('v2 -> 'v2) { v: AnyRef => Option(v).map { _.toString }.getOrElse("NULL") } - .write(Tsv(args("output"))) -} - -class LeftJoinTest extends 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")) - var idx = 0 - JobTest(new LeftJoinJob(_)) - .arg("input1", "fakeInput1") - .arg("input2", "fakeInput2") - .arg("output", "fakeOutput") - .source(Tsv("fakeInput1"), input1) - .source(Tsv("fakeInput2"), input2) - .sink[(String, Int, JInt)](Tsv("fakeOutput")) { outBuf => - val actualOutput = outBuf.map { input: (String, Int, AnyRef) => - println(input) - val (k, v1, v2) = input - (k, (v1, v2)) - }.toMap - (idx + ": join tuples with the same key") in { - correctOutput shouldBe actualOutput - } - idx += 1 - } - .run - .runHadoop - .finish() - } -} - -class LeftJoinWithLargerJob(args: Args) extends Job(args) { - val p1 = Tsv(args("input1")) - .mapTo((0, 1) -> ('k1, 'v1)) { v: (String, Int) => v } - val p2 = Tsv(args("input2")) - .mapTo((0, 1) -> ('k2, 'v2)) { v: (String, Int) => v } - // Note i am specifying the joiner explicitly since this did not work properly before (leftJoinWithLarger always worked) - p1.joinWithLarger('k1 -> 'k2, p2, new cascading.pipe.joiner.LeftJoin) - .project('k1, 'v1, 'v2) - // Null sent to TSV will not be read in properly - .map('v2 -> 'v2) { v: AnyRef => Option(v).map { _.toString }.getOrElse("NULL") } - .write(Tsv(args("output"))) -} - -class 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")) - var idx = 0 - JobTest(new LeftJoinWithLargerJob(_)) - .arg("input1", "fakeInput1") - .arg("input2", "fakeInput2") - .arg("output", "fakeOutput") - .source(Tsv("fakeInput1"), input1) - .source(Tsv("fakeInput2"), input2) - .sink[(String, Int, JInt)](Tsv("fakeOutput")) { outBuf => - val actualOutput = outBuf.map { input: (String, Int, AnyRef) => - println(input) - val (k, v1, v2) = input - (k, (v1, v2)) - }.toMap - s"$idx: join tuples with the same key" in { - correctOutput shouldBe actualOutput - } - idx += 1 - } - .run - .runHadoop - .finish() - } -} - class MergeTestJob(args: Args) extends Job(args) { val in = TextLine(args("in")).read.mapTo(1 -> ('x, 'y)) { line: String => val p = line.split(" ").map { _.toDouble } @@ -770,35 +547,6 @@ class HistogramTest extends WordSpec with Matchers { } } -class ForceReducersJob(args: Args) extends Job(args) { - TextLine("in").read - .rename((0, 1) -> ('num, 'line)) - .flatMap('line -> 'words){ l: String => l.split(" ") } - .groupBy('num){ _.toList[String]('words -> 'wordList).forceToReducers } - .map('wordList -> 'wordList){ w: List[String] => w.mkString(" ") } - .project('num, 'wordList) - .write(Tsv("out")) -} - -class ForceReducersTest extends WordSpec with Matchers { - "A ForceReducersJob" should { - var idx = 0 - JobTest(new ForceReducersJob(_)) - .source(TextLine("in"), List("0" -> "single test", "1" -> "single result")) - .sink[(Int, String)](Tsv("out")) { outBuf => - (idx + ": must get the result right") in { - //need to convert to sets because order - outBuf(0)._2.split(" ").toSet shouldBe Set("single", "test") - outBuf(1)._2.split(" ").toSet shouldBe Set("single", "result") - } - idx += 1 - } - .run - .runHadoop - .finish() - } -} - class ToListJob(args: Args) extends Job(args) { TextLine(args("in")).read .flatMap('line -> 'words){ l: String => l.split(" ") } @@ -854,102 +602,6 @@ class ToListTest extends WordSpec with Matchers { } } -class CrossJob(args: Args) extends Job(args) { - val p1 = Tsv(args("in1")).read - .mapTo((0, 1) -> ('x, 'y)) { tup: (Int, Int) => tup } - val p2 = Tsv(args("in2")).read - .mapTo(0 -> 'z) { (z: Int) => z } - p1.crossWithTiny(p2).write(Tsv(args("out"))) -} - -class CrossTest extends WordSpec with Matchers { - "A CrossJob" should { - 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 => - (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 - .finish() - } -} - -class GroupAllCrossJob(args: Args) extends Job(args) { - val p1 = Tsv(args("in1")).read - .mapTo((0, 1) -> ('x, 'y)) { tup: (Int, Int) => tup } - .groupAll { _.max('x) } - .map('x -> 'x) { x: Int => List(x) } - - val p2 = Tsv(args("in2")).read - .mapTo(0 -> 'z) { (z: Int) => z } - p2.crossWithTiny(p1) - .map('x -> 'x) { l: List[Int] => l.size } - .project('x, 'z) - .write(Tsv(args("out"))) -} - -class GroupAllCrossTest extends WordSpec with Matchers { - "A GroupAllCrossJob" should { - var idx = 0 - JobTest(new GroupAllCrossJob(_)) - .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)](Tsv("fakeOut")) { outBuf => - (idx + ": must look exactly right") in { - outBuf should have size 2 - outBuf.toSet shouldBe Set((1, 4), (1, 5)) - } - idx += 1 - } - .run - .runHadoop - .finish() - } -} - -class SmallCrossJob(args: Args) extends Job(args) { - val p1 = Tsv(args("in1")).read - .mapTo((0, 1) -> ('x, 'y)) { tup: (Int, Int) => tup } - val p2 = Tsv(args("in2")).read - .mapTo(0 -> 'z) { (z: Int) => z } - p1.crossWithSmaller(p2).write(Tsv(args("out"))) -} - -class SmallCrossTest extends WordSpec with Matchers { - "A SmallCrossJob" should { - 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 => - (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 - .finish() - } -} - class TopKJob(args: Args) extends Job(args) { Tsv(args("in")).read .mapTo(0 -> 'x) { (tup: Int) => tup } @@ -975,37 +627,6 @@ class TopKTest extends WordSpec with Matchers { } } -class ScanJob(args: Args) extends Job(args) { - Tsv("in", ('x, 'y, 'z)) - .groupBy('x) { - _.sortBy('y) - .scanLeft('y -> 'ys)(0) { (oldV: Int, newV: Int) => oldV + newV } - } - .project('x, 'ys, 'z) - .map('z -> 'z) { z: Int => z } //Make sure the null z is converted to an int - .write(Tsv("out")) -} - -class ScanTest extends WordSpec with Matchers { - import Dsl._ - - "A ScanJob" should { - 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)) - (idx + ": have a working scanLeft") in { - outBuf.toList shouldBe correct - } - idx += 1 - } - .run - .runHadoop - .finish() - } -} - class TakeJob(args: Args) extends Job(args) { val input = Tsv("in").read .mapTo((0, 1, 2) -> ('x, 'y, 'z)) { tup: (Int, Int, Int) => tup } @@ -1105,51 +726,6 @@ class PivotTest extends WordSpec with Matchers with FieldConversions { } } -class IterableSourceJob(args: Args) extends Job(args) { - val list = List((1, 2, 3), (4, 5, 6), (3, 8, 9)) - val iter = IterableSource(list, ('x, 'y, 'z)) - Tsv("in", ('x, 'w)) - .joinWithSmaller('x -> 'x, iter) - .write(Tsv("out")) - - Tsv("in", ('x, 'w)) - .joinWithTiny('x -> 'x, iter) - .write(Tsv("tiny")) - //Now without fields and using the implicit: - Tsv("in", ('x, 'w)) - .joinWithTiny('x -> 0, list).write(Tsv("imp")) -} - -class IterableSourceTest extends WordSpec with Matchers with FieldConversions { - val input = List((1, 10), (2, 20), (3, 30)) - "A IterableSourceJob" should { - var idx = 0 - JobTest(new IterableSourceJob(_)) - .source(Tsv("in", ('x, 'w)), input) - .sink[(Int, Int, Int, Int)](Tsv("out")) { outBuf => - 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 => - 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 => - 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 - .finish() - } -} - class HeadLastJob(args: Args) extends Job(args) { Tsv("input", ('x, 'y)).groupBy('x) { _.sortBy('y) @@ -1271,34 +847,6 @@ class FoldJobTest extends WordSpec with Matchers { } } -// TODO make a Product serializer that clean $outer parameters -case class V(v: Int) -class InnerCaseJob(args: Args) extends Job(args) { - val res = TypedTsv[Int]("input") - .mapTo(('xx, 'vx)) { x => (x * x, V(x)) } - .groupBy('xx) { _.head('vx) } - .map('vx -> 'x) { v: V => v.v } - .project('x, 'xx) - .write(Tsv("output")) -} - -class InnerCaseTest extends WordSpec with Matchers { - import Dsl._ - - val input = List(Tuple1(1), Tuple1(2), Tuple1(2), Tuple1(4)) - "An InnerCaseJob" should { - JobTest(new InnerCaseJob(_)) - .source(TypedTsv[Int]("input"), input) - .sink[(Int, Int)](Tsv("output")) { outBuf => - "Correctly handle inner case classes" in { - outBuf.toSet shouldBe Set((1, 1), (2, 4), (4, 16)) - } - } - .runHadoop - .finish() - } -} - class NormalizeJob(args: Args) extends Job(args) { Tsv("in") .read @@ -1323,40 +871,6 @@ class NormalizeTest extends WordSpec with Matchers { } } -class ForceToDiskJob(args: Args) extends Job(args) { - val x = Tsv("in", ('x, 'y)) - .read - .filter('x) { x: Int => x > 0 } - .rename('x -> 'x1) - Tsv("in", ('x, 'y)) - .read - .joinWithTiny('y -> 'y, x.forceToDisk) - .project('x, 'x1, 'y) - .write(Tsv("out")) -} - -class ForceToDiskTest extends WordSpec with Matchers { - import Dsl._ - - "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 => - (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 shouldBe correct - } - idx += 1 - } - .run - .runHadoop - .finish() - } -} - class ThrowsErrorsJob(args: Args) extends Job(args) { Tsv("input", ('letter, 'x)) .read @@ -1488,122 +1002,6 @@ class TypedItsATrapTest extends WordSpec with Matchers { } } -class GroupAllToListTestJob(args: Args) extends Job(args) { - TypedTsv[(Long, String, Double)]("input") - .mapTo('a, 'b) { case (id, k, v) => (id, Map(k -> v)) } - .groupBy('a) { _.sum[Map[String, Double]]('b) } - .groupAll { - _.toList[(Long, Map[String, Double])](('a, 'b) -> 'abList) - } - .map('abList -> 'abMap) { - list: List[(Long, Map[String, Double])] => list.toMap - } - .project('abMap) - .map('abMap -> 'abMap) { x: AnyRef => x.toString } - .write(Tsv("output")) -} - -class GroupAllToListTest extends WordSpec with Matchers { - import Dsl._ - - "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)) - JobTest(new GroupAllToListTestJob(_)) - .source(TypedTsv[(Long, String, Double)]("input"), input) - .sink[String](Tsv("output")) { outBuf => - "must properly aggregate stuff into a single map" in { - outBuf should have size 1 - outBuf(0) shouldBe output.toString - } - } - .runHadoop - .finish() - } -} - -class ToListGroupAllToListTestJob(args: Args) extends Job(args) { - TypedTsv[(Long, String)]("input") - .mapTo('b, 'c) { case (k, v) => (k, v) } - .groupBy('c) { _.toList[Long]('b -> 'bList) } - .groupAll { - _.toList[(String, List[Long])](('c, 'bList) -> 'cbList) - } - .project('cbList) - .write(Tsv("output")) -} - -class ToListGroupAllToListSpec extends WordSpec with Matchers { - import Dsl._ - - val expected = List(("us", List(1)), ("jp", List(3, 2)), ("gb", List(3, 1))) - - "A ToListGroupAllToListTestJob" should { - JobTest(new ToListGroupAllToListTestJob(_)) - .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 should have size 1 - outBuf.head shouldBe (expected.toString) - println(outBuf.head) - } - } - .runHadoop - .finish() - - JobTest(new ToListGroupAllToListTestJob(_)) - .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 should have size 1 - outBuf.head shouldBe expected - println(outBuf.head) - } - } - .run - .finish() - } -} - -// TODO: HangingTest is very flaky now because we enabled multi-thread testing. Need to be fixed later. -/* -class HangingJob(args : Args) extends Job(args) { - val x = Tsv("in", ('x,'y)) - .read - .filter('x, 'y) { t: (Int, Int) => - val (x, y) = t - timeout(Millisecs(2)) { - if (y % 2 == 1) Thread.sleep(1000) - x > 0 - } getOrElse false - } - .write(Tsv("out")) -} - -class HangingTest extends Specification { - import Dsl._ - noDetailedDiffs() - - "A HangingJob" should { - val input = (1 to 100).flatMap { i => List((-1, i), (1, i)) }.toList - JobTest(new HangingJob(_)) - .source(Tsv("in",('x,'y)), input) - .sink[(Int,Int)](Tsv("out")) { outBuf => - "run correctly when task times out" in { - //outBuf.size must_== 100 - //val correct = (1 to 100).map { i => (1, i) } - outBuf.size must_== 50 - val correct = (1 to 50).map { i => (1, i*2) } - outBuf.toList.sorted must_== correct - } - } - .run - .runHadoop - .finish() - } -} -*/ - class Function2Job(args: Args) extends Job(args) { import FunctionImplicits._ Tsv("in", ('x, 'y)).mapTo(('x, 'y) -> 'xy) { (x: String, y: String) => x + y }.write(Tsv("output")) 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 a4449410f7..5c3ec4790f 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/FileSourceTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/FileSourceTest.scala @@ -270,35 +270,6 @@ class FileSourceTest extends WordSpec with Matchers { } } - "invalid source input" should { - "Throw in validateTaps in strict mode" in { - val e = intercept[InvalidSourceException] { - TestInvalidFileSource.validateTaps(Hdfs(strict = true, new Configuration())) - } - assert(e.getMessage.endsWith("Data is missing from one or more paths in: List(invalid_hdfs_path)")) - } - - "Throw in validateTaps in non-strict mode" in { - val e = intercept[InvalidSourceException] { - TestInvalidFileSource.validateTaps(Hdfs(strict = false, new Configuration())) - } - assert(e.getMessage.endsWith("No good paths in: List(invalid_hdfs_path)")) - } - - "Throw in toIterator because no data is present in strict mode" in { - val e = intercept[InvalidSourceException] { - TestInvalidFileSource.toIterator(Config.default, Hdfs(strict = true, new Configuration())) - } - assert(e.getMessage.endsWith("Data is missing from one or more paths in: List(invalid_hdfs_path)")) - } - - "Throw in toIterator because no data is present in non-strict mode" in { - val e = intercept[InvalidSourceException] { - TestInvalidFileSource.toIterator(Config.default, Hdfs(strict = false, new Configuration())) - } - assert(e.getMessage.endsWith("No good paths in: List(invalid_hdfs_path)")) - } - } } object TestPath { 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 1179bb956b..d000ae3261 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/JobTestTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/JobTestTest.scala @@ -34,6 +34,7 @@ class JobTestTest extends WordSpec with Matchers { 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/TestTapFactoryTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/TestTapFactoryTest.scala index b071edabfc..ea108c17d5 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/TestTapFactoryTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/TestTapFactoryTest.scala @@ -1,7 +1,8 @@ package com.twitter.scalding -import cascading.tap.Tap +import cascading.tap.{ SinkMode, Tap } import cascading.tuple.{ Fields, Tuple } + import scala.collection.mutable.Buffer import org.scalatest.{ Matchers, WordSpec } @@ -18,11 +19,11 @@ class TestTapFactoryTest extends WordSpec with Matchers { val testTapFactory = TestTapFactory(testSource, new Fields()) def createIllegalTap(): Tap[Any, Any, Any] = - testTapFactory.createTap(Read)(testMode).asInstanceOf[Tap[Any, Any, Any]] + testMode.storageMode.createTap(testSource, Read, testMode, SinkMode.UPDATE).asInstanceOf[Tap[Any, Any, Any]] the[IllegalArgumentException] thrownBy { createIllegalTap() - } should have message ("requirement failed: " + TestTapFactory.sourceNotFoundError.format(testSource)) + } should have message ("Failed to create tap for: com.twitter.scalding.Tsv(path), with error: requirement failed: " + TestTapFactory.sourceNotFoundError.format(testSource)) } } } 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 4bca622e3e..f1440a6940 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 @@ -25,7 +25,7 @@ import scala.collection.mutable // 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 + //case class UnknownMode(buffers: Map[Source, mutable.Buffer[Tuple]]) extends TestMode with CascadingLocal /* val conf = smartMock[Configuration] diff --git a/scalding-fabric-flink/src/main/scala/com/twitter/scalding/FlinkMode.scala b/scalding-fabric-flink/src/main/scala/com/twitter/scalding/FlinkMode.scala new file mode 100644 index 0000000000..5f905e692d --- /dev/null +++ b/scalding-fabric-flink/src/main/scala/com/twitter/scalding/FlinkMode.scala @@ -0,0 +1,81 @@ +package com.twitter.scalding + +import java.util + +import cascading.flow.{ FlowConnector, FlowProcess, FlowRuntimeProps } +import cascading.tuple.Tuple +import com.dataartisans.flink.cascading.FlinkConnector +import com.dataartisans.flink.cascading.runtime.util.FlinkFlowProcess +import org.apache.flink.api.common.{ ExecutionMode => FlinkApiExecutionMode } +import org.apache.flink.api.java.ExecutionEnvironment +import org.apache.flink.configuration.ConfigConstants +import org.apache.hadoop.conf.Configuration + +import scala.collection.mutable.Buffer + +class FlinkExecutionMode(override val mode: Mode, @transient override val jobConf: Configuration) extends HadoopExecutionModeBase[Configuration] { + + override protected def newFlowConnector(rawConf: util.Map[AnyRef, AnyRef]): FlowConnector = { + // val env = ExecutionEnvironment.createRemoteEnvironment(host, port, jarfile, ...) + val env = ExecutionEnvironment.createLocalEnvironment(4) + new FlinkConnector(env, rawConf) + } + + protected def defaultConfiguration: Configuration = new Configuration(true) // initialize the default config + + protected def newFlowProcess(conf: Configuration): FlowProcess[Configuration] = { + val jobManHost = Option(conf.get(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY)) + val jobManPort = Option(conf.get(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY)).map(_.toInt) + + val resourceManPort = Option(conf.get(ConfigConstants.RESOURCE_MANAGER_IPC_PORT_KEY)).map(_.toInt) + + val env = ExecutionEnvironment.getExecutionEnvironment + env.getConfig.setExecutionMode(FlinkApiExecutionMode.BATCH) + + new FlinkFlowProcess(conf) + } + + override private[scalding] def setupCounterCreation(conf: Config): Config = { + val nconf = conf - CounterImpl.CounterImplClass + + (CounterImpl.CounterImplClass -> classOf[FlinkFlowPCounterImpl].getCanonicalName) + + nconf + } +} + +private[scalding] case class FlinkFlowPCounterImpl(fp: FlinkFlowProcess, statKey: StatKey) extends CounterImpl { + def this(fp: FlowProcess[_], statKey: StatKey) { // this alternate ctor is the one that will actually be used at runtime + this(CounterImpl.upcast[FlinkFlowProcess](fp), statKey) + } + + override def increment(amount: Long): Unit = fp.increment(statKey.group, statKey.counter, amount) + // to access the value: fp.getCounterValue(statKey.group, statKey.counter) +} + +case class FlinkMode(strictSources: Boolean, @transient jobConf: Configuration) extends HadoopFamilyMode { + val name = "flink" + + override val storageMode: StorageMode = new HdfsStorageMode(strictSources, jobConf) + override val executionMode: ExecutionMode = new FlinkExecutionMode(this, jobConf) +} + +case class FlinkTestMode(@transient jobConf: Configuration, + @transient override val buffers: Source => Option[Buffer[Tuple]]) + extends HadoopFamilyTestMode { + + val strictSources = false + val name = "flink-test" + + override val storageMode: TestStorageMode = new HdfsTestStorageMode(false, jobConf, this.getWritePathFor) + override val executionMode: ExecutionMode = new FlinkExecutionMode(this, jobConf) { + + override protected def newFlowConnector(rawConf: util.Map[AnyRef, AnyRef]): FlowConnector = { + // val env = ExecutionEnvironment.createRemoteEnvironment(host, port, jarfile, ...) + val env = ExecutionEnvironment.createLocalEnvironment(4) + new FlinkConnector(env, rawConf) + } + + override private[scalding] def setupCounterCreation(conf: Config): Config = + super.setupCounterCreation(conf) + } +} diff --git a/scalding-fabric-flink/src/test/scala/com/twitter/scalding/ModeParseFlinkSpec.scala b/scalding-fabric-flink/src/test/scala/com/twitter/scalding/ModeParseFlinkSpec.scala new file mode 100644 index 0000000000..f6d7a637d3 --- /dev/null +++ b/scalding-fabric-flink/src/test/scala/com/twitter/scalding/ModeParseFlinkSpec.scala @@ -0,0 +1,37 @@ +package com.twitter.scalding + +import org.scalatest.{ Matchers, WordSpec } + +class ModeParseFlinkSpec extends WordSpec with Matchers { + "Parsing for Hadoop" should { + val parser = new ExecutionApp { + def job = Execution.from(()) + } + "parse args correctly (local)" in { + val (conf, mode) = parser.config(Array("--local")) + mode shouldBe a[LocalMode] + + mode.name === "local" + } + + "reject legacy args (hdfs)" in { + the[Exception] thrownBy { + val (conf, mode) = parser.config(Array("--hdfs")) + } shouldBe a[ArgsException] + } + + "parse args correctly (flink)" in { + val (conf, mode) = parser.config(Array("--flink")) + mode shouldBe a[FlinkMode] + + mode.name === "flink" + } + + "parse args correctly (autoCluster)" in { + val (conf, mode) = parser.config(Array("--autoCluster")) + mode shouldBe a[FlinkMode] + + mode.name === "flink" + } + } +} \ No newline at end of file diff --git a/scalding-fabric-hadoop/src/main/scala/com/twitter/scalding/LegacyHadoopMode.scala b/scalding-fabric-hadoop/src/main/scala/com/twitter/scalding/LegacyHadoopMode.scala new file mode 100644 index 0000000000..e612880d45 --- /dev/null +++ b/scalding-fabric-hadoop/src/main/scala/com/twitter/scalding/LegacyHadoopMode.scala @@ -0,0 +1,57 @@ +package com.twitter.scalding + +import java.util + +import cascading.flow._ +import cascading.flow.hadoop.{ HadoopFlowConnector, HadoopFlowProcess } +import cascading.tuple.Tuple +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.mapred.JobConf + +import scala.collection.mutable.Buffer + +class LegacyHadoopExecutionMode(override val mode: Mode, + @transient override val jobConf: Configuration) + extends HadoopExecutionModeBase[JobConf] { + + override protected def newFlowConnector(rawConf: util.Map[AnyRef, AnyRef]): FlowConnector = new HadoopFlowConnector(rawConf) + protected def defaultConfiguration: JobConf = new JobConf(true) // initialize the default config + + override protected def newFlowProcess(conf: JobConf): HadoopFlowProcess = new HadoopFlowProcess(conf) + + override private[scalding] def setupCounterCreation(conf: Config): Config = + conf + (CounterImpl.CounterImplClass -> classOf[HadoopFlowPCounterImpl].getCanonicalName) +} +private[scalding] case class HadoopFlowPCounterImpl(fp: HadoopFlowProcess, statKey: StatKey) extends CounterImpl { + def this(fp: FlowProcess[_], statKey: StatKey) { // this alternate ctor is the one that will actually be used at runtime + this(CounterImpl.upcast[HadoopFlowProcess](fp), statKey) + } + + private[this] val cntr = fp.getReporter().getCounter(statKey.group, statKey.counter) + override def increment(amount: Long): Unit = cntr.increment(amount) +} + +class LegacyHadoopModeCommon(override val strictSources: Boolean, @transient override val jobConf: Configuration) extends HadoopFamilyMode { + val name = "hadoop" + + override val storageMode: StorageMode = new HdfsStorageMode(strictSources, jobConf) + override val executionMode: ExecutionMode = new LegacyHadoopExecutionMode(this, jobConf) +} + +case class LegacyHadoopMode(override val strictSources: Boolean, @transient override val jobConf: Configuration) + extends LegacyHadoopModeCommon(strictSources, jobConf) {} + +@deprecated("please use LegacyHadoopMode instead, or let Mode.apply decide", "0.17.0") +case class Hdfs(override val strictSources: Boolean, @transient override val jobConf: Configuration) + extends LegacyHadoopModeCommon(strictSources, jobConf) {} + +case class HadoopTest(@transient jobConf: Configuration, + @transient override val buffers: Source => Option[Buffer[Tuple]]) + extends HadoopFamilyTestMode { + + val strictSources = false + val name = "hadoop-test" + + override val storageMode: TestStorageMode = new HdfsTestStorageMode(false, jobConf, this.getWritePathFor) + override val executionMode: ExecutionMode = new LegacyHadoopExecutionMode(this, jobConf) +} diff --git a/scalding-fabric-hadoop/src/test/scala/com/twitter/scalding/ExecutionHadoopExtraTest.scala b/scalding-fabric-hadoop/src/test/scala/com/twitter/scalding/ExecutionHadoopExtraTest.scala new file mode 100644 index 0000000000..525c3e62cb --- /dev/null +++ b/scalding-fabric-hadoop/src/test/scala/com/twitter/scalding/ExecutionHadoopExtraTest.scala @@ -0,0 +1,54 @@ +package com.twitter.scalding + +import com.twitter.algebird.monad.Reader + +import com.twitter.scalding.serialization.macros.impl.ordered_serialization.runtime_helpers.MacroEqualityOrderedSerialization +import com.twitter.scalding.serialization.OrderedSerialization + +import java.nio.file.FileSystems +import java.nio.file.Path + +import org.scalatest.{ Matchers, WordSpec } + +import scala.concurrent.ExecutionContext.Implicits.global +import scala.concurrent.{ Await, Future, ExecutionContext => ConcurrentExecutionContext, Promise } +import scala.util.Random +import scala.util.{ Try, Success, Failure } + +import ExecutionContext._ + +class ExecutionHadoopExtraTest extends ExecutionTest /* WordSpec with Matchers */ { + "ExecutionApp (continued)" should { + val parser = new ExecutionApp { + def job = Execution.from(()) + } + + "parse args correctly (legacy hdfs)" in { + val (conf1, mode) = parser.config(Array("--test", "-Dmapred.reduce.tasks=110", "--hdfs")) + + mode shouldBe a[LegacyHadoopMode] + val hconf = mode.asInstanceOf[LegacyHadoopMode].jobConf + + conf1.get("mapred.reduce.tasks") should contain("110") + conf1.getArgs.boolean("test") shouldBe true + hconf.get("mapred.reduce.tasks") shouldBe "110" + } + "parse args correctly (hadoop1) " in { + val (conf1, mode) = parser.config(Array("--test", "-Dmapred.reduce.tasks=110", "--hadoop1")) + + mode shouldBe a[LegacyHadoopMode] + val hconf = mode.asInstanceOf[LegacyHadoopMode].jobConf + + conf1.get("mapred.reduce.tasks") should contain("110") + conf1.getArgs.boolean("test") shouldBe true + hconf.get("mapred.reduce.tasks") shouldBe "110" + } + + "parse args correctly (autoCluster) " in { + val (conf1, mode) = parser.config(Array("--test", "--autoCluster")) + + mode shouldBe a[ClusterMode] // Can be anything that's available in the classpath. + } + + } +} diff --git a/scalding-fabric-hadoop/src/test/scala/com/twitter/scalding/ModeParseHadoop.scala b/scalding-fabric-hadoop/src/test/scala/com/twitter/scalding/ModeParseHadoop.scala new file mode 100644 index 0000000000..68a60c6980 --- /dev/null +++ b/scalding-fabric-hadoop/src/test/scala/com/twitter/scalding/ModeParseHadoop.scala @@ -0,0 +1,38 @@ +package com.twitter.scalding + +import org.scalatest.{ Matchers, WordSpec } + +class ModeParseHadoopSpec extends WordSpec with Matchers { + "Parsing for Hadoop" should { + val parser = new ExecutionApp { + def job = Execution.from(()) + } + "parse args correctly (local)" in { + val (conf, mode) = parser.config(Array("--local")) + mode shouldBe a[LocalMode] + + mode.name === "local" + } + + "parse args correctly (hdfs, LEGACY)" in { + val (conf, mode) = parser.config(Array("--hdfs")) + mode shouldBe a[LegacyHadoopMode] + + mode.name === "hadoop" + } + + "parse args correctly (hadoop1)" in { + val (conf, mode) = parser.config(Array("--hadoop1")) + mode shouldBe a[LegacyHadoopMode] + + mode.name === "hadoop" + } + + "parse args correctly (autoCluster)" in { + val (conf, mode) = parser.config(Array("--autoCluster")) + mode shouldBe a[LegacyHadoopMode] + + mode.name === "hadoop" + } + } +} \ No newline at end of file diff --git a/scalding-fabric-hadoop2-mr1/src/main/scala/com/twitter/scalding/Hadoop2Mr1Mode.scala b/scalding-fabric-hadoop2-mr1/src/main/scala/com/twitter/scalding/Hadoop2Mr1Mode.scala new file mode 100644 index 0000000000..2b58ef2132 --- /dev/null +++ b/scalding-fabric-hadoop2-mr1/src/main/scala/com/twitter/scalding/Hadoop2Mr1Mode.scala @@ -0,0 +1,51 @@ +package com.twitter.scalding + +import java.util + +import cascading.flow.{ FlowConnector, FlowProcess } +import cascading.flow.hadoop.HadoopFlowProcess +import cascading.flow.hadoop2.Hadoop2MR1FlowConnector +import cascading.tuple.Tuple +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.mapred.JobConf + +import scala.collection.mutable.Buffer + +class Hadoop2Mr1ExecutionMode(override val mode: Mode, @transient override val jobConf: Configuration) extends HadoopExecutionModeBase[JobConf] { + + override protected def newFlowConnector(rawConf: util.Map[AnyRef, AnyRef]): FlowConnector = new Hadoop2MR1FlowConnector(rawConf) + + protected def defaultConfiguration: JobConf = new JobConf(true) // initialize the default config + + override protected def newFlowProcess(conf: JobConf): FlowProcess[JobConf] = new HadoopFlowProcess(conf) + + override private[scalding] def setupCounterCreation(conf: Config): Config = + conf + (CounterImpl.CounterImplClass -> classOf[Hadoop2Mr1FlowPCounterImpl].getCanonicalName) +} + +private[scalding] case class Hadoop2Mr1FlowPCounterImpl(fp: HadoopFlowProcess, statKey: StatKey) extends CounterImpl { + def this(fp: FlowProcess[_], statKey: StatKey) { // this alternate ctor is the one that will actually be used at runtime + this(CounterImpl.upcast[HadoopFlowProcess](fp), statKey) + } + + private[this] val cntr = fp.getReporter().getCounter(statKey.group, statKey.counter) + override def increment(amount: Long): Unit = cntr.increment(amount) +} + +case class Hadoop2Mr1Mode(strictSources: Boolean, @transient jobConf: Configuration) extends HadoopFamilyMode { + val name = "hadoop2-mr1" + + override val storageMode: StorageMode = new HdfsStorageMode(strictSources, jobConf) + override val executionMode: ExecutionMode = new Hadoop2Mr1ExecutionMode(this, jobConf) +} + +case class Hadoop2Mr1TestMode(@transient jobConf: Configuration, + @transient override val buffers: Source => Option[Buffer[Tuple]]) + extends HadoopFamilyTestMode { + + val strictSources = false + val name = "hadoop2-mr1-test" + + override val storageMode: TestStorageMode = new HdfsTestStorageMode(false, jobConf, this.getWritePathFor) + override val executionMode: ExecutionMode = new Hadoop2Mr1ExecutionMode(this, jobConf) +} diff --git a/scalding-fabric-hadoop2-mr1/src/test/scala/com/twitter/scalding/ModeParseHadoop2Mr1Spec.scala b/scalding-fabric-hadoop2-mr1/src/test/scala/com/twitter/scalding/ModeParseHadoop2Mr1Spec.scala new file mode 100644 index 0000000000..802a2b11c9 --- /dev/null +++ b/scalding-fabric-hadoop2-mr1/src/test/scala/com/twitter/scalding/ModeParseHadoop2Mr1Spec.scala @@ -0,0 +1,37 @@ +package com.twitter.scalding + +import org.scalatest.{ Matchers, WordSpec } + +class ModeParseHadoop2Mr1Spec extends WordSpec with Matchers { + "Parsing for Hadoop" should { + val parser = new ExecutionApp { + def job = Execution.from(()) + } + "parse args correctly (local)" in { + val (conf, mode) = parser.config(Array("--local")) + mode shouldBe a[LocalMode] + + mode.name === "local" + } + + "reject legacy args (hdfs)" in { + the[Exception] thrownBy { + val (conf, mode) = parser.config(Array("--hdfs")) + } shouldBe a[ArgsException] + } + + "parse args correctly (hadoop2-mr1)" in { + val (conf, mode) = parser.config(Array("--hadoop2-mr1")) + mode shouldBe a[Hadoop2Mr1Mode] + + mode.name === "hadoop2-mr1" + } + + "parse args correctly (autoCluster)" in { + val (conf, mode) = parser.config(Array("--autoCluster")) + mode shouldBe a[Hadoop2Mr1Mode] + + mode.name === "hadoop2-mr1" + } + } +} \ No newline at end of file diff --git a/scalding-fabric-tez/src/main/scala/com/twitter/scalding/TezMode.scala b/scalding-fabric-tez/src/main/scala/com/twitter/scalding/TezMode.scala new file mode 100644 index 0000000000..c74fc03ee2 --- /dev/null +++ b/scalding-fabric-tez/src/main/scala/com/twitter/scalding/TezMode.scala @@ -0,0 +1,72 @@ +package com.twitter.scalding + +import java.util + +import cascading.flow.{ FlowConnector, FlowProcess, FlowRuntimeProps } +import cascading.flow.tez.{ Hadoop2TezFlowConnector, Hadoop2TezFlowProcess } +import cascading.tuple.Tuple +import org.apache.hadoop.conf.Configuration +import org.apache.tez.dag.api.TezConfiguration + +import scala.collection.mutable.Buffer + +class TezExecutionMode(override val mode: Mode, @transient override val jobConf: Configuration) extends HadoopExecutionModeBase[TezConfiguration] { + + override protected def newFlowConnector(rawConf: util.Map[AnyRef, AnyRef]): FlowConnector = new Hadoop2TezFlowConnector(rawConf) + + protected def defaultConfiguration: TezConfiguration = new TezConfiguration(true) // initialize the default config + + protected def newFlowProcess(conf: TezConfiguration): FlowProcess[TezConfiguration] = { + val ownrednum = Option(conf.get(FlowRuntimeProps.GATHER_PARTITIONS)) + + val confToUse = ownrednum match { + case Some(value) => conf // User already specified the Gather Partitions parameters in Tez terms; no override needed. + case None => { + val newConf = new TezConfiguration(conf) + newConf.set(FlowRuntimeProps.GATHER_PARTITIONS, Option(conf.get(Config.HadoopNumReducers)) + .getOrElse("4")) /* TEZ FIXME: ensure a better managed way of dealing with this */ + newConf + } + } + + new Hadoop2TezFlowProcess(confToUse) + } + + override private[scalding] def setupCounterCreation(conf: Config): Config = { + val nconf = conf - CounterImpl.CounterImplClass - FlowRuntimeProps.GATHER_PARTITIONS + + (CounterImpl.CounterImplClass -> classOf[TezFlowPCounterImpl].getCanonicalName) + + (FlowRuntimeProps.GATHER_PARTITIONS -> "4") /* TEZ FIXME: ensure a better managed way of dealing with this */ + + println(s"using ${FlowRuntimeProps.GATHER_PARTITIONS} -> ${nconf.get(FlowRuntimeProps.GATHER_PARTITIONS)}") + nconf + } + +} + +private[scalding] case class TezFlowPCounterImpl(fp: Hadoop2TezFlowProcess, statKey: StatKey) extends CounterImpl { + def this(fp: FlowProcess[_], statKey: StatKey) { // this alternate ctor is the one that will actually be used at runtime + this(CounterImpl.upcast[Hadoop2TezFlowProcess](fp), statKey) + } + + private[this] val cntr = fp.getReporter.getCounter(statKey.group, statKey.counter) + override def increment(amount: Long): Unit = cntr.increment(amount) +} + +case class TezMode(strictSources: Boolean, @transient jobConf: Configuration) extends HadoopFamilyMode { + val name = "hadoop2-tez" + + override val storageMode: StorageMode = new HdfsStorageMode(strictSources, jobConf) + override val executionMode: ExecutionMode = new TezExecutionMode(this, jobConf) +} + +/* TODO: TezTestMode based on HadoopTest */ +case class TezTestMode(@transient jobConf: Configuration, + @transient override val buffers: Source => Option[Buffer[Tuple]]) + extends HadoopFamilyTestMode { + + val strictSources = false + val name = "tez-test" + + override val storageMode: TestStorageMode = new HdfsTestStorageMode(false, jobConf, this.getWritePathFor) + override val executionMode: ExecutionMode = new TezExecutionMode(this, jobConf) +} diff --git a/scalding-fabric-tez/src/test/scala/com/twitter/scalding/ModeParseTezSpec.scala b/scalding-fabric-tez/src/test/scala/com/twitter/scalding/ModeParseTezSpec.scala new file mode 100644 index 0000000000..e7b9af08ff --- /dev/null +++ b/scalding-fabric-tez/src/test/scala/com/twitter/scalding/ModeParseTezSpec.scala @@ -0,0 +1,44 @@ +package com.twitter.scalding + +import org.scalatest.{ Matchers, WordSpec } + +class ModeParseTezSpec extends WordSpec with Matchers { + "Parsing for Hadoop" should { + val parser = new ExecutionApp { + def job = Execution.from(()) + } + "parse args correctly (local)" in { + val (conf, mode) = parser.config(Array("--local")) + mode shouldBe a[LocalMode] + + mode.name === "local" + } + + "reject legacy args (hdfs)" in { + the[Exception] thrownBy { + val (conf, mode) = parser.config(Array("--hdfs")) + } shouldBe a[ArgsException] + } + + "parse args correctly (tez)" in { + val (conf, mode) = parser.config(Array("--tez")) + mode shouldBe a[TezMode] + + mode.name === "tez" + } + + "parse args correctly (hadoop2-tez)" in { + val (conf, mode) = parser.config(Array("--hadoop2-tez")) + mode shouldBe a[TezMode] + + mode.name === "tez" + } + + "parse args correctly (autoCluster)" in { + val (conf, mode) = parser.config(Array("--autoCluster")) + mode shouldBe a[TezMode] + + mode.name === "tez" + } + } +} \ No newline at end of file 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 28ea9e075f..2740de8978 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 @@ -16,7 +16,6 @@ limitations under the License. package com.twitter.scalding.platform import com.twitter.scalding._ - import java.io.{ File, RandomAccessFile } import java.nio.channels.FileLock @@ -25,6 +24,7 @@ import org.apache.hadoop.mapreduce.filecache.DistributedCache import org.apache.hadoop.fs.{ FileUtil, Path } import org.apache.hadoop.hdfs.MiniDFSCluster import org.apache.hadoop.mapred.{ JobConf, MiniMRCluster } +import org.apache.hadoop.yarn.server.MiniYARNCluster import org.slf4j.LoggerFactory import org.slf4j.impl.Log4jLoggerAdapter @@ -43,7 +43,7 @@ class LocalCluster(mutex: Boolean = true) { private val LOG = LoggerFactory.getLogger(getClass) - private var hadoop: Option[(MiniDFSCluster, MiniMRCluster, JobConf)] = None + private var hadoop: Option[(MiniDFSCluster, MiniYARNCluster, JobConf)] = None private def getHadoop = hadoop.getOrElse(throw new Exception("Hadoop has not been initialized")) private def dfs = getHadoop._1 @@ -86,11 +86,17 @@ class LocalCluster(mutex: Boolean = true) { } new File(System.getProperty("hadoop.log.dir")).mkdirs() - val conf = new Configuration - val dfs = new MiniDFSCluster(conf, 4, true, null) + val clusterConfiguration = new Configuration() + + val dfs = new MiniDFSCluster.Builder(clusterConfiguration) + .numDataNodes(4) + .format(true) + .build() + val fileSystem = dfs.getFileSystem - val cluster = new MiniMRCluster(4, fileSystem.getUri.toString, 1, null, null, new JobConf(conf)) - val mrJobConf = cluster.createJobConf() + val cluster = new MiniYARNCluster(fileSystem.getUri.toString, 1, 4, 1, 1) + val mrJobConf = new JobConf(fileSystem.getConf) + mrJobConf.setInt("mapred.submit.replication", 2) mrJobConf.set("mapred.map.max.attempts", "2") mrJobConf.set("mapred.reduce.max.attempts", "2") @@ -170,7 +176,7 @@ class LocalCluster(mutex: Boolean = true) { private def getFileForClass[T](clazz: Class[T]): File = new File(clazz.getProtectionDomain.getCodeSource.getLocation.toURI) - def mode: Mode = Hdfs(true, jobConf) + def mode: Mode = Mode(Args(Seq("--autoCluster")), jobConf) // TODO: no longer require this version of Hadoop def putFile(file: File, location: String): Boolean = { val hdfsLocation = new Path(location) @@ -183,8 +189,8 @@ class LocalCluster(mutex: Boolean = true) { def shutdown(): Unit = { hadoop.foreach { case (dfs, mr, _) => + mr.stop() dfs.shutdown() - mr.shutdown() } hadoop = None if (mutex) { 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 d452f2c00e..022c16cc1e 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 @@ -530,8 +530,8 @@ class CheckForFlowProcessInTypedJob(args: Args) extends Job(args) { case class BypassValidationSource(path: String) extends FixedTypedText[Int](TypedText.TAB, path) { override def validateTaps(mode: Mode): Unit = () override def createTap(readOrWrite: AccessMode)(implicit mode: Mode): Tap[_, _, _] = - (mode, readOrWrite) match { - case (hdfsMode: Hdfs, Read) => new InvalidSourceTap(Seq(path)) + (mode.storageMode, readOrWrite) match { + case (_: HdfsStorageModeCommon, Read) => new InvalidSourceTap(Seq(path)) case _ => super.createTap(readOrWrite) } } @@ -585,7 +585,7 @@ class GroupByCoGroupCoGroupTriangleJob(args: Args) extends Job(args) { object PlatformTest { def setAutoForceRight(mode: Mode, autoForce: Boolean): Unit = { mode match { - case h: HadoopMode => + case h: HadoopFamilyMode => val config = h.jobConf config.setBoolean(Config.HashJoinAutoForceRight, autoForce) case _ => () diff --git a/scalding-hraven/src/main/scala/com/twitter/scalding/hraven/reducer_estimation/HRavenHistoryService.scala b/scalding-hraven/src/main/scala/com/twitter/scalding/hraven/reducer_estimation/HRavenHistoryService.scala index 083084d091..c89caf4f6a 100644 --- a/scalding-hraven/src/main/scala/com/twitter/scalding/hraven/reducer_estimation/HRavenHistoryService.scala +++ b/scalding-hraven/src/main/scala/com/twitter/scalding/hraven/reducer_estimation/HRavenHistoryService.scala @@ -125,7 +125,7 @@ object HRavenHistoryService extends HistoryService { def findMatchingJobStep(pastFlow: Flow) = pastFlow.getJobs.asScala.find { step => try { - step.getConfiguration.get("cascading.flow.step.num").toInt == stepNum + step.getConfiguration.get("cascading.flow.step.num").toInt == stepNum.toInt } catch { case _: NumberFormatException => false } diff --git a/scalding-jdbc/src/main/scala/com/twitter/scalding/jdbc/JDBCSource.scala b/scalding-jdbc/src/main/scala/com/twitter/scalding/jdbc/JDBCSource.scala index 7b48c77cce..392622f1d1 100644 --- a/scalding-jdbc/src/main/scala/com/twitter/scalding/jdbc/JDBCSource.scala +++ b/scalding-jdbc/src/main/scala/com/twitter/scalding/jdbc/JDBCSource.scala @@ -16,9 +16,9 @@ limitations under the License. package com.twitter.scalding.jdbc -import com.twitter.scalding.{ AccessMode, Hdfs, Mode, Source, TestTapFactory } +import com.twitter.scalding._ import cascading.jdbc.JDBCTap -import cascading.tap.Tap +import cascading.tap.{ SinkMode, Tap } import cascading.tuple.Fields /** @@ -45,7 +45,7 @@ import cascading.tuple.Fields * @author Oscar Boykin * @author Kevin Lin */ -abstract class JDBCSource extends Source with ColumnDefiner with JdbcDriver { +abstract class JDBCSource extends Source with ColumnDefiner with JdbcDriver with HfsTapProvider { // Override the following three members when you extend this class val tableName: TableName @@ -91,10 +91,14 @@ abstract class JDBCSource extends Source with ColumnDefiner with JdbcDriver { } override def createTap(readOrWrite: AccessMode)(implicit mode: Mode): Tap[_, _, _] = - mode match { - case Hdfs(_, _) => createJDBCTap.asInstanceOf[Tap[_, _, _]] - // TODO: support Local mode here, and better testing. - case _ => TestTapFactory(this, fields).createTap(readOrWrite) + (mode, mode.storageMode) match { + case (_, _: HdfsStorageModeCommon) => + createJDBCTap.asInstanceOf[Tap[_, _, _]] + + case (testMode: TestMode, _: LocalStorageModeCommon) => + TestTapFactory(this, fields).createLocalTap(readOrWrite: AccessMode, testMode) + + case _ => ??? // TODO: support Local mode here, and better testing. } // Generate SQL statement to create the DB table if not existing. 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 f50850d647..dc53e3aec6 100644 --- a/scalding-json/src/main/scala/com/twitter/scalding/JsonLine.scala +++ b/scalding-json/src/main/scala/com/twitter/scalding/JsonLine.scala @@ -45,6 +45,9 @@ case class JsonLine(p: String, fields: Fields = Fields.ALL, import Dsl._ import JsonLine._ + private val outFields = new Fields("json") + override def sinkFields = outFields + override def transformForWrite(pipe: Pipe) = pipe.mapTo(fields -> 'json) { t: TupleEntry => mapper.writeValueAsString(TupleConverter.ToMap(t)) } diff --git a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/thrift/PartitionedParquetThriftSourceTests.scala b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/thrift/PartitionedParquetThriftSourceTests.scala index e34da43d4e..7100dc10cc 100644 --- a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/thrift/PartitionedParquetThriftSourceTests.scala +++ b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/thrift/PartitionedParquetThriftSourceTests.scala @@ -44,17 +44,21 @@ class PartitionedParquetThriftSourceTests extends WordSpec with Matchers { .runHadoop .finish() - val testMode = job.mode.asInstanceOf[HadoopTest] + job.mode match { + case testMode: HadoopFamilyTestMode => + val directory = new File(testMode.getWritePathFor(partitionSource)) - val directory = new File(testMode.getWritePathFor(partitionSource)) + directory.listFiles().map({ + _.getName() + }).toSet shouldBe Set("94111", "10075") - directory.listFiles().map({ _.getName() }).toSet shouldBe Set("94111", "10075") - - // check that the partitioning is done correctly by zipcode - validate(new Path(directory.getPath + "/94111/part-00000-00000-m-00000.parquet"), - new Address("123 Embarcadero", "94111")) - validate(new Path(directory.getPath + "/10075/part-00000-00001-m-00000.parquet"), - new Address("123 E 79th St", "10075"), new Address("456 W 80th St", "10075")) + // check that the partitioning is done correctly by zipcode + validate(new Path(directory.getPath + "/94111/part-00000-00000-m-00000.parquet"), + new Address("123 Embarcadero", "94111")) + validate(new Path(directory.getPath + "/10075/part-00000-00001-m-00000.parquet"), + new Address("123 E 79th St", "10075"), new Address("456 W 80th St", "10075")) + case _ => ??? + } } } } \ No newline at end of file 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 fe297fb741..48a688d206 100644 --- a/scalding-repl/src/main/scala/com/twitter/scalding/ReplImplicits.scala +++ b/scalding-repl/src/main/scala/com/twitter/scalding/ReplImplicits.scala @@ -42,7 +42,7 @@ trait BaseReplState { * If the repl is started in Hdfs mode, this field is used to preserve the settings * when switching Modes. */ - var storedHdfsMode: Option[Hdfs] = None + var storedHdfsMode: Option[ClusterMode] = None /** Switch to Local mode */ def useLocalMode(): Unit = { @@ -82,13 +82,13 @@ trait BaseReplState { case localMode: Local => { (localMode.toString, System.getProperty("user.dir")) } - case hdfsMode: Hdfs => { + case hdfsMode: HadoopFamilyMode => { val defaultFs = FileSystem.get(hdfsMode.jobConf) val m = customConfig.get(mr2Key) match { case Some("local") => - s"${hdfsMode.getClass.getSimpleName}Local(${hdfsMode.strict})" + s"${hdfsMode.getClass.getSimpleName}Local(${hdfsMode.strictSources})" case _ => - s"${hdfsMode.getClass.getSimpleName}(${hdfsMode.strict})" + s"${hdfsMode.getClass.getSimpleName}(${hdfsMode.strictSources})" } (m, defaultFs.getWorkingDirectory.toString) } @@ -99,7 +99,7 @@ trait BaseReplState { private def modeHadoopConf: Configuration = { mode match { - case hdfsMode: Hdfs => hdfsMode.jobConf + case hdfsMode: HadoopFamilyMode => hdfsMode.jobConf case _ => new Configuration(false) } } 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 fa809ca850..862fa60b37 100644 --- a/scalding-repl/src/main/scala/com/twitter/scalding/ScaldingShell.scala +++ b/scalding-repl/src/main/scala/com/twitter/scalding/ScaldingShell.scala @@ -84,14 +84,11 @@ trait BaseScaldingShell extends MainGenericRunner { val repl = scaldingREPLProvider.apply() scaldingREPL = Some(repl) replState.mode = mode - replState.customConfig = replState.customConfig ++ (mode match { - case _: HadoopMode => cfg - case _ => Config.empty - }) + replState.customConfig = mode.executionMode.defaultConfig ++ replState.customConfig // if in Hdfs mode, store the mode to enable switching between Local and Hdfs mode match { - case m @ Hdfs(_, _) => replState.storedHdfsMode = Some(m) + case m: ClusterMode => replState.storedHdfsMode = Some(m) case _ => () } 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 424a3ca8d2..da37678843 100644 --- a/scalding-repl/src/test/scala/com/twitter/scalding/ReplTest.scala +++ b/scalding-repl/src/test/scala/com/twitter/scalding/ReplTest.scala @@ -34,10 +34,7 @@ class ReplTest extends WordSpec { def test() = { - val suffix = mode match { - case _: CascadingLocal => "local" - case _: HadoopMode => "hadoop" - } + val suffix = mode.name val testPath = "/tmp/scalding-repl/test/" + suffix + "/" val helloRef = List("Hello world", "Goodbye world") @@ -60,7 +57,7 @@ class ReplTest extends WordSpec { assert(s.toString.contains("IterablePipe") || s.toString.contains("TypedPipeFactory")) val pipeName = mode match { - case m: HadoopMode => m.jobConf.get("hadoop.tmp.dir") + case m: HadoopFamilyMode => m.jobConf.get("hadoop.tmp.dir") case _ => "IterableSource" } assert(s.toPipe(Fields.ALL).toString.contains(pipeName)) @@ -187,10 +184,13 @@ class ReplTest extends WordSpec { } "REPL in Hadoop mode" should { - mode = Hdfs(strict = true, new JobConf) + mode = LegacyHadoopMode(strictSources = true, new JobConf) test() } + /* TODO: try to run REPL in Hadoop2-MR1, Tez, and/or Flink modes as well (to the extent all can be + loaded in the same classpath) */ + "findAllUpPath" should { val root = Files.createTempDirectory("scalding-repl").toFile root.deleteOnExit() diff --git a/scripts/scald.rb b/scripts/scald.rb index 898c167f79..973add593f 100755 --- a/scripts/scald.rb +++ b/scripts/scald.rb @@ -61,6 +61,12 @@ VERSIONFILE = open(CONFIG["repo_root"] + "/version.sbt").read SCALDING_VERSION=VERSIONFILE.match(/version.*:=\s*\"([^\"]+)\"/)[1] +FABRIC_SUFFIX=if ENV['SCALD_FABRIC_SUFFIX'] + "-"+ ENV['SCALD_FABRIC_SUFFIX'] + else + "" + end + #optionally set variables (not linux often doesn't have this set, and falls back to TMP. Set up a #YAML file in .scaldrc with "tmpdir: my_tmp_directory_name" or export TMPDIR="/my/tmp" to set on #linux @@ -262,7 +268,7 @@ def get_dep_location(org, dep, version) if (!CONFIG["jar"]) #what jar has all the dependencies for this job - CONFIG["jar"] = repo_root + "/scalding-core/target/scala-#{SHORT_SCALA_VERSION}/scalding-core-assembly-#{SCALDING_VERSION}.jar" + CONFIG["jar"] = repo_root + "/scalding-core/target/scala-#{SHORT_SCALA_VERSION}/scalding-core-assembly#{FABRIC_SUFFIX}-#{SCALDING_VERSION}.jar" end #Check that we can find the jar: diff --git a/scripts/testValidator.sh b/scripts/testValidator.sh index 1e52372e35..0b6a6c0029 100755 --- a/scripts/testValidator.sh +++ b/scripts/testValidator.sh @@ -4,26 +4,44 @@ BASE_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )"/.. && pwd )" cd $BASE_DIR - #Which things do we want to build, all things that start with scalding- or maple are build targets # This will produce a long string targets separated by spaces -TARGET_NAMES=$(ls -d scalding-* maple) +TARGET_NAMES=$(ls -d scalding-* maple | sort) # Cat the travis build file, ignoring the assembly lines # Reformatting any quotes to new lines so we get things nicer split up # Then grep this for the scalding- and maple from above -BUILDS_WE_HAVE=$(cat .travis.yml | grep -v scripts/build_assembly_no_test | tr '"' '\n' | tr ' ' '\n' | grep -e scalding- -e maple ) +BUILDS_WE_HAVE=$(cat .travis.yml |grep "env:" | + sed -e's/env://' -e 's/TEST_TARGET=\"\([^\"]*\)\"/\1/' | \ + tr ' ' '\n' | grep -e scalding- -e maple ) + +BUILDS_WE_REPEAT_PER_FABRIC=$(cat .travis.yml |grep "env:" | grep SCALD_FABRIC_SUFFIX | \ + sed -e's/env://' -e 's/TEST_TARGET=\"\([^\"]*\)\"/\1/' | \ + tr ' ' '\n' | grep -e scalding- -e maple | sort ) # Grab the blacklist, lines starting with #'s BLACKLIST_BUILDS=$(cat .travis.blacklist | egrep -v '^\s*#') TEST_ID=$(date '+%s') + +TEMP_GOAL_PATH="/tmp/scalding_temp_goal.$TEST_ID.txt" +TEMP2_GOAL_PATH="/tmp/scalding_temp2_goal.$TEST_ID.txt" +echo $TARGET_NAMES | tr ' ' '\n' | sort | uniq > $TEMP_GOAL_PATH +echo $BUILDS_WE_REPEAT_PER_FABRIC | tr ' ' '\n' | sort > $TEMP2_GOAL_PATH + +TARGET_WE_REPEAT_PER_FABRIC=$(join $TEMP_GOAL_PATH $TEMP2_GOAL_PATH) +rm -f TEMP_GOAL_PATH +rm -f TEMP2_GOAL_PATH + + GOAL_PATH="/tmp/scalding_goal.$TEST_ID.txt" HAVE_PATH="/tmp/scalding_gHAVE.$TEST_ID.txt" + + # Ideally we want to have each target twice, once for 2.10 and 2.11 # So echo them twice, counting their frequency into the goal path -echo $TARGET_NAMES $TARGET_NAMES | tr ' ' '\n' | sort | uniq -c > $GOAL_PATH +echo $TARGET_NAMES $TARGET_NAMES $TARGET_WE_REPEAT_PER_FABRIC | tr ' ' '\n' | sort | uniq -c > $GOAL_PATH #Now we take the builds we have, appending the #blacklist builds @@ -41,4 +59,4 @@ if [ $RET -eq 0 ]; then else echo -e "Missing some builds, diff $DIFF" exit 1 -fi \ No newline at end of file +fi