From 1a42b5a35fb7a3be7b370b01d4a9301147196b16 Mon Sep 17 00:00:00 2001 From: Anton Panasenko Date: Wed, 22 Feb 2017 17:02:09 -0800 Subject: [PATCH 01/16] Use JobConf input files list for input size computation used by ReducersEstimators --- .../scalding/reducer_estimation/Common.scala | 25 ++++++------------- 1 file changed, 7 insertions(+), 18 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala index 3520774a9c..dab3fddd9d 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala @@ -4,8 +4,8 @@ import cascading.flow.{ FlowStep, Flow, FlowStepStrategy } import com.twitter.algebird.Monoid import com.twitter.scalding.{ StringUtility, Config } import cascading.tap.{ Tap, CompositeTap } -import cascading.tap.hadoop.Hfs -import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapred.{FileInputFormat, JobConf} import org.slf4j.LoggerFactory import java.util.{ List => JList } @@ -54,23 +54,12 @@ object Common { def unrollTaps(step: FlowStep[JobConf]): Seq[Tap[_, _, _]] = unrollTaps(step.getSources.asScala.toSeq) - /** - * Get the total size of the file(s) specified by the Hfs, which may contain a glob - * pattern in its path, so we must be ready to handle that case. - */ - def size(f: Hfs, conf: JobConf): Long = { - val fs = f.getPath.getFileSystem(conf) - fs.globStatus(f.getPath) - .map{ s => fs.getContentSummary(s.getPath).getLength } - .sum - } - - def inputSizes(step: FlowStep[JobConf]): Seq[(String, Long)] = { + def inputSizes(step: FlowStep[JobConf]): Seq[(Path, Long)] = { val conf = step.getConfig - unrollTaps(step).flatMap { - case tap: Hfs => Some(tap.toString -> size(tap, conf)) - case _ => None - } + + FileInputFormat + .getInputPaths(conf) + .map { path => path -> path.getFileSystem(conf).getContentSummary(path).getLength } } def totalInputSize(step: FlowStep[JobConf]): Long = inputSizes(step).map(_._2).sum From 6e3ad9661a69c6cc926915a535adc791ecbcd384 Mon Sep 17 00:00:00 2001 From: Anton Panasenko Date: Wed, 22 Feb 2017 16:16:19 -0800 Subject: [PATCH 02/16] Using MAP_OUTPUT_BYTES instead of FILE_BYTES_READ in RatioBasedEstimator --- .../scalding/reducer_estimation/Common.scala | 1 + .../RatioBasedEstimator.scala | 3 ++- .../RatioBasedEstimatorTest.scala | 9 ++++--- .../HRavenHistoryService.scala | 26 ++++++++++++++----- 4 files changed, 28 insertions(+), 11 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala index 3520774a9c..0d205d379c 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala @@ -253,6 +253,7 @@ final case class FlowStepHistory(keys: FlowStepKeys, failedReduces: Long, mapFileBytesRead: Long, mapFileBytesWritten: Long, + mapOutputBytes: Long, reduceFileBytesRead: Long, hdfsBytesRead: Long, hdfsBytesWritten: Long, diff --git a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimator.scala b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimator.scala index 9cc52ec9ff..a895257472 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimator.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimator.scala @@ -60,8 +60,9 @@ abstract class RatioBasedEstimator extends ReducerEstimator { } else { val ratios = for { h <- history + if h.mapOutputBytes > 0 if acceptableInputRatio(inputBytes, h.hdfsBytesRead, threshold) - } yield h.reduceFileBytesRead / h.hdfsBytesRead.toDouble + } yield h.mapOutputBytes / h.hdfsBytesRead.toDouble if (ratios.isEmpty) { LOG.warn(s"No matching history found within input ratio threshold: $threshold") diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimatorTest.scala b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimatorTest.scala index 5a9cadc1a8..1ee2d281f4 100644 --- a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimatorTest.scala +++ b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimatorTest.scala @@ -31,10 +31,10 @@ object ErrorHistoryService extends HistoryService { object HistoryServiceWithData { // we only care about these two input size fields for RatioBasedEstimator - def makeHistory(inputHdfsBytesRead: Long, inputHdfsReduceFileBytesRead: Long): FlowStepHistory = - makeHistory(inputHdfsBytesRead, inputHdfsReduceFileBytesRead, Seq()) + def makeHistory(inputHdfsBytesRead: Long, mapOutputBytes: Long): FlowStepHistory = + makeHistory(inputHdfsBytesRead, mapOutputBytes, Seq()) - def makeHistory(inputHdfsBytesRead: Long, inputHdfsReduceFileBytesRead: Long, taskRuntimes: Seq[Long]): FlowStepHistory = { + def makeHistory(inputHdfsBytesRead: Long, mapOutputBytes: Long, taskRuntimes: Seq[Long]): FlowStepHistory = { val random = new scala.util.Random(123) val tasks = taskRuntimes.map { time => val startTime = random.nextLong @@ -58,7 +58,8 @@ object HistoryServiceWithData { failedReduces = 0L, mapFileBytesRead = 0L, mapFileBytesWritten = 0L, - reduceFileBytesRead = inputHdfsReduceFileBytesRead, + mapOutputBytes = mapOutputBytes, + reduceFileBytesRead = 0l, hdfsBytesRead = inputHdfsBytesRead, hdfsBytesWritten = 0L, mapperTimeMillis = 0L, 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 0e7b0ffa68..1d3678f652 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 @@ -1,17 +1,15 @@ package com.twitter.scalding.hraven.reducer_estimation import java.io.IOException - import cascading.flow.FlowStep -import com.twitter.hraven.{ Flow, JobDetails } +import com.twitter.hraven.{Constants, CounterMap, Flow, HistoryFileType, JobDetails} import com.twitter.hraven.rest.client.HRavenRestClient import com.twitter.scalding.reducer_estimation._ import org.apache.hadoop.mapred.JobConf import org.slf4j.LoggerFactory import scala.collection.JavaConverters._ -import com.twitter.hraven.JobDescFactory.{ JOBTRACKER_KEY, RESOURCE_MANAGER_KEY } - -import scala.util.{ Failure, Success, Try } +import com.twitter.hraven.JobDescFactory.{JOBTRACKER_KEY, RESOURCE_MANAGER_KEY} +import scala.util.{Failure, Success, Try} object HRavenClient { import HRavenHistoryService.jobConfToRichConfig @@ -46,6 +44,7 @@ object HRavenHistoryService extends HistoryService { "status", "startTime", "finishTime").asJava + private val MapOutputBytesKey = "MAP_OUTPUT_BYTES" val RequiredJobConfigs = Seq("cascading.flow.step.num") @@ -179,7 +178,7 @@ object HRavenHistoryService extends HistoryService { } yield toFlowStepHistory(keys, step, tasks) } - private def toFlowStepHistory(keys: FlowStepKeys, step: JobDetails, tasks: Seq[Task]) = + private def toFlowStepHistory(keys: FlowStepKeys, step: JobDetails, tasks: Seq[Task]) = { FlowStepHistory( keys = keys, submitTime = step.getSubmitTime, @@ -193,6 +192,7 @@ object HRavenHistoryService extends HistoryService { failedReduces = step.getFailedReduces, mapFileBytesRead = step.getMapFileBytesRead, mapFileBytesWritten = step.getMapFileBytesWritten, + mapOutputBytes = mapOutputBytes(step), reduceFileBytesRead = step.getReduceFileBytesRead, hdfsBytesRead = step.getHdfsBytesRead, hdfsBytesWritten = step.getHdfsBytesWritten, @@ -201,6 +201,20 @@ object HRavenHistoryService extends HistoryService { reduceShuffleBytes = step.getReduceShuffleBytes, cost = 0, tasks = tasks) + } + + private def mapOutputBytes(step: JobDetails): Long = { + if (step.getHistoryFileType == HistoryFileType.TWO) { + getCounterValueAsLong(step.getMapCounters, Constants.TASK_COUNTER_HADOOP2, MapOutputBytesKey) + } else { + getCounterValueAsLong(step.getMapCounters, Constants.TASK_COUNTER, MapOutputBytesKey) + } + } + + private def getCounterValueAsLong(counters: CounterMap, counterGroupName: String, counterName: String): Long = { + val counter = counters.getCounter(counterGroupName, counterName) + if (counter != null) counter.getValue else 0L + } } class HRavenRatioBasedEstimator extends RatioBasedEstimator { From 8bebb576d692c7e1fb6ddc9f9f48cd59dca56dce Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Tue, 28 Feb 2017 12:01:57 -1000 Subject: [PATCH 03/16] Use Batched in Sketch production (#1648) --- .../com/twitter/scalding/typed/Sketched.scala | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/Sketched.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/Sketched.scala index 0a8a2fea6b..3c0f86f0df 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/Sketched.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/Sketched.scala @@ -41,12 +41,21 @@ case class Sketched[K, V](pipe: TypedPipe[(K, V)], def reducers = Some(numReducers) private lazy implicit val cms = CMS.monoid[Bytes](eps, delta, seed) - lazy val sketch: TypedPipe[CMS[Bytes]] = + lazy val sketch: TypedPipe[CMS[Bytes]] = { + // every 10k items, compact into a CMS to prevent very slow mappers + lazy implicit val batchedSG = Batched.compactingSemigroup[CMS[Bytes]](10000) + pipe - .map { case (k, _) => cms.create(Bytes(serialization(k))) } - .sum // sum everything to one value - .toTypedPipe - .forceToDisk + .map { case (k, _) => ((), Batched(cms.create(Bytes(serialize(k))))) } + .sumByLocalKeys + .map { + case (_, batched) => batched.sum + } // remove the Batched before going to the reducers + .groupAll + .sum + .values + .forceToDisk // make sure we materialize when we have 1 item + } /** * Like a hashJoin, this joiner does not see all the values V at one time, only one at a time. From b892acf2d9ab2378d6697b579f705ff0b6cb7bea Mon Sep 17 00:00:00 2001 From: Anton Panasenko Date: Thu, 2 Mar 2017 09:31:34 -0800 Subject: [PATCH 04/16] Fixed reduce estimator for paths with a glob pattern --- .../scalding/reducer_estimation/Common.scala | 13 ++++++- .../ReducerEstimatorTest.scala | 39 ++++++++++++++++++- 2 files changed, 50 insertions(+), 2 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala index 7b76ef3714..3a821e43a0 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala @@ -54,12 +54,23 @@ object Common { def unrollTaps(step: FlowStep[JobConf]): Seq[Tap[_, _, _]] = unrollTaps(step.getSources.asScala.toSeq) + /** + * Get the total size of the input paths, which may contain a glob + * pattern in its path, so we must be ready to handle that case. + */ def inputSizes(step: FlowStep[JobConf]): Seq[(Path, Long)] = { val conf = step.getConfig FileInputFormat .getInputPaths(conf) - .map { path => path -> path.getFileSystem(conf).getContentSummary(path).getLength } + .map { path => + val fs = path.getFileSystem(conf) + val size = fs.globStatus(path) + .map(status => fs.getContentSummary(status.getPath).getLength) + .sum + + path -> size + } } def totalInputSize(step: FlowStep[JobConf]): Long = inputSizes(step).map(_._2).sum diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala index 3fe692bdfa..9a43376428 100644 --- a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala +++ b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala @@ -9,7 +9,8 @@ import scala.collection.JavaConverters._ object HipJob { val InSrcFileSize = 2496L - val inSrc = TextLine(getClass.getResource("/hipster.txt").toString) // file size is 2496 bytes + val inPath = getClass.getResource("/hipster.txt") // file size is 2496 bytes + val inSrc = TextLine(inPath.toString) val InScoresFileSize = 174L val inScores = TypedTsv[(String, Double)](getClass.getResource("/scores.tsv").toString) // file size is 174 bytes val out = TypedTsv[Double]("output") @@ -62,6 +63,24 @@ class SimpleJob(args: Args, customConfig: Config) extends Job(args) { .write(counts) } +class SimpleGlobJob(args: Args, customConfig: Config) extends Job(args) { + import HipJob._ + + val inSrcGlob = inPath.toString.replace("hipster", "*") + val inSrc = TextLine(inSrcGlob) + + override def config = super.config ++ customConfig.toMap.toMap + + TypedPipe.from(inSrc) + .flatMap(_.split("[^\\w]+")) + .map(_.toLowerCase -> 1) + .group + // force the number of reducers to two, to test with/without estimation + .withReducers(2) + .sum + .write(counts) +} + class GroupAllJob(args: Args, customConfig: Config) extends Job(args) { import HipJob._ @@ -106,6 +125,24 @@ class ReducerEstimatorTest extends WordSpec with Matchers with HadoopSharedPlatf .run() } + "run with correct number of reducers when we have a glob pattern in path" in { + val customConfig = Config.empty.addReducerEstimator(classOf[InputSizeReducerEstimator]) + + (InputSizeReducerEstimator.BytesPerReducer -> (1L << 10).toString) + + (Config.ReducerEstimatorOverride -> "true") + + HadoopPlatformJobTest(new SimpleGlobJob(_, customConfig), cluster) + .inspectCompletedFlow { flow => + val steps = flow.getFlowSteps.asScala + steps should have size 1 + + val conf = Config.fromHadoop(steps.head.getConfig) + conf.getNumReducers should contain (3) + conf.get(EstimatorConfig.originalNumReducers) should contain ("2") + } + .run() + + } + "run with correct number of reducers when overriding set values" in { val customConfig = Config.empty.addReducerEstimator(classOf[InputSizeReducerEstimator]) + (InputSizeReducerEstimator.BytesPerReducer -> (1L << 10).toString) + From f780fd8686c17c2e1fe9459b1d8b6011cab5a781 Mon Sep 17 00:00:00 2001 From: Jeff Balogh Date: Wed, 8 Mar 2017 15:11:43 -0800 Subject: [PATCH 05/16] Expand libjars globs to match the behavior of Tool. --- .../src/main/scala/com/twitter/scalding/ScaldingShell.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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..cd312010ea 100644 --- a/scalding-repl/src/main/scala/com/twitter/scalding/ScaldingShell.scala +++ b/scalding-repl/src/main/scala/com/twitter/scalding/ScaldingShell.scala @@ -124,7 +124,7 @@ trait BaseScaldingShell extends MainGenericRunner { * @param args from the command line. */ def main(args: Array[String]): Unit = { - val retVal = process(args) + val retVal = process(ExpandLibJarsGlobs(args)) if (!retVal) { sys.exit(1) } From 793d051379a01cb7c1153c2e75289f616fbf31f6 Mon Sep 17 00:00:00 2001 From: Alex Lambert Date: Wed, 22 Mar 2017 11:23:39 -0400 Subject: [PATCH 06/16] Improve Vertica support for scalding-db --- .../impl/ColumnDefinitionProviderImpl.scala | 15 ++++++- .../scalding/db/macros/MacrosUnitTests.scala | 41 +++++++++++++++++++ version.sbt | 2 +- 3 files changed, 56 insertions(+), 2 deletions(-) diff --git a/scalding-db/src/main/scala/com/twitter/scalding/db/macros/impl/ColumnDefinitionProviderImpl.scala b/scalding-db/src/main/scala/com/twitter/scalding/db/macros/impl/ColumnDefinitionProviderImpl.scala index a860bbf8a1..3003e15383 100644 --- a/scalding-db/src/main/scala/com/twitter/scalding/db/macros/impl/ColumnDefinitionProviderImpl.scala +++ b/scalding-db/src/main/scala/com/twitter/scalding/db/macros/impl/ColumnDefinitionProviderImpl.scala @@ -191,8 +191,10 @@ object ColumnDefinitionProviderImpl { case (cf: ColumnFormat[_], pos: Int) => val fieldName = cf.fieldName.toStr val typeNameTerm = newTermName(c.fresh(s"colTypeName_$pos")) + // MySQL uses names like `DATE`, `INTEGER` and `VARCHAR`; + // Vertica uses names like `Date`, `Integer` and `Varchar` val typeName = q""" - val $typeNameTerm = $rsmdTerm.getColumnTypeName(${pos + 1}) + val $typeNameTerm = $rsmdTerm.getColumnTypeName(${pos + 1}).toUpperCase(java.util.Locale.US) """ // certain types have synonyms, so we group them together here // note: this is mysql specific @@ -201,6 +203,17 @@ object ColumnDefinitionProviderImpl { case "VARCHAR" => q"""List("VARCHAR", "CHAR").contains($typeNameTerm)""" case "BOOLEAN" | "TINYINT" => q"""List("BOOLEAN", "BOOL", "TINYINT").contains($typeNameTerm)""" case "INT" => q"""List("INTEGER", "INT").contains($typeNameTerm)""" + // In Vertica, `INTEGER`, `INT`, `BIGINT`, `INT8`, `SMALLINT`, and `TINYINT` are all 64 bits + // https://my.vertica.com/docs/7.1.x/HTML/Content/Authoring/SQLReferenceManual/DataTypes/Numeric/INTEGER.htm + // In MySQL, `TINYINT`, `SMALLINT`, `MEDIUMINT`, `INT`, and `BIGINT` are all <= 64 bits + // https://dev.mysql.com/doc/refman/5.7/en/integer-types.html + // As the user has told us this field can store a `BIGINT`, we can safely accept any of these + // types from the database. + case "BIGINT" => + q"""List("INTEGER", "INT", "BIGINT", "INT8", "SMALLINT", + "TINYINT", "SMALLINT", "MEDIUMINT").contains($typeNameTerm)""" + // for Vertica support + case "DATE" => q"""List("DATE").contains($typeNameTerm)""" case f => q"""$f == $typeNameTerm""" } val typeAssert = q""" diff --git a/scalding-db/src/test/scala/com/twitter/scalding/db/macros/MacrosUnitTests.scala b/scalding-db/src/test/scala/com/twitter/scalding/db/macros/MacrosUnitTests.scala index f46908d55f..9136a43a58 100644 --- a/scalding-db/src/test/scala/com/twitter/scalding/db/macros/MacrosUnitTests.scala +++ b/scalding-db/src/test/scala/com/twitter/scalding/db/macros/MacrosUnitTests.scala @@ -69,6 +69,11 @@ case class ExhaustiveJdbcCaseClass( optiLong: Option[Long] // Nullable long ) +private final case class VerticaCaseClass( + verticaLong: Long, + @date verticaDate: Date, + @varchar @size(size = 1) verticaVarchar1: String) + case class CaseClassWithDate( id: Long, myDateWithTime: Date, @@ -226,6 +231,42 @@ class JdbcMacroUnitTests extends WordSpec with Matchers with MockitoSugar { } + "interoperates with Vertica, which uses different type names" should { + val typeDescriptor = DBMacro.toDBTypeDescriptor[VerticaCaseClass] + val expectedColumns = List( + ColumnDefinition(BIGINT, ColumnName("verticaLong"), NotNullable, None, None), + ColumnDefinition(DATE, ColumnName("verticaDate"), NotNullable, None, None), + ColumnDefinition(VARCHAR, ColumnName("verticaVarchar1"), NotNullable, Some(1), None)) + assert(typeDescriptor.columnDefn.columns.toList === expectedColumns) + + // Vertica uses `Integer` + val int64TypeNames = List("Integer", "INTEGER", "INT", "BIGINT", "INT8", "SMALLINT", + "TINYINT", "SMALLINT", "MEDIUMINT") + // Vertica uses `Date` + val dateTypeNames = List("Date", "DATE") + // Vertica uses `Varchar` + val varcharTypeNames = List("Varchar", "VARCHAR") + + int64TypeNames foreach { int64TypeName => + dateTypeNames foreach { dateTypeName => + varcharTypeNames foreach { varcharTypeName => + val resultSetMetaData = mock[ResultSetMetaData] + when(resultSetMetaData.getColumnTypeName(1)) thenReturn (int64TypeName) + when(resultSetMetaData.isNullable(1)) thenReturn (ResultSetMetaData.columnNoNulls) + when(resultSetMetaData.getColumnTypeName(2)) thenReturn (dateTypeName) + when(resultSetMetaData.isNullable(2)) thenReturn (ResultSetMetaData.columnNoNulls) + when(resultSetMetaData.getColumnTypeName(3)) thenReturn (varcharTypeName) + when(resultSetMetaData.isNullable(3)) thenReturn (ResultSetMetaData.columnNoNulls) + + val validationResult = + typeDescriptor.columnDefn.resultSetExtractor.validate(resultSetMetaData) + + assert(validationResult.isSuccess, validationResult) + } + } + } + } + "Big Jdbc Test" should { isColumnDefinitionAvailable[ExhaustiveJdbcCaseClass] diff --git a/version.sbt b/version.sbt index 6927b97e82..c40880e8e9 100644 --- a/version.sbt +++ b/version.sbt @@ -1 +1 @@ -version in ThisBuild := "0.16.1-SNAPSHOT" \ No newline at end of file +version in ThisBuild := "0.16.1-SNAPSHOT4" From 9731e0ef87143ca11c24f40b93f86f5c02228ea2 Mon Sep 17 00:00:00 2001 From: Alex Lambert Date: Wed, 22 Mar 2017 16:02:57 -0400 Subject: [PATCH 07/16] reset version --- version.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/version.sbt b/version.sbt index c40880e8e9..6927b97e82 100644 --- a/version.sbt +++ b/version.sbt @@ -1 +1 @@ -version in ThisBuild := "0.16.1-SNAPSHOT4" +version in ThisBuild := "0.16.1-SNAPSHOT" \ No newline at end of file From 96e9ff721768f986a08a85efed9d0501fa56805b Mon Sep 17 00:00:00 2001 From: Alex Lambert Date: Wed, 22 Mar 2017 16:05:00 -0400 Subject: [PATCH 08/16] remove unnecessary condition --- .../scalding/db/macros/impl/ColumnDefinitionProviderImpl.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/scalding-db/src/main/scala/com/twitter/scalding/db/macros/impl/ColumnDefinitionProviderImpl.scala b/scalding-db/src/main/scala/com/twitter/scalding/db/macros/impl/ColumnDefinitionProviderImpl.scala index 3003e15383..23037ddd41 100644 --- a/scalding-db/src/main/scala/com/twitter/scalding/db/macros/impl/ColumnDefinitionProviderImpl.scala +++ b/scalding-db/src/main/scala/com/twitter/scalding/db/macros/impl/ColumnDefinitionProviderImpl.scala @@ -212,8 +212,6 @@ object ColumnDefinitionProviderImpl { case "BIGINT" => q"""List("INTEGER", "INT", "BIGINT", "INT8", "SMALLINT", "TINYINT", "SMALLINT", "MEDIUMINT").contains($typeNameTerm)""" - // for Vertica support - case "DATE" => q"""List("DATE").contains($typeNameTerm)""" case f => q"""$f == $typeNameTerm""" } val typeAssert = q""" From 12aaa7e1f4315c2227080f18856003d2780e7e8f Mon Sep 17 00:00:00 2001 From: Piyush Narang Date: Wed, 22 Mar 2017 19:49:42 -0700 Subject: [PATCH 09/16] Scalding updates for Scala 2.12 (#1646) * start upgrade to 2.12 * Bump bijection, chill * Add Scala 2.12 related repl code * Scala 2.12 version bumps + _equals fix * Suppress Option[Partial] warnings * Add fallback for scrooge-serializer on Scala 2.10 * Add scala 2.12 build targets * Fix parquet-scrooge and parquet tests on 2.12 * Drop support for 2.10 * Fix scalding-serialization tests * Get thrift macros to compile with latest Scalacheck * Fix scalding-db tests * Exclude guava from scrooge imports * Use mima 0.1.14 * Update scalding-thrift-macros jdk version to java8 * Fix OrderedSerialization test failures * Break boxed lambdas into 2 lists to get around Scala 2.12 bug * Try catching CNF in LineNumber * Drop line numbers from hashJoin scalding hadoop test * Switch default to jdk8 as scrooge deps have been built with that * Update scald.rb to pick up 2.12 * Try latest jline version * Try and fix merge issues * Wrap Execution line better * Remove scrooge method * Drop _equals method * Try private[ser] for Boxed issues * Reorder hashCode and equals --- .travis.yml | 40 ++--- build.sbt | 150 +++++++----------- project/plugins.sbt | 9 +- .../com/twitter/scalding/Execution.scala | 10 +- .../com/twitter/scalding/GroupBuilder.scala | 3 +- .../scala/com/twitter/scalding/JobTest.scala | 3 +- .../com/twitter/scalding/LineNumber.scala | 15 +- .../scala/com/twitter/scalding/Mode.scala | 2 +- .../com/twitter/scalding/Operations.scala | 14 +- .../com/twitter/scalding/TestTapFactory.scala | 1 + .../com/twitter/scalding/TuplePacker.scala | 2 +- .../impl/TypeDescriptorProviderImpl.scala | 1 + .../scalding/mathematics/Matrix2.scala | 2 +- .../scalding/reducer_estimation/Common.scala | 2 +- .../serialization/WrappedSerialization.scala | 2 +- .../twitter/scalding/typed/CoGrouped.scala | 2 +- .../twitter/scalding/typed/TypedPipe.scala | 1 + .../scalding/db/macros/MacrosUnitTests.scala | 9 +- .../platform/PlatformExecutionTest.scala | 4 +- .../scalding/platform/PlatformTest.scala | 13 +- .../HRavenHistoryService.scala | 6 +- .../src/test/resources/test.thrift | 8 - .../com/twitter/scalding/ILoopCompat.scala | 11 ++ .../scalding/serialization/Boxed.scala | 16 +- .../scalding/serialization/Writer.scala | 2 +- .../SealedTraitLike.scala | 12 +- .../providers/ProductOrderedBuf.scala | 2 +- .../macros/MacroOrderingProperties.scala | 4 +- .../macros/TraversableHelperLaws.scala | 4 +- .../ordered_serialization/UnionLike.scala | 12 +- .../thrift/macros/ScroogeGenerators.scala | 9 +- scripts/scald.rb | 2 + 32 files changed, 189 insertions(+), 184 deletions(-) create mode 100644 scalding-repl/src/main/scala-2.12/com/twitter/scalding/ILoopCompat.scala diff --git a/.travis.yml b/.travis.yml index 920317aeb5..7342e19a7a 100644 --- a/.travis.yml +++ b/.travis.yml @@ -1,5 +1,5 @@ language: scala -jdk: oraclejdk7 +jdk: oraclejdk8 sudo: false before_install: @@ -27,55 +27,55 @@ addons: matrix: include: #BASE TESTS - - scala: 2.10.6 + - scala: 2.11.8 env: BUILD="base" TEST_TARGET="scalding-args scalding-date maple" script: "scripts/run_test.sh" - - scala: 2.11.8 + - scala: 2.12.1 env: BUILD="base" TEST_TARGET="scalding-args scalding-date maple" script: "scripts/run_test.sh" - - scala: 2.10.6 + - scala: 2.11.8 env: BUILD="base" TEST_TARGET="scalding-avro scalding-hraven scalding-commons scalding-parquet scalding-parquet-scrooge" script: "scripts/run_test.sh" - - scala: 2.11.8 + - scala: 2.12.1 env: BUILD="base" TEST_TARGET="scalding-avro scalding-hraven scalding-commons scalding-parquet scalding-parquet-scrooge" script: "scripts/run_test.sh" - - scala: 2.10.6 + - scala: 2.11.8 env: BUILD="base" TEST_TARGET="scalding-core scalding-jdbc scalding-json scalding-db" script: "scripts/run_test.sh" - - scala: 2.11.8 + - scala: 2.12.1 env: BUILD="base" TEST_TARGET="scalding-core scalding-jdbc scalding-json scalding-db" script: "scripts/run_test.sh" - - scala: 2.10.6 + - scala: 2.11.8 env: BUILD="base" TEST_TARGET="scalding-hadoop-test" script: "scripts/run_test.sh" - - scala: 2.11.8 + - scala: 2.12.1 env: BUILD="base" TEST_TARGET="scalding-hadoop-test" script: "scripts/run_test.sh" - - scala: 2.10.6 + - scala: 2.11.8 env: BUILD="base" TEST_TARGET="scalding-serialization" script: "scripts/run_test.sh" - - scala: 2.11.8 + - scala: 2.12.1 env: BUILD="base" TEST_TARGET="scalding-serialization" script: "scripts/run_test.sh" - - scala: 2.10.6 + - scala: 2.11.8 env: BUILD="base" TEST_TARGET="scalding-thrift-macros" script: "scripts/run_test.sh" - - scala: 2.11.8 + - scala: 2.12.1 env: BUILD="base" TEST_TARGET="scalding-thrift-macros" script: "scripts/run_test.sh" - - scala: 2.10.6 + - scala: 2.11.8 env: BUILD="test tutorials and matrix tutorials and repl" TEST_TARGET="scalding-repl" script: - "scripts/run_test.sh" @@ -84,7 +84,7 @@ matrix: - "scripts/build_assembly_no_test.sh scalding-assembly" - "scripts/test_matrix_tutorials.sh" - - scala: 2.11.8 + - scala: 2.12.1 env: BUILD="test tutorials and matrix tutorials and repl" TEST_TARGET="scalding-repl" script: - "scripts/run_test.sh" @@ -93,9 +93,10 @@ matrix: - "scripts/build_assembly_no_test.sh scalding-assembly" - "scripts/test_matrix_tutorials.sh" - - scala: 2.10.6 - env: BUILD="test repl and typed tutorials" + - scala: 2.11.8 + env: BUILD="test repl and typed tutorials and microsite" script: + - ./sbt ++$TRAVIS_SCALA_VERSION clean docs/makeMicrosite - "scripts/build_assembly_no_test.sh scalding-repl" - "scripts/test_repl_tutorial.sh" - "scripts/build_assembly_no_test.sh scalding-core" @@ -103,10 +104,9 @@ matrix: - "scripts/build_assembly_no_test.sh execution-tutorial" - "scripts/test_execution_tutorial.sh" - - scala: 2.11.8 - env: BUILD="test repl and typed tutorials and microsite" + - scala: 2.12.1 + env: BUILD="test repl and typed tutorials" script: - - ./sbt ++$TRAVIS_SCALA_VERSION clean docs/makeMicrosite - "scripts/build_assembly_no_test.sh scalding-repl" - "scripts/test_repl_tutorial.sh" - "scripts/build_assembly_no_test.sh scalding-core" diff --git a/build.sbt b/build.sbt index 806d0517b6..d448c83dda 100644 --- a/build.sbt +++ b/build.sbt @@ -1,6 +1,5 @@ import AssemblyKeys._ import ReleaseTransformations._ -import com.twitter.scrooge.ScroogeSBT import com.typesafe.sbt.SbtGhPages.GhPagesKeys._ import com.typesafe.sbt.SbtScalariform._ import com.typesafe.tools.mima.plugin.MimaPlugin.mimaDefaultSettings @@ -15,33 +14,32 @@ def scalaBinaryVersion(scalaVersion: String) = scalaVersion match { case version if version startsWith "2.12" => "2.12" case _ => sys.error("unknown error") } -def isScala210x(scalaVersion: String) = scalaBinaryVersion(scalaVersion) == "2.10" val algebirdVersion = "0.13.0" val apacheCommonsVersion = "2.2" val avroVersion = "1.7.4" -val bijectionVersion = "0.9.1" +val bijectionVersion = "0.9.5" val cascadingAvroVersion = "2.1.2" -val chillVersion = "0.8.3" +val chillVersion = "0.8.4" val elephantbirdVersion = "4.8" val hadoopLzoVersion = "0.4.19" val hadoopVersion = "2.5.0" val hbaseVersion = "0.94.10" val hravenVersion = "0.9.17.t05" -val jacksonVersion = "2.4.2" -val json4SVersion = "3.2.11" +val jacksonVersion = "2.8.7" +val json4SVersion = "3.5.0" val paradiseVersion = "2.1.0" val parquetVersion = "1.8.1" val protobufVersion = "2.4.1" -val quasiquotesVersion = "2.0.1" -val scalaCheckVersion = "1.12.2" -val scalaTestVersion = "2.2.6" -val scalameterVersion = "0.6" -val scroogeVersion = "3.20.0" +val scalameterVersion = "0.8.2" +val scalaCheckVersion = "1.13.4" +val scalaTestVersion = "3.0.1" +val scroogeVersion = "4.12.0" val slf4jVersion = "1.6.6" val thriftVersion = "0.5.0" val junitVersion = "4.10" val macroCompatVersion = "1.1.1" +val jlineVersion = "2.14.3" val printDependencyClasspath = taskKey[Unit]("Prints location of the dependencies") @@ -50,7 +48,7 @@ val sharedSettings = assemblySettings ++ scalariformSettings ++ Seq( scalaVersion := "2.11.8", - crossScalaVersions := Seq("2.10.6", scalaVersion.value), + crossScalaVersions := Seq(scalaVersion.value, "2.12.1"), ScalariformKeys.preferences := formattingPreferences, @@ -69,10 +67,9 @@ val sharedSettings = assemblySettings ++ scalariformSettings ++ Seq( ), resolvers ++= Seq( - "Local Maven Repository" at "file://" + Path.userHome.absolutePath + "/.m2/repository", - "maven central" at "https://repo.maven.apache.org/maven2", - "releases" at "https://oss.sonatype.org/content/repositories/releases", - "snapshots" at "https://oss.sonatype.org/content/repositories/snapshots", + Opts.resolver.mavenLocalFile, + Opts.resolver.sonatypeSnapshots, + Opts.resolver.sonatypeReleases, "Concurrent Maven Repo" at "http://conjars.org/repo", "Twitter Maven" at "http://maven.twttr.com", "Cloudera" at "https://repository.cloudera.com/artifactory/cloudera-repos/" @@ -105,13 +102,6 @@ val sharedSettings = assemblySettings ++ scalariformSettings ++ Seq( "-language:existentials" ), - scalacOptions ++= { - if (isScala210x(scalaVersion.value)) - Seq("-Xdivergence211") - else - Seq() - }, - /** * add linter for common scala issues: * https://github.com/HairyFotr/linter @@ -172,6 +162,7 @@ val sharedSettings = assemblySettings ++ scalariformSettings ++ Seq( case s if s.endsWith("pom.xml") => MergeStrategy.last case s if s.endsWith(".jnilib") => MergeStrategy.rename case s if s.endsWith("jansi.dll") => MergeStrategy.rename + case s if s.endsWith("libjansi.so") => MergeStrategy.rename case s if s.endsWith("properties") => MergeStrategy.filterDistinctLines case x => (mergeStrategy in assembly).value(x) }, @@ -212,12 +203,9 @@ val sharedSettings = assemblySettings ++ scalariformSettings ++ Seq( lazy val scalding = Project( id = "scalding", base = file("."), - settings = sharedSettings -).settings( - test := {}, - publish := {}, // skip publishing for this root project. - publishLocal := {} -).aggregate( + settings = sharedSettings) + .settings(noPublishSettings) + .aggregate( scaldingArgs, scaldingDate, scaldingCore, @@ -240,12 +228,9 @@ lazy val scalding = Project( lazy val scaldingAssembly = Project( id = "scalding-assembly", base = file("assembly"), - settings = sharedSettings -).settings( - test := {}, - publish := {}, // skip publishing for this root project. - publishLocal := {} -).aggregate( + settings = sharedSettings) + .settings(noPublishSettings) + .aggregate( scaldingArgs, scaldingDate, scaldingCore, @@ -279,11 +264,11 @@ lazy val noPublishSettings = Seq( * This returns the youngest jar we released that is compatible with * the current. */ -val unreleasedModules = Set[String]("hadoop-test") //releases 0.11 +val ignoredModules = Set[String]("benchmarks") def youngestForwardCompatible(subProj: String) = Some(subProj) - .filterNot(unreleasedModules.contains(_)) + .filterNot(ignoredModules.contains(_)) .map { s => "com.twitter" %% (s"scalding-$s") % "0.16.0" } @@ -306,14 +291,15 @@ lazy val cascadingVersion = lazy val cascadingJDBCVersion = System.getenv.asScala.getOrElse("SCALDING_CASCADING_JDBC_VERSION", "2.6.0") -lazy val scaldingBenchmarks = module("benchmarks").settings( - libraryDependencies ++= Seq( - "com.storm-enroute" %% "scalameter" % scalameterVersion % "test", - "org.scalacheck" %% "scalacheck" % scalaCheckVersion % "test" - ), - testFrameworks += new TestFramework("org.scalameter.ScalaMeterFramework"), - parallelExecution in Test := false -).dependsOn(scaldingCore) +lazy val scaldingBenchmarks = module("benchmarks") + .settings( + libraryDependencies ++= Seq( + "com.storm-enroute" %% "scalameter" % scalameterVersion % "test", + "org.scalacheck" %% "scalacheck" % scalaCheckVersion % "test" + ), + testFrameworks += new TestFramework("org.scalameter.ScalaMeterFramework"), + parallelExecution in Test := false + ).dependsOn(scaldingCore) lazy val scaldingCore = module("core").settings( libraryDependencies ++= Seq( @@ -333,10 +319,7 @@ lazy val scaldingCore = module("core").settings( "org.scala-lang" % "scala-library" % scalaVersion.value, "org.scala-lang" % "scala-reflect" % scalaVersion.value, "org.slf4j" % "slf4j-api" % slf4jVersion, - "org.slf4j" % "slf4j-log4j12" % slf4jVersion % "provided") ++ - (if (isScala210x(scalaVersion.value)) - Seq("org.scalamacros" %% "quasiquotes" % quasiquotesVersion) - else Seq()), + "org.slf4j" % "slf4j-log4j12" % slf4jVersion % "provided"), addCompilerPlugin("org.scalamacros" % "paradise" % paradiseVersion cross CrossVersion.full) ).dependsOn(scaldingArgs, scaldingDate, scaldingSerialization, maple) @@ -354,7 +337,8 @@ lazy val scaldingCommons = module("commons").settings( "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided", "org.apache.thrift" % "libthrift" % thriftVersion, // TODO: split this out into a scalding-scrooge - "com.twitter" %% "scrooge-serializer" % scroogeVersion % "provided", + "com.twitter" %% "scrooge-serializer" % scroogeVersion % "provided" + exclude("com.google.guava", "guava"), "org.slf4j" % "slf4j-api" % slf4jVersion, "org.slf4j" % "slf4j-log4j12" % slf4jVersion % "provided", "junit" % "junit" % junitVersion % "test" @@ -373,18 +357,10 @@ lazy val scaldingAvro = module("avro").settings( lazy val scaldingParquetFixtures = module("parquet-fixtures") .settings( scroogeThriftSourceFolder in Test := baseDirectory.value / "src/test/resources", - sourceGenerators in Test += Def.task { - val sources = (scroogeThriftSources in Test).value - val out = streams.value - // for some reason, sbt sometimes calls us multiple times, often with no source files. - if ((scroogeIsDirty in Test).value && sources.nonEmpty) { - out.log.info("Generating scrooge thrift for %s ...".format(sources.mkString(", "))) - ScroogeSBT.compile(out.log, sourceManaged.value, sources.toSet, Set(), Map(), "java", Set("--language", "java")) - } - (sourceManaged.value ** "*.java").get.toSeq - }.taskValue, + scroogeLanguages in Test := Seq("java", "scala"), libraryDependencies ++= Seq( - "com.twitter" %% "scrooge-serializer" % scroogeVersion % "provided", + "com.twitter" %% "scrooge-serializer" % scroogeVersion % "provided" + exclude("com.google.guava", "guava"), "commons-lang" % "commons-lang" % apacheCommonsVersion, // needed for HashCodeBuilder used in thriftjava "org.apache.thrift" % "libthrift" % thriftVersion ) @@ -408,7 +384,7 @@ lazy val scaldingParquet = module("parquet").settings( "com.twitter" %% "chill-bijection" % chillVersion, "com.twitter.elephantbird" % "elephant-bird-core" % elephantbirdVersion % "test", "org.typelevel" %% "macro-compat" % macroCompatVersion - ) ++ (if(isScala210x(scalaVersion.value)) Seq("org.scalamacros" %% "quasiquotes" % quasiquotesVersion) else Seq()), + ), addCompilerPlugin("org.scalamacros" % "paradise" % paradiseVersion cross CrossVersion.full)) .dependsOn(scaldingCore, scaldingHadoopTest % "test", scaldingParquetFixtures % "test->test") @@ -417,18 +393,10 @@ lazy val scaldingParquet = module("parquet").settings( lazy val scaldingParquetScroogeFixtures = module("parquet-scrooge-fixtures") .settings( scroogeThriftSourceFolder in Test := baseDirectory.value / "src/test/resources", - sourceGenerators in Test += Def.task { - val sources = (scroogeThriftSources in Test).value - val out = streams.value - // for some reason, sbt sometimes calls us multiple times, often with no source files. - if ((scroogeIsDirty in Test).value && sources.nonEmpty) { - out.log.info("Generating scrooge thrift for %s ...".format(sources.mkString(", "))) - ScroogeSBT.compile(out.log, sourceManaged.value, sources.toSet, Set(), Map(), "java", Set()) - } - (sourceManaged.value ** "*.java").get.toSeq - }.taskValue, + scroogeLanguages in Test := Seq("java", "scala"), libraryDependencies ++= Seq( - "com.twitter" %% "scrooge-serializer" % scroogeVersion % "provided", + "com.twitter" %% "scrooge-serializer" % scroogeVersion % "provided" + exclude("com.google.guava", "guava"), "commons-lang" % "commons-lang" % apacheCommonsVersion, // needed for HashCodeBuilder used in thriftjava "org.apache.thrift" % "libthrift" % thriftVersion ) @@ -443,7 +411,8 @@ lazy val scaldingParquetScrooge = module("parquet-scrooge") 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, + "com.twitter" %% "scrooge-serializer" % scroogeVersion + exclude("com.google.guava", "guava"), "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided", "com.novocode" % "junit-interface" % "0.11" % "test", "junit" % "junit" % junitVersion % "test" @@ -489,7 +458,7 @@ lazy val scaldingRepl = module("repl") import com.twitter.scalding.ReplImplicitContext._ """, libraryDependencies ++= Seq( - "jline" % "jline" % scalaVersion.value.take(4), + "jline" % "jline" % jlineVersion, "org.scala-lang" % "scala-compiler" % scalaVersion.value, "org.scala-lang" % "scala-reflect" % scalaVersion.value, "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided", @@ -514,7 +483,7 @@ lazy val scaldingRepl = module("repl") lazy val scaldingSerialization = module("serialization").settings( libraryDependencies ++= Seq( "org.scala-lang" % "scala-reflect" % scalaVersion.value - ) ++ (if(isScala210x(scalaVersion.value)) Seq("org.scalamacros" %% "quasiquotes" % "2.0.1") else Seq()), + ), addCompilerPlugin("org.scalamacros" % "paradise" % paradiseVersion cross CrossVersion.full) ) @@ -592,15 +561,16 @@ lazy val scaldingDb = module("db").settings( "org.scala-lang" % "scala-library" % scalaVersion.value, "org.scala-lang" % "scala-reflect" % scalaVersion.value, "com.twitter" %% "bijection-macros" % bijectionVersion - ) ++ (if(isScala210x(scalaVersion.value)) Seq("org.scalamacros" %% "quasiquotes" % "2.0.1") else Seq()), -addCompilerPlugin("org.scalamacros" % "paradise" % paradiseVersion cross CrossVersion.full) + ), + addCompilerPlugin("org.scalamacros" % "paradise" % paradiseVersion cross CrossVersion.full) ).dependsOn(scaldingCore) lazy val scaldingThriftMacrosFixtures = module("thrift-macros-fixtures") .settings( scroogeThriftSourceFolder in Test := baseDirectory.value / "src/test/resources", libraryDependencies ++= Seq( - "com.twitter" %% "scrooge-serializer" % scroogeVersion % "provided", + "com.twitter" %% "scrooge-serializer" % scroogeVersion % "provided" + exclude("com.google.guava", "guava"), "org.apache.thrift" % "libthrift" % thriftVersion ) ) @@ -611,7 +581,8 @@ lazy val scaldingThriftMacros = module("thrift-macros") "org.scala-lang" % "scala-reflect" % scalaVersion.value, "com.twitter" %% "bijection-macros" % bijectionVersion, "com.twitter" % "chill-thrift" % chillVersion % "test", - "com.twitter" %% "scrooge-serializer" % scroogeVersion % "provided", + "com.twitter" %% "scrooge-serializer" % scroogeVersion % "provided" + exclude("com.google.guava", "guava"), "org.apache.thrift" % "libthrift" % thriftVersion, "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "test", "org.apache.hadoop" % "hadoop-minicluster" % hadoopVersion % "test", @@ -622,7 +593,7 @@ lazy val scaldingThriftMacros = module("thrift-macros") "org.apache.hadoop" % "hadoop-hdfs" % hadoopVersion classifier "tests", "org.apache.hadoop" % "hadoop-common" % hadoopVersion classifier "tests", "org.apache.hadoop" % "hadoop-mapreduce-client-jobclient" % hadoopVersion classifier "tests" - ) ++ (if (isScala210x(scalaVersion.value)) Seq("org.scalamacros" %% "quasiquotes" % "2.0.1") else Seq()), + ), addCompilerPlugin("org.scalamacros" % "paradise" % paradiseVersion cross CrossVersion.full) ).dependsOn( scaldingCore, @@ -630,16 +601,8 @@ lazy val scaldingThriftMacros = module("thrift-macros") scaldingSerialization, scaldingThriftMacrosFixtures % "test->test") -/** - * Remove 2.10 projects from doc generation, as the macros used in the projects - * cause problems generating the documentation on scala 2.10. As the APIs for 2.10 - * and 2.11 are the same this has no effect on the resultant documentation, though - * it does mean that the scaladocs cannot be generated when the build is in 2.10 mode. - */ -def docsSourcesAndProjects(sv: String): (Boolean, Seq[ProjectReference]) = - CrossVersion.partialVersion(sv) match { - case Some((2, 10)) => (false, Nil) - case _ => (true, Seq( +def docsSourcesAndProjects(sv: String): Seq[ProjectReference] = + Seq( scaldingArgs, scaldingDate, scaldingCore @@ -655,8 +618,7 @@ def docsSourcesAndProjects(sv: String): (Boolean, Seq[ProjectReference]) = // maple, // scaldingSerialization, // scaldingThriftMacros - )) - } + ) lazy val docsMappingsAPIDir = settingKey[String]("Name of subdirectory in site target directory for api docs") @@ -682,7 +644,7 @@ lazy val docSettings = Seq( "white-color" -> "#FFFFFF"), autoAPIMappings := true, unidocProjectFilter in (ScalaUnidoc, unidoc) := - inProjects(docsSourcesAndProjects(scalaVersion.value)._2:_*), + inProjects(docsSourcesAndProjects(scalaVersion.value):_*), docsMappingsAPIDir := "api", addMappingsToSiteDir(mappings in (ScalaUnidoc, packageDoc), docsMappingsAPIDir), ghpagesNoJekyll := false, diff --git a/project/plugins.sbt b/project/plugins.sbt index 9cddd8c81a..5dacf1329c 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -11,12 +11,11 @@ addSbtPlugin("com.eed3si9n" % "sbt-unidoc" % "0.3.3") addSbtPlugin("com.fortysevendeg" % "sbt-microsites" % "0.3.3") addSbtPlugin("com.github.gseitz" % "sbt-release" % "1.0.0") addSbtPlugin("com.jsuereth" % "sbt-pgp" % "1.0.0") -addSbtPlugin("com.twitter" %% "scrooge-sbt-plugin" % "3.20.0") -addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.1.12") +addSbtPlugin("com.twitter" %% "scrooge-sbt-plugin" % "4.14.0") +addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.1.14") addSbtPlugin("com.typesafe.sbt" % "sbt-ghpages" % "0.5.4") addSbtPlugin("com.typesafe.sbt" % "sbt-git" % "0.6.2") addSbtPlugin("com.typesafe.sbt" % "sbt-scalariform" % "1.3.0") -addSbtPlugin("org.scoverage" % "sbt-scoverage" % "1.0.4") +addSbtPlugin("org.scoverage" % "sbt-scoverage" % "1.5.0") addSbtPlugin("org.xerial.sbt" % "sbt-sonatype" % "1.0") -addSbtPlugin("pl.project13.scala" % "sbt-jmh" % "0.2.17") -addSbtPlugin("org.brianmckenna" % "sbt-wartremover" % "0.14") +addSbtPlugin("org.wartremover" % "sbt-wartremover" % "2.0.2") diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala b/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala index f8fc8b645a..bdeab9e4dc 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala @@ -201,8 +201,14 @@ sealed trait Execution[+T] extends java.io.Serializable { self: Product => Zipped(this, that) override val hashCode: Int = ScalaRunTime._hashCode(self) - override def equals(other: Any): Boolean = - hashCode == other.hashCode && ScalaRunTime._equals(self, other) + + override def equals(other: Any): Boolean = { + other match { + case p: Product if self.productArity == p.productArity => + self.hashCode == p.hashCode && (self.productIterator sameElements p.productIterator) + case _ => false + } + } } /** diff --git a/scalding-core/src/main/scala/com/twitter/scalding/GroupBuilder.scala b/scalding-core/src/main/scala/com/twitter/scalding/GroupBuilder.scala index 41b225dc36..818e5f07b0 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/GroupBuilder.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/GroupBuilder.scala @@ -289,6 +289,7 @@ class GroupBuilder(val groupFields: Fields) extends FoldOperations[GroupBuilder] gb } + @SuppressWarnings(Array("org.wartremover.warts.OptionPartial")) def schedule(name: String, pipe: Pipe): Pipe = { val maybeProjectedPipe = projectFields.map { pipe.project(_) }.getOrElse(pipe) groupMode match { @@ -381,7 +382,7 @@ class ScanLeftIterator[T, U](it: Iterator[T], init: U, fn: (U, T) => U) extends protected var prev: Option[U] = None def hasNext: Boolean = { prev.isEmpty || it.hasNext } // Don't use pattern matching in a performance-critical section - @SuppressWarnings(Array("org.brianmckenna.wartremover.warts.OptionPartial")) + @SuppressWarnings(Array("org.wartremover.warts.OptionPartial")) def next = { prev = prev.map { fn(_, it.next) } .orElse(Some(init)) 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 7e280a7aea..70fb598846 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/JobTest.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/JobTest.scala @@ -108,7 +108,7 @@ class JobTest(cons: (Args) => Job) { sourceBuffer(s, iTuple) // This use of `_.get` is probably safe, but difficult to prove correct - @SuppressWarnings(Array("org.brianmckenna.wartremover.warts.OptionPartial")) + @SuppressWarnings(Array("org.wartremover.warts.OptionPartial")) def sink[A](s: Source)(op: Buffer[A] => Unit)(implicit conv: TupleConverter[A]) = { if (sourceMap(s).isEmpty) { // if s is also used as a source, we shouldn't reset its buffer @@ -200,6 +200,7 @@ class JobTest(cons: (Args) => Job) { } @tailrec + @SuppressWarnings(Array("org.wartremover.warts.OptionPartial")) private final def runJob(job: Job, runNext: Boolean): Unit = { // Disable automatic cascading update System.setProperty("cascading.update.skip", "true") diff --git a/scalding-core/src/main/scala/com/twitter/scalding/LineNumber.scala b/scalding-core/src/main/scala/com/twitter/scalding/LineNumber.scala index 85ee1dc93f..58f44e5929 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/LineNumber.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/LineNumber.scala @@ -15,6 +15,8 @@ limitations under the License. */ package com.twitter.scalding +import org.slf4j.{ Logger, LoggerFactory } + object LineNumber { /** * depth 0 means the StackTraceElement for the caller @@ -30,6 +32,8 @@ object LineNumber { def ignorePath(classPrefixes: Set[String]): Option[StackTraceElement] = ignorePaths(classPrefixes, Thread.currentThread().getStackTrace) + private val LOG: Logger = LoggerFactory.getLogger(LineNumber.getClass) + private[this] def ignorePaths(classPrefixes: Set[String], stack: Seq[StackTraceElement]): Option[StackTraceElement] = stack.drop(2) .dropWhile { ste => @@ -72,8 +76,15 @@ object LineNumber { .iterator .filter { se => se.getClassName.startsWith(scaldingPrefix) } .filter { se => - val cls = Class.forName(se.getClassName) - jobClass.isAssignableFrom(cls) + try { + val cls = Class.forName(se.getClassName) + jobClass.isAssignableFrom(cls) + } catch { + // skip classes that we don't find. We seem to run into this for some lambdas on Scala 2.12 in travis + case cnf: ClassNotFoundException => + LOG.warn(s"Skipping $se.getClassName as we can't find the class") + false + } }) scaldingJobCaller 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..9103e7dcf5 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Mode.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Mode.scala @@ -247,7 +247,7 @@ case class HadoopTest(@transient conf: Configuration, * functions, and those functions have been documented accordingly to * warn about this invariant. */ - @SuppressWarnings(Array("org.brianmckenna.wartremover.warts.OptionPartial")) // Get the buffer for the given source, and empty it: + @SuppressWarnings(Array("org.wartremover.warts.OptionPartial")) // Get the buffer for the given source, and empty it: val buf = buffers(src).get buf.clear() // Now fill up this buffer with the content of the file diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Operations.scala b/scalding-core/src/main/scala/com/twitter/scalding/Operations.scala index ea131a0a6e..6f95c95077 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Operations.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Operations.scala @@ -155,7 +155,7 @@ package com.twitter.scalding { // Use iterator and while for optimal performance (avoid closures/fn calls) if (evicted.isDefined) { // Don't use pattern matching in performance-critical code - @SuppressWarnings(Array("org.brianmckenna.wartremover.warts.OptionPartial")) + @SuppressWarnings(Array("org.wartremover.warts.OptionPartial")) val it = evicted.get.iterator val tecol = functionCall.getOutputCollector while (it.hasNext) { @@ -213,7 +213,7 @@ package com.twitter.scalding { } // Don't use pattern matching in a performance-critical section - @SuppressWarnings(Array("org.brianmckenna.wartremover.warts.OptionPartial")) + @SuppressWarnings(Array("org.wartremover.warts.OptionPartial")) @inline private def add(evicted: Option[Map[K, V]], functionCall: FunctionCall[MapsideCache[K, V]]): Unit = { // Use iterator and while for optimal performance (avoid closures/fn calls) @@ -241,7 +241,7 @@ package com.twitter.scalding { } // Don't use pattern matching in a performance-critical section - @SuppressWarnings(Array("org.brianmckenna.wartremover.warts.OptionPartial")) + @SuppressWarnings(Array("org.wartremover.warts.OptionPartial")) private[this] def mergeTraversableOnce[K, V: Semigroup](items: TraversableOnce[(K, V)]): Map[K, V] = { val mutable = scala.collection.mutable.OpenHashMap[K, V]() // Scala's OpenHashMap seems faster than Java and Scala's HashMap Impl's val innerIter = items.toIterator @@ -315,7 +315,7 @@ package com.twitter.scalding { def flush = summingCache.flush // Don't use pattern matching in performance-critical code - @SuppressWarnings(Array("org.brianmckenna.wartremover.warts.OptionPartial")) + @SuppressWarnings(Array("org.wartremover.warts.OptionPartial")) def put(key: K, value: V): Option[Map[K, V]] = { val (curHits, evicted) = summingCache.putWithHits(Map(key -> value)) misses.increment(1 - curHits) @@ -327,7 +327,7 @@ package com.twitter.scalding { } // Don't use pattern matching in a performance-critical section - @SuppressWarnings(Array("org.brianmckenna.wartremover.warts.OptionPartial")) + @SuppressWarnings(Array("org.wartremover.warts.OptionPartial")) def putAll(kvs: Map[K, V]): Option[Map[K, V]] = { val (curHits, evicted) = summingCache.putWithHits(kvs) misses.increment(kvs.size - curHits) @@ -350,7 +350,7 @@ package com.twitter.scalding { def flush = adaptiveCache.flush // Don't use pattern matching in performance-critical code - @SuppressWarnings(Array("org.brianmckenna.wartremover.warts.OptionPartial")) + @SuppressWarnings(Array("org.wartremover.warts.OptionPartial")) def put(key: K, value: V) = { val (stats, evicted) = adaptiveCache.putWithStats(Map(key -> value)) misses.increment(1 - stats.hits) @@ -366,7 +366,7 @@ package com.twitter.scalding { } // Don't use pattern matching in a performance-critical section - @SuppressWarnings(Array("org.brianmckenna.wartremover.warts.OptionPartial")) + @SuppressWarnings(Array("org.wartremover.warts.OptionPartial")) def putAll(kvs: Map[K, V]): Option[Map[K, V]] = { val (stats, evicted) = adaptiveCache.putWithStats(kvs) misses.increment(kvs.size - stats.hits) 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 db3426a9ff..75238144d7 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/TestTapFactory.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/TestTapFactory.scala @@ -59,6 +59,7 @@ class TestTapFactory(src: Source, sinkMode: SinkMode) extends Serializable { def hdfsScheme: Option[Scheme[JobConf, RecordReader[_, _], OutputCollector[_, _], _, _]] = None + @SuppressWarnings(Array("org.wartremover.warts.OptionPartial")) def createTap(readOrWrite: AccessMode)(implicit mode: Mode): Tap[_, _, _] = { mode match { case Test(buffers) => { diff --git a/scalding-core/src/main/scala/com/twitter/scalding/TuplePacker.scala b/scalding-core/src/main/scala/com/twitter/scalding/TuplePacker.scala index 29b4709f3e..43c539716e 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/TuplePacker.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/TuplePacker.scala @@ -67,7 +67,7 @@ class ReflectionTupleConverter[T](fields: Fields)(implicit m: Manifest[T]) exten /* The `_.get` is safe because of the `_.isEmpty` check. ScalaTest does not * seem to support a more type safe way of doing this. */ - @SuppressWarnings(Array("org.brianmckenna.wartremover.warts.OptionPartial")) + @SuppressWarnings(Array("org.wartremover.warts.OptionPartial")) def validate(): Unit = { //We can't touch setters because that shouldn't be accessed until map/reduce side, not //on submitter. diff --git a/scalding-core/src/main/scala/com/twitter/scalding/macros/impl/TypeDescriptorProviderImpl.scala b/scalding-core/src/main/scala/com/twitter/scalding/macros/impl/TypeDescriptorProviderImpl.scala index a6fdbf47be..3dfd47e272 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/macros/impl/TypeDescriptorProviderImpl.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/macros/impl/TypeDescriptorProviderImpl.scala @@ -55,6 +55,7 @@ object TypeDescriptorProviderImpl { .map(_.returnType.asSeenFrom(t, t.typeSymbol.asClass)) .toList + @SuppressWarnings(Array("org.wartremover.warts.OptionPartial")) def go(t: Type, offset: Int): (Int, Option[Int]) = { val thisColumn = (offset + 1, Some(offset)) t match { diff --git a/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix2.scala b/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix2.scala index 67d71b43b0..afb3aa1d0c 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix2.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix2.scala @@ -629,7 +629,7 @@ object Matrix2 { /* The only case where `product` will be `None` is if the result is an * intermediate matrix (like `OneC`). This is not yet forbidden in the types. */ - @SuppressWarnings(Array("org.brianmckenna.wartremover.warts.OptionPartial")) + @SuppressWarnings(Array("org.wartremover.warts.OptionPartial")) def generatePlan(i: Int, j: Int): Matrix2[Any, Any, V] = { if (i == j) p(i) else { diff --git a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala index 3a821e43a0..7dd2b78a1a 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala @@ -5,7 +5,7 @@ import com.twitter.algebird.Monoid import com.twitter.scalding.{ StringUtility, Config } import cascading.tap.{ Tap, CompositeTap } import org.apache.hadoop.fs.Path -import org.apache.hadoop.mapred.{FileInputFormat, JobConf} +import org.apache.hadoop.mapred.{ FileInputFormat, JobConf } import org.slf4j.LoggerFactory import java.util.{ List => JList } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/WrappedSerialization.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/WrappedSerialization.scala index ebeb7d67b7..abd2ba0cbd 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/WrappedSerialization.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/WrappedSerialization.scala @@ -37,7 +37,7 @@ class WrappedSerialization[T] extends HSerialization[T] with Configurable { /* This use of `_.get` can't be fixed since this is constrained by * Hadoop's `Configurable` interface. */ - @SuppressWarnings(Array("org.brianmckenna.wartremover.warts.OptionPartial")) + @SuppressWarnings(Array("org.wartremover.warts.OptionPartial")) override def getConf: Configuration = conf.get override def setConf(config: Configuration): Unit = { conf = Some(config) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/CoGrouped.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/CoGrouped.scala index 1694756a3e..dd9392c6e7 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/CoGrouped.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/CoGrouped.scala @@ -334,7 +334,7 @@ abstract class CoGroupedJoiner[K](inputSize: Int, override def getIterator(jc: JoinerClosure) = { val iters = (0 until distinctSize).map { jc.getIterator(_).asScala.buffered } // This use of `_.get` is safe, but difficult to prove in the types. - @SuppressWarnings(Array("org.brianmckenna.wartremover.warts.OptionPartial")) + @SuppressWarnings(Array("org.wartremover.warts.OptionPartial")) val keyTuple = iters .collectFirst { case iter if iter.nonEmpty => iter.head } .get // One of these must have a key 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 328a168ba8..5e11853df4 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 @@ -848,6 +848,7 @@ final case class IterablePipe[T](iterable: Iterable[T]) extends TypedPipe[T] { Semigroup.sumOption[U](iterable).map(LiteralValue(_)) .getOrElse(EmptyValue) + @SuppressWarnings(Array("org.wartremover.warts.OptionPartial")) override def sumByLocalKeys[K, V](implicit ev: T <:< (K, V), sg: Semigroup[V]) = { val kvit = raiseTo[(K, V)] match { case IterablePipe(kviter) => kviter diff --git a/scalding-db/src/test/scala/com/twitter/scalding/db/macros/MacrosUnitTests.scala b/scalding-db/src/test/scala/com/twitter/scalding/db/macros/MacrosUnitTests.scala index f46908d55f..637b9dde7b 100644 --- a/scalding-db/src/test/scala/com/twitter/scalding/db/macros/MacrosUnitTests.scala +++ b/scalding-db/src/test/scala/com/twitter/scalding/db/macros/MacrosUnitTests.scala @@ -182,6 +182,7 @@ class JdbcMacroUnitTests extends WordSpec with Matchers with MockitoSugar { when(rs.getString("gender")) thenReturn ("F") assert(columnDef.resultSetExtractor.toCaseClass(rs, typeDesc.converter) == User(123, "alice", Some(26), "F")) + () // Need this till: https://github.com/scalatest/scalatest/issues/1107 } "Produces the ColumnDefinition for nested case class " should { @@ -208,6 +209,7 @@ class JdbcMacroUnitTests extends WordSpec with Matchers with MockitoSugar { when(rs.getString("gender")) thenReturn ("F") assert(columnDef.resultSetExtractor.toCaseClass(rs, typeDesc.converter) == User2(123, "alice", Demographics(Some(26), "F"))) + () // Need this till: https://github.com/scalatest/scalatest/issues/1107 } "Produces the DBTypeDescriptor" should { @@ -223,7 +225,7 @@ class JdbcMacroUnitTests extends WordSpec with Matchers with MockitoSugar { ColumnDefinition(VARCHAR, ColumnName("gender"), NotNullable, Some(22), Some("male"))) assert(DBMacro.toDBTypeDescriptor[User].columnDefn.columns.toList === expectedColumns) - + () // Need this till: https://github.com/scalatest/scalatest/issues/1107 } "Big Jdbc Test" should { @@ -319,6 +321,7 @@ class JdbcMacroUnitTests extends WordSpec with Matchers with MockitoSugar { new Date(1111L), new Date(1112L), Some(1113L))) + () // Need this till: https://github.com/scalatest/scalatest/issues/1107 } "TupleConverter for Date" should { @@ -331,6 +334,7 @@ class JdbcMacroUnitTests extends WordSpec with Matchers with MockitoSugar { t.set(1, date1) t.set(2, date2) assert(CaseClassWithDate(99L, date1, date2) == converter(new TupleEntry(t))) + () // Need this till: https://github.com/scalatest/scalatest/issues/1107 } "ResultSetExtractor validation for nullable columns" should { @@ -355,6 +359,7 @@ class JdbcMacroUnitTests extends WordSpec with Matchers with MockitoSugar { when(rsmd.isNullable(7)) thenReturn (ResultSetMetaData.columnNullable) assert(columnDef.resultSetExtractor.validate(rsmd).isSuccess) + () // Need this till: https://github.com/scalatest/scalatest/issues/1107 } "ResultSetExtractor when nullable values are not null" should { @@ -379,6 +384,7 @@ class JdbcMacroUnitTests extends WordSpec with Matchers with MockitoSugar { assert(columnDef.resultSetExtractor.toCaseClass(rs, typeDesc.converter) == CaseClassWithOptions(Some(26), Some("alice"), Some(new Date(1111L)), Some(true), Some(2), Some(2000L), Some(2.2))) + () // Need this till: https://github.com/scalatest/scalatest/issues/1107 } "ResultSetExtractor when null values" should { @@ -402,6 +408,7 @@ class JdbcMacroUnitTests extends WordSpec with Matchers with MockitoSugar { assert(columnDef.resultSetExtractor.toCaseClass(rs, typeDesc.converter) == CaseClassWithOptions(None, None, None, None, None, None, None)) + () // Need this till: https://github.com/scalatest/scalatest/issues/1107 } "ResultSetExtractor for DB schema type mismatch" in { diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/platform/PlatformExecutionTest.scala b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/platform/PlatformExecutionTest.scala index b7c6044b25..42c92f0119 100644 --- a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/platform/PlatformExecutionTest.scala +++ b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/platform/PlatformExecutionTest.scala @@ -1,7 +1,7 @@ package com.twitter.scalding.platform -import com.twitter.scalding.{Config, Execution, TypedPipe, TypedTsv} -import org.scalatest.{Matchers, WordSpec} +import com.twitter.scalding.{ Config, Execution, TypedPipe, TypedTsv } +import org.scalatest.{ Matchers, WordSpec } import scala.io.Source object InAndOutExecution extends Function[Config, Execution[Unit]] { 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 b52b0cd0f7..c2c42584cf 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 @@ -297,8 +297,13 @@ object OrderedSerializationTest { case class NestedCaseClass(day: RichDate, key: (String, String)) -class ComplexJob(input: List[NestedCaseClass], args: Args) extends Job(args) { +// Need to define this in a separate companion object to work around Scala 2.12 compile issues +object OrderedSerializationImplicitDefs { implicit def primitiveOrderedBufferSupplier[T]: OrderedSerialization[T] = macro com.twitter.scalding.serialization.macros.impl.OrderedSerializationProviderImpl[T] +} + +class ComplexJob(input: List[NestedCaseClass], args: Args) extends Job(args) { + import OrderedSerializationImplicitDefs._ val ds1 = TypedPipe.from(input).map(_ -> 1L).group.sorted.mapValueStream(_.map(_ * 2)).toTypedPipe.group @@ -316,7 +321,7 @@ class ComplexJob(input: List[NestedCaseClass], args: Args) extends Job(args) { } class ComplexJob2(input: List[NestedCaseClass], args: Args) extends Job(args) { - implicit def primitiveOrderedBufferSupplier[T]: OrderedSerialization[T] = macro com.twitter.scalding.serialization.macros.impl.OrderedSerializationProviderImpl[T] + import OrderedSerializationImplicitDefs._ val ds1 = TypedPipe.from(input).map(_ -> (1L, "asfg")) @@ -516,12 +521,8 @@ class PlatformTest extends WordSpec with Matchers with HadoopSharedPlatformTest val steps = flow.getFlowSteps.asScala steps should have size 1 val firstStep = steps.headOption.map(_.getConfig.get(Config.StepDescriptions)).getOrElse("") - val lines = List(16, 18, 19, 22, 23).map { i => - s"com.twitter.scalding.platform.TypedPipeJoinWithDescriptionJob.(TestJobsWithDescriptions.scala:$i" - } firstStep should include ("leftJoin") firstStep should include ("hashJoin") - lines.foreach { l => firstStep should include (l) } steps.map(_.getConfig.get(Config.StepDescriptions)).foreach(s => info(s)) } .run() 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 1d3678f652..3b9d2779b9 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 @@ -2,14 +2,14 @@ package com.twitter.scalding.hraven.reducer_estimation import java.io.IOException import cascading.flow.FlowStep -import com.twitter.hraven.{Constants, CounterMap, Flow, HistoryFileType, JobDetails} +import com.twitter.hraven.{ Constants, CounterMap, Flow, HistoryFileType, JobDetails } import com.twitter.hraven.rest.client.HRavenRestClient import com.twitter.scalding.reducer_estimation._ import org.apache.hadoop.mapred.JobConf import org.slf4j.LoggerFactory import scala.collection.JavaConverters._ -import com.twitter.hraven.JobDescFactory.{JOBTRACKER_KEY, RESOURCE_MANAGER_KEY} -import scala.util.{Failure, Success, Try} +import com.twitter.hraven.JobDescFactory.{ JOBTRACKER_KEY, RESOURCE_MANAGER_KEY } +import scala.util.{ Failure, Success, Try } object HRavenClient { import HRavenHistoryService.jobConfToRichConfig diff --git a/scalding-parquet-scrooge-fixtures/src/test/resources/test.thrift b/scalding-parquet-scrooge-fixtures/src/test/resources/test.thrift index b4a19eef83..d76cd848b1 100644 --- a/scalding-parquet-scrooge-fixtures/src/test/resources/test.thrift +++ b/scalding-parquet-scrooge-fixtures/src/test/resources/test.thrift @@ -86,14 +86,6 @@ struct StructWithReorderedOptionalFields { 1: optional i32 fieldOne, } -struct TestPersonWithRequiredPhone { - 1: required Name name, - 2: optional i32 age, - 3: Address address, - 4: string info, - 5: required Phone phone -} - struct StructWithIndexStartsFrom4 { 6: required Phone phone } diff --git a/scalding-repl/src/main/scala-2.12/com/twitter/scalding/ILoopCompat.scala b/scalding-repl/src/main/scala-2.12/com/twitter/scalding/ILoopCompat.scala new file mode 100644 index 0000000000..1d44b67093 --- /dev/null +++ b/scalding-repl/src/main/scala-2.12/com/twitter/scalding/ILoopCompat.scala @@ -0,0 +1,11 @@ +package com.twitter.scalding + +import java.io.BufferedReader + +import scala.tools.nsc.interpreter.ILoop +import scala.tools.nsc.interpreter.JPrintWriter + +class ILoopCompat(in: Option[BufferedReader], out: JPrintWriter) + extends ILoop(in, out) { + def addThunk(f: => Unit): Unit = intp.initialize(f) +} diff --git a/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/Boxed.scala b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/Boxed.scala index 6cce123083..1a70fe1916 100644 --- a/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/Boxed.scala +++ b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/Boxed.scala @@ -542,7 +542,9 @@ case class BoxedOrderedSerialization[K](box: K => Boxed[K], override def dynamicSize(k: Boxed[K]) = ord.dynamicSize(k.get) } -object Boxed { +// Moving boxed lamdbas into a new object and breaking them up into 2 lists +// to get around this Scala 2.12 bug: https://issues.scala-lang.org/browse/SI-10232 +object BoxedLambdas { /* You might wonder: "Why not do something a little more type-safe like this?" * * private[this] def f0[K](t: K) = new Boxed0(t) @@ -560,7 +562,7 @@ object Boxed { * using `Any` here instead of parametrizing everything over a type parameter * `K`. */ - private[this] val allBoxes = List( + private[serialization] val boxes1 = List( ({ t: Any => new Boxed0(t) }, classOf[Boxed0[Any]]), ({ t: Any => new Boxed1(t) }, classOf[Boxed1[Any]]), ({ t: Any => new Boxed2(t) }, classOf[Boxed2[Any]]), @@ -685,7 +687,9 @@ object Boxed { ({ t: Any => new Boxed121(t) }, classOf[Boxed121[Any]]), ({ t: Any => new Boxed122(t) }, classOf[Boxed122[Any]]), ({ t: Any => new Boxed123(t) }, classOf[Boxed123[Any]]), - ({ t: Any => new Boxed124(t) }, classOf[Boxed124[Any]]), + ({ t: Any => new Boxed124(t) }, classOf[Boxed124[Any]])) + + private[serialization] val boxes2 = List( ({ t: Any => new Boxed125(t) }, classOf[Boxed125[Any]]), ({ t: Any => new Boxed126(t) }, classOf[Boxed126[Any]]), ({ t: Any => new Boxed127(t) }, classOf[Boxed127[Any]]), @@ -812,6 +816,12 @@ object Boxed { ({ t: Any => new Boxed248(t) }, classOf[Boxed248[Any]]), ({ t: Any => new Boxed249(t) }, classOf[Boxed249[Any]]), ({ t: Any => new Boxed250(t) }, classOf[Boxed250[Any]])) +} + +object Boxed { + import BoxedLambdas._ + + private[this] val allBoxes = boxes1 ++ boxes2 private[this] val boxes: AtomicReference[List[(Any => Boxed[Any], Class[_ <: Boxed[Any]])]] = new AtomicReference(allBoxes) diff --git a/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/Writer.scala b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/Writer.scala index 0e5f267eab..c99a4134dc 100644 --- a/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/Writer.scala +++ b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/Writer.scala @@ -70,7 +70,7 @@ object Writer { implicit def option[T: Writer]: Writer[Option[T]] = new Writer[Option[T]] { val w = implicitly[Writer[T]] // Don't use pattern matching in a performance-critical section - @SuppressWarnings(Array("org.brianmckenna.wartremover.warts.OptionPartial")) + @SuppressWarnings(Array("org.wartremover.warts.OptionPartial")) def write(os: OutputStream, t: Option[T]) = if (t.isDefined) { os.write(1: Byte) diff --git a/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/SealedTraitLike.scala b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/SealedTraitLike.scala index f8e49ee527..f4f75e38f3 100644 --- a/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/SealedTraitLike.scala +++ b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/SealedTraitLike.scala @@ -34,7 +34,7 @@ object SealedTraitLike { * and finally a means to compare two instances of this type. */ // This `_.get` could be removed by switching `subData` to a non-empty list type - @SuppressWarnings(Array("org.brianmckenna.wartremover.warts.OptionPartial")) + @SuppressWarnings(Array("org.wartremover.warts.OptionPartial")) def compareBinary(c: Context)(inputStreamA: c.TermName, inputStreamB: c.TermName)(subData: List[(Int, c.Type, TreeOrderedBuf[c.type])]): c.Tree = { import c.universe._ def freshT(id: String) = newTermName(c.fresh(id)) @@ -80,7 +80,7 @@ object SealedTraitLike { } // This `_.get` could be removed by switching `subData` to a non-empty list type - @SuppressWarnings(Array("org.brianmckenna.wartremover.warts.OptionPartial")) + @SuppressWarnings(Array("org.wartremover.warts.OptionPartial")) def hash(c: Context)(element: c.TermName)(subData: List[(Int, c.Type, TreeOrderedBuf[c.type])]): c.Tree = { import c.universe._ def freshT(id: String) = newTermName(c.fresh(id)) @@ -115,7 +115,7 @@ object SealedTraitLike { } // This `_.get` could be removed by switching `subData` to a non-empty list type - @SuppressWarnings(Array("org.brianmckenna.wartremover.warts.OptionPartial")) + @SuppressWarnings(Array("org.wartremover.warts.OptionPartial")) def put(c: Context)(inputStream: c.TermName, element: c.TermName)(subData: List[(Int, c.Type, TreeOrderedBuf[c.type])]): c.Tree = { import c.universe._ def freshT(id: String) = newTermName(c.fresh(id)) @@ -149,7 +149,7 @@ object SealedTraitLike { } // This `_.get` could be removed by switching `subData` to a non-empty list type - @SuppressWarnings(Array("org.brianmckenna.wartremover.warts.OptionPartial")) + @SuppressWarnings(Array("org.wartremover.warts.OptionPartial")) def length(c: Context)(element: c.Tree)(subData: List[(Int, c.Type, TreeOrderedBuf[c.type])]): CompileTimeLengthTypes[c.type] = { import CompileTimeLengthTypes._ import c.universe._ @@ -208,7 +208,7 @@ object SealedTraitLike { } // This `_.get` could be removed by switching `subData` to a non-empty list type - @SuppressWarnings(Array("org.brianmckenna.wartremover.warts.OptionPartial")) + @SuppressWarnings(Array("org.wartremover.warts.OptionPartial")) def get(c: Context)(inputStream: c.TermName)(subData: List[(Int, c.Type, TreeOrderedBuf[c.type])]): c.Tree = { import c.universe._ def freshT(id: String) = newTermName(c.fresh(id)) @@ -246,7 +246,7 @@ object SealedTraitLike { } // This `_.get` could be removed by switching `subData` to a non-empty list type - @SuppressWarnings(Array("org.brianmckenna.wartremover.warts.OptionPartial")) + @SuppressWarnings(Array("org.wartremover.warts.OptionPartial")) def compare(c: Context)(cmpType: c.Type, elementA: c.TermName, elementB: c.TermName)(subData: List[(Int, c.Type, TreeOrderedBuf[c.type])]): c.Tree = { import c.universe._ diff --git a/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/ProductOrderedBuf.scala b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/ProductOrderedBuf.scala index 604331dca2..d97c7be71b 100644 --- a/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/ProductOrderedBuf.scala +++ b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/ProductOrderedBuf.scala @@ -55,7 +55,7 @@ object ProductOrderedBuf { // The `_.get` is safe since it's always preceded by a matching // `_.isDefined` check in `validType` - @SuppressWarnings(Array("org.brianmckenna.wartremover.warts.OptionPartial")) + @SuppressWarnings(Array("org.wartremover.warts.OptionPartial")) def symbolFor(subType: Type): Type = { val superType = validTypes.find{ t => subType.erasure <:< t }.get subType.baseType(superType.typeSymbol) diff --git a/scalding-serialization/src/test/scala/com/twitter/scalding/serialization/macros/MacroOrderingProperties.scala b/scalding-serialization/src/test/scala/com/twitter/scalding/serialization/macros/MacroOrderingProperties.scala index cba5dfc59e..6cc94ce61d 100644 --- a/scalding-serialization/src/test/scala/com/twitter/scalding/serialization/macros/MacroOrderingProperties.scala +++ b/scalding-serialization/src/test/scala/com/twitter/scalding/serialization/macros/MacroOrderingProperties.scala @@ -23,7 +23,7 @@ import com.twitter.scalding.serialization.{ JavaStreamEnrichments, Law, Law1, La import org.scalacheck.Arbitrary.{ arbitrary => arb } import org.scalacheck.{ Arbitrary, Gen, Prop } import org.scalatest.prop.{ Checkers, PropertyChecks } -import org.scalatest.{ FunSuite, ShouldMatchers } +import org.scalatest.{ FunSuite, Matchers } import scala.collection.immutable.Queue import scala.language.experimental.macros @@ -202,7 +202,7 @@ object Container { type SetAlias = Set[Double] case class InnerCaseClass(e: SetAlias) } -class MacroOrderingProperties extends FunSuite with PropertyChecks with ShouldMatchers with LowerPriorityImplicit { +class MacroOrderingProperties extends FunSuite with PropertyChecks with Matchers with LowerPriorityImplicit { type SetAlias = Set[Double] import ByteBufferArb._ diff --git a/scalding-serialization/src/test/scala/com/twitter/scalding/serialization/macros/TraversableHelperLaws.scala b/scalding-serialization/src/test/scala/com/twitter/scalding/serialization/macros/TraversableHelperLaws.scala index 922c74f51f..cddbc900d6 100644 --- a/scalding-serialization/src/test/scala/com/twitter/scalding/serialization/macros/TraversableHelperLaws.scala +++ b/scalding-serialization/src/test/scala/com/twitter/scalding/serialization/macros/TraversableHelperLaws.scala @@ -16,13 +16,13 @@ limitations under the License. package com.twitter.scalding.serialization.macros -import org.scalatest.{ FunSuite, ShouldMatchers } +import org.scalatest.{ FunSuite, Matchers } import org.scalatest.prop.Checkers import org.scalatest.prop.PropertyChecks import impl.ordered_serialization.runtime_helpers.TraversableHelpers._ -class TraversableHelperLaws extends FunSuite with PropertyChecks with ShouldMatchers { +class TraversableHelperLaws extends FunSuite with PropertyChecks with Matchers { test("Iterator ordering should be Iterable ordering") { forAll { (l1: List[Int], l2: List[Int]) => assert(iteratorCompare[Int](l1.iterator, l2.iterator) === diff --git a/scalding-thrift-macros/src/main/scala/com/twitter/scalding/thrift/macros/impl/ordered_serialization/UnionLike.scala b/scalding-thrift-macros/src/main/scala/com/twitter/scalding/thrift/macros/impl/ordered_serialization/UnionLike.scala index ae758e52cb..55c0ac7a57 100644 --- a/scalding-thrift-macros/src/main/scala/com/twitter/scalding/thrift/macros/impl/ordered_serialization/UnionLike.scala +++ b/scalding-thrift-macros/src/main/scala/com/twitter/scalding/thrift/macros/impl/ordered_serialization/UnionLike.scala @@ -25,7 +25,7 @@ import com.twitter.scalding.serialization.macros.impl.ordered_serialization._ object UnionLike { // This `_.get` could be removed by switching `subData` to a non-empty list type - @SuppressWarnings(Array("org.brianmckenna.wartremover.warts.OptionPartial")) + @SuppressWarnings(Array("org.wartremover.warts.OptionPartial")) def compareBinary(c: Context)(inputStreamA: c.TermName, inputStreamB: c.TermName)(subData: List[(Int, c.Type, Option[TreeOrderedBuf[c.type]])]): c.Tree = { import c.universe._ def freshT(id: String) = newTermName(c.fresh(id)) @@ -72,7 +72,7 @@ object UnionLike { } // This `_.get` could be removed by switching `subData` to a non-empty list type - @SuppressWarnings(Array("org.brianmckenna.wartremover.warts.OptionPartial")) + @SuppressWarnings(Array("org.wartremover.warts.OptionPartial")) def hash(c: Context)(element: c.TermName)(subData: List[(Int, c.Type, Option[TreeOrderedBuf[c.type]])]): c.Tree = { import c.universe._ def freshT(id: String) = newTermName(c.fresh(id)) @@ -110,7 +110,7 @@ object UnionLike { } // This `_.get` could be removed by switching `subData` to a non-empty list type - @SuppressWarnings(Array("org.brianmckenna.wartremover.warts.OptionPartial")) + @SuppressWarnings(Array("org.wartremover.warts.OptionPartial")) def put(c: Context)(inputStream: c.TermName, element: c.TermName)(subData: List[(Int, c.Type, Option[TreeOrderedBuf[c.type]])]): c.Tree = { import c.universe._ def freshT(id: String) = newTermName(c.fresh(id)) @@ -146,7 +146,7 @@ object UnionLike { } // This `_.get` could be removed by switching `subData` to a non-empty list type - @SuppressWarnings(Array("org.brianmckenna.wartremover.warts.OptionPartial")) + @SuppressWarnings(Array("org.wartremover.warts.OptionPartial")) def length(c: Context)(element: c.Tree)(subData: List[(Int, c.Type, Option[TreeOrderedBuf[c.type]])]): CompileTimeLengthTypes[c.type] = { import CompileTimeLengthTypes._ import c.universe._ @@ -205,7 +205,7 @@ object UnionLike { } // This `_.get` could be removed by switching `subData` to a non-empty list type - @SuppressWarnings(Array("org.brianmckenna.wartremover.warts.OptionPartial")) + @SuppressWarnings(Array("org.wartremover.warts.OptionPartial")) def get(c: Context)(inputStream: c.TermName)(subData: List[(Int, c.Type, Option[TreeOrderedBuf[c.type]])]): c.Tree = { import c.universe._ def freshT(id: String) = newTermName(c.fresh(id)) @@ -249,7 +249,7 @@ object UnionLike { } // This `_.get` could be removed by switching `subData` to a non-empty list type - @SuppressWarnings(Array("org.brianmckenna.wartremover.warts.OptionPartial")) + @SuppressWarnings(Array("org.wartremover.warts.OptionPartial")) def compare(c: Context)(cmpType: c.Type, elementA: c.TermName, elementB: c.TermName)(subData: List[(Int, c.Type, Option[TreeOrderedBuf[c.type]])]): c.Tree = { import c.universe._ diff --git a/scalding-thrift-macros/src/test/scala/com/twitter/scalding/thrift/macros/ScroogeGenerators.scala b/scalding-thrift-macros/src/test/scala/com/twitter/scalding/thrift/macros/ScroogeGenerators.scala index 02dbdfdec5..6ffec9da59 100644 --- a/scalding-thrift-macros/src/test/scala/com/twitter/scalding/thrift/macros/ScroogeGenerators.scala +++ b/scalding-thrift-macros/src/test/scala/com/twitter/scalding/thrift/macros/ScroogeGenerators.scala @@ -4,6 +4,8 @@ import com.twitter.scalding.thrift.macros.scalathrift._ import org.scalacheck.{ Arbitrary, Gen, Prop } import org.scalacheck.Arbitrary.{ arbitrary => arb } import java.nio.ByteBuffer +import org.scalacheck.Gen.Parameters +import org.scalacheck.rng.Seed private object Perturbers { def perturb(t0: TestStruct, t1: TestStruct, i: Int): TestStruct = { @@ -88,11 +90,8 @@ object ScroogeGenerators { def dataProvider[T: Arbitrary](i: Int): T = { @annotation.tailrec def g(innerI: Int, loops: Int): T = { - val p = new org.scalacheck.Gen.Parameters { - override val size = 2 - override val rng = new scala.util.Random(innerI) - } - implicitly[Arbitrary[T]].arbitrary(p) match { + val p = Parameters.default.withSize(2) + implicitly[Arbitrary[T]].arbitrary(p, Seed(innerI)) match { case Some(s) => s case None if loops < 5 => g(innerI + 1, loops + 1) case None => sys.error("Cannot appear to get Some for this generator.") diff --git a/scripts/scald.rb b/scripts/scald.rb index 898c167f79..7e8770bbfe 100755 --- a/scripts/scald.rb +++ b/scripts/scald.rb @@ -140,6 +140,8 @@ def maven_filename(jar_filename) "2.10" elsif SCALA_VERSION.start_with?("2.11") "2.11" +elsif SCALA_VERSION.start_with?("2.12") + "2.12" else SCALA_VERSION end From 968f381933e00953979c72043b586707007b70f0 Mon Sep 17 00:00:00 2001 From: Luca Clem Date: Thu, 23 Mar 2017 20:07:40 -0700 Subject: [PATCH 10/16] Add andThen to Mappable (#1656) Mappable.andThen is defined in TypedSource and when invoked it returns a TypedSource. In this patch I'm overriding that method so that when you invoke a Mappable.andThen it returns a Mappable object. This can be useful when the user of Mappable wants to do toIeratable after applying a transformation using andThen --- .../scala/com/twitter/scalding/Source.scala | 18 +++++++ .../com/twitter/scalding/SourceSpec.scala | 51 +++++++++++++++++++ 2 files changed, 69 insertions(+) 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 bbbeb58027..35c9a30b0b 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Source.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Source.scala @@ -250,6 +250,24 @@ trait Mappable[+T] extends Source with TypedSource[T] { val conv = converter mode.openForRead(config, tap).asScala.map { te => conv(te.selectEntry(sourceFields)) } } + + /** + * Transform this Mappable into another by mapping after. + * We don't call this map because of conflicts with Mappable, unfortunately + */ + override def andThen[U](fn: T => U): Mappable[U] = { + val self = this // compiler generated self can cause problems with serialization + new Mappable[U] { + override def sourceFields = self.sourceFields + def converter[V >: U]: TupleConverter[V] = self.converter.andThen(fn) + override def read(implicit fd: FlowDef, mode: Mode): Pipe = self.read + override def andThen[U1](fn2: U => U1) = self.andThen(fn.andThen(fn2)) + def createTap(readOrWrite: AccessMode)(implicit mode: Mode): Tap[_, _, _] = + self.createTap(readOrWrite)(mode) + override def validateTaps(mode: Mode): Unit = self.validateTaps(mode) + } + } + } /** diff --git a/scalding-core/src/test/scala/com/twitter/scalding/SourceSpec.scala b/scalding-core/src/test/scala/com/twitter/scalding/SourceSpec.scala index 713d464daf..3d603ae44c 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/SourceSpec.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/SourceSpec.scala @@ -104,3 +104,54 @@ class AddRemoveOneJob(args: Args) extends Job(args) { .write(RemoveOneTsv("output")) } + +class MapTypedPipe(args: Args) extends Job(args) { + TypedPipe.from(TypedText.tsv[(Int, String)]("input")) + .map(MapFunctionAndThenTest.mapFunction) + .write(TypedText.tsv[(Int, String, Int)]("output")) +} + +class IdentityTypedPipe(args: Args) extends Job(args) { + TypedPipe.from( + TypedText.tsv[(Int, String)]("input") + .andThen(MapFunctionAndThenTest.mapFunction)) + .write(TypedText.tsv[(Int, String, Int)]("output")) +} + +object MapFunctionAndThenTest { + def mapFunction(input: (Int, String)): (Int, String, Int) = + (input._1, input._2, input._1) + + val input: List[(Int, String)] = List((0, "a"), (1, "b"), (2, "c")) + val output: List[(Int, String, Int)] = List((0, "a", 0), (1, "b", 1), (2, "c", 2)) +} +class TypedPipeAndThenTest extends WordSpec with Matchers { + import Dsl._ + import MapFunctionAndThenTest._ + "Mappable.andThen is like TypedPipe.map" should { + JobTest(new MapTypedPipe(_)) + .source(TypedText.tsv[(Int, String)]("input"), input) + .typedSink(TypedText.tsv[(Int, String, Int)]("output")){ outputBuffer => + val outMap = outputBuffer.toList + "TypedPipe return proper results" in { + outMap should have size 3 + outMap shouldBe output + } + } + .run + .finish() + + JobTest(new IdentityTypedPipe(_)) + .source(TypedText.tsv[(Int, String)]("input"), input) + .typedSink(TypedText.tsv[(Int, String, Int)]("output")){ outputBuffer => + val outMap = outputBuffer.toList + "Mappable.andThen return proper results" in { + outMap should have size 3 + outMap shouldBe output + } + } + .run + .finish() + + } +} From f53b5c1531547e75133703a61779d3c51ed84bdd Mon Sep 17 00:00:00 2001 From: Anton Panasenko Date: Wed, 22 Mar 2017 12:29:03 -0700 Subject: [PATCH 11/16] Estimate input size only for Hfs and GlobHfs taps --- .../scalding/commons/tap/VersionedTap.java | 9 ++- .../commons/VersionedKeyValSourceTest.scala | 31 +++++++-- .../com/twitter/scalding/tap/GlobHfs.java | 52 ++++++++++++++ .../scalding/reducer_estimation/Common.scala | 42 +++++------- .../InputSizeReducerEstimator.scala | 2 +- .../scalding/platform/HadoopPlatform.scala | 5 ++ .../ReducerEstimatorTest.scala | 67 ++++++++++++++++++- 7 files changed, 176 insertions(+), 32 deletions(-) create mode 100644 scalding-core/src/main/java/com/twitter/scalding/tap/GlobHfs.java diff --git a/scalding-commons/src/main/java/com/twitter/scalding/commons/tap/VersionedTap.java b/scalding-commons/src/main/java/com/twitter/scalding/commons/tap/VersionedTap.java index 63fad4d53c..93a014ed19 100644 --- a/scalding-commons/src/main/java/com/twitter/scalding/commons/tap/VersionedTap.java +++ b/scalding-commons/src/main/java/com/twitter/scalding/commons/tap/VersionedTap.java @@ -3,6 +3,7 @@ import java.io.IOException; import com.twitter.scalding.commons.datastores.VersionedStore; +import com.twitter.scalding.tap.GlobHfs; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -14,9 +15,8 @@ import cascading.flow.FlowProcess; import cascading.scheme.Scheme; -import cascading.tap.hadoop.Hfs; -public class VersionedTap extends Hfs { +public class VersionedTap extends GlobHfs { public static enum TapMode {SOURCE, SINK} public Long version = null; @@ -106,6 +106,11 @@ public void sinkConfInit(FlowProcess process, JobConf conf) { FileOutputFormat.setOutputPath(conf, new Path(newVersionPath)); } + @Override + public long getSize(JobConf conf) throws IOException { + return getSize(new Path(getSourcePath(conf)), conf); + } + @Override public boolean resourceExists(JobConf jc) throws IOException { return getStore(jc).mostRecentVersion() != null; 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 e3189f17b1..815f3fa13f 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 @@ -17,13 +17,12 @@ package com.twitter.scalding.commons.source import org.scalatest.{ Matchers, WordSpec } import com.twitter.scalding._ -import com.twitter.scalding.commons.datastores.VersionedStore; +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.JobConf - -import java.io.File +import java.io.{ File, FileWriter } // Use the scalacheck generators import scala.collection.mutable.Buffer @@ -124,18 +123,42 @@ class VersionedKeyValSourceTest extends WordSpec with Matchers { // should not throw validateVersion(path) } + + "calculate right size of source" in { + val oldContent = "size of old content should be ignored" + val content = "Hello World" + val contentSize = content.getBytes.length + val path = setupLocalVersionStore(100L to 102L, { + case 102L => Some(content) + case _ => Some(oldContent) + }) + + val keyValueSize = VersionedKeyValSource(path) + .source + .getSize(new JobConf()) + + contentSize should be (keyValueSize) + } } /** * Creates a temp dir and then creates the provided versions within it. */ - private def setupLocalVersionStore(versions: Seq[Long]): String = { + private def setupLocalVersionStore(versions: Seq[Long], contentFn: Long => Option[String] = _ => None): String = { val root = Files.createTempDir() root.deleteOnExit() val store = new VersionedStore(root.getAbsolutePath) versions foreach { v => val p = store.createVersion(v) new File(p).mkdirs() + + contentFn(v) + .foreach { text => + val content = new FileWriter(new File(p + "/test")) + content.write(text) + content.close() + } + store.succeedVersion(p) } diff --git a/scalding-core/src/main/java/com/twitter/scalding/tap/GlobHfs.java b/scalding-core/src/main/java/com/twitter/scalding/tap/GlobHfs.java new file mode 100644 index 0000000000..70b5775a45 --- /dev/null +++ b/scalding-core/src/main/java/com/twitter/scalding/tap/GlobHfs.java @@ -0,0 +1,52 @@ +package com.twitter.scalding.tap; + +import java.io.FileNotFoundException; +import java.io.IOException; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.OutputCollector; +import org.apache.hadoop.mapred.RecordReader; + +import cascading.scheme.Scheme; +import cascading.tap.hadoop.Hfs; + +/** + * Default implementation of getSize in {@link Hfs} don't respect to paths with glob patterns, + * that will throw IOException where we actually can calculate size of source. + */ +public class GlobHfs extends Hfs { + public GlobHfs(Scheme scheme) { + super(scheme); + } + + public GlobHfs(Scheme scheme, String stringPath) { + super(scheme, stringPath); + } + + @Override + public long getSize(JobConf conf) throws IOException { + return getSize(getPath(), conf); + } + + /** + * Get the total size of the file(s) specified by the Hfs, which may contain a glob + * pattern in its path, so we must be ready to handle that case. + */ + public static long getSize(Path path, JobConf conf) throws IOException { + FileSystem fs = path.getFileSystem(conf); + FileStatus[] statuses = fs.globStatus(path); + + if (statuses == null) { + throw new FileNotFoundException(String.format("File not found: %s", path)); + } + + long size = 0; + for (FileStatus status : statuses) { + size += fs.getContentSummary(status.getPath()).getLength(); + } + return size; + } +} diff --git a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala index 7dd2b78a1a..c190731e2c 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala @@ -1,16 +1,16 @@ package com.twitter.scalding.reducer_estimation -import cascading.flow.{ FlowStep, Flow, FlowStepStrategy } +import cascading.flow.{ Flow, FlowStep, FlowStepStrategy } +import cascading.tap.hadoop.Hfs +import cascading.tap.{ CompositeTap, Tap } import com.twitter.algebird.Monoid -import com.twitter.scalding.{ StringUtility, Config } -import cascading.tap.{ Tap, CompositeTap } -import org.apache.hadoop.fs.Path -import org.apache.hadoop.mapred.{ FileInputFormat, JobConf } -import org.slf4j.LoggerFactory +import com.twitter.scalding.tap.GlobHfs +import com.twitter.scalding.{ Config, StringUtility } import java.util.{ List => JList } - +import org.apache.hadoop.mapred.JobConf +import org.slf4j.LoggerFactory import scala.collection.JavaConverters._ -import scala.util.{ Try, Success, Failure } +import scala.util.{ Failure, Success, Try } object EstimatorConfig { @@ -44,6 +44,8 @@ object EstimatorConfig { } object Common { + private[this] val LOG = LoggerFactory.getLogger(this.getClass) + private def unrollTaps(taps: Seq[Tap[_, _, _]]): Seq[Tap[_, _, _]] = taps.flatMap { case multi: CompositeTap[_] => @@ -54,23 +56,15 @@ object Common { def unrollTaps(step: FlowStep[JobConf]): Seq[Tap[_, _, _]] = unrollTaps(step.getSources.asScala.toSeq) - /** - * Get the total size of the input paths, which may contain a glob - * pattern in its path, so we must be ready to handle that case. - */ - def inputSizes(step: FlowStep[JobConf]): Seq[(Path, Long)] = { + def inputSizes(step: FlowStep[JobConf]): Seq[(String, Long)] = { val conf = step.getConfig - - FileInputFormat - .getInputPaths(conf) - .map { path => - val fs = path.getFileSystem(conf) - val size = fs.globStatus(path) - .map(status => fs.getContentSummary(status.getPath).getLength) - .sum - - path -> size - } + unrollTaps(step).flatMap { + case tap: GlobHfs => Some(tap.toString -> tap.getSize(conf)) + case tap: Hfs => Some(tap.toString -> GlobHfs.getSize(tap.getPath, conf)) + case tap => + LOG.warn("InputSizeReducerEstimator unable to calculate size: " + tap) + None + } } def totalInputSize(step: FlowStep[JobConf]): Long = inputSizes(step).map(_._2).sum diff --git a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/InputSizeReducerEstimator.scala b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/InputSizeReducerEstimator.scala index 10982eab2c..3f68cbdbee 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/InputSizeReducerEstimator.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/InputSizeReducerEstimator.scala @@ -32,7 +32,7 @@ object InputSizeReducerEstimator { Common.inputSizes(info.step) match { case Nil => LOG.warn("InputSizeReducerEstimator unable to estimate reducers; " + - "cannot compute size of:\n - " + + "cannot compute size of (is it a non hfs tap?):\n - " + Common.unrollTaps(info.step).filterNot(_.isInstanceOf[Hfs]).mkString("\n - ")) None case inputSizes => diff --git a/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/HadoopPlatform.scala b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/HadoopPlatform.scala index 0a11c96186..580bbf24f9 100644 --- a/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/HadoopPlatform.scala +++ b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/HadoopPlatform.scala @@ -7,6 +7,8 @@ import java.io.{ BufferedWriter, File, FileWriter } import org.slf4j.LoggerFactory +import scala.util.Try + trait HadoopPlatform[P, R, T <: HadoopPlatform[P, R, T]] { private val LOG = LoggerFactory.getLogger(getClass) @@ -33,6 +35,9 @@ trait HadoopPlatform[P, R, T <: HadoopPlatform[P, R, T]] { def run(): Unit + def runExpectFailure[K](fn: Throwable => K): K = + fn(Try { run() }.failed.get) + def init(cons: P => R): R def execute(unit: R): Unit diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala index 9a43376428..6b07958f65 100644 --- a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala +++ b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala @@ -7,6 +7,8 @@ import org.scalatest.{ Matchers, WordSpec } import scala.collection.JavaConverters._ +import java.io.FileNotFoundException + object HipJob { val InSrcFileSize = 2496L val inPath = getClass.getResource("/hipster.txt") // file size is 2496 bytes @@ -81,6 +83,43 @@ class SimpleGlobJob(args: Args, customConfig: Config) extends Job(args) { .write(counts) } +class SimpleMemoryJob(args: Args, customConfig: Config) extends Job(args) { + import HipJob._ + + val inSrc = IterableSource(List( + "Direct trade American Apparel squid umami tote bag. Lo-fi XOXO gluten-free meh literally, typewriter readymade wolf salvia whatever drinking vinegar organic. Four loko literally bicycle rights drinking vinegar Cosby sweater hella stumptown. Dreamcatcher iPhone 90's organic chambray cardigan, wolf fixie gluten-free Brooklyn four loko. Mumblecore ennui twee, 8-bit food truck sustainable tote bag Williamsburg mixtape biodiesel. Semiotics Helvetica put a bird on it, roof party fashion axe organic post-ironic readymade Wes Anderson Pinterest keffiyeh. Craft beer meggings sartorial, butcher Marfa kitsch art party mustache Brooklyn vinyl.", + "Wolf flannel before they sold out vinyl, selfies four loko Bushwick Banksy Odd Future. Chillwave banh mi iPhone, Truffaut shabby chic craft beer keytar DIY. Scenester selvage deep v YOLO paleo blog photo booth fap. Sustainable wolf mixtape small batch skateboard, pop-up brunch asymmetrical seitan butcher Thundercats disrupt twee Etsy. You probably haven't heard of them freegan skateboard before they sold out, mlkshk pour-over Echo Park keytar retro farm-to-table. Tattooed sustainable beard, Helvetica Wes Anderson pickled vinyl yr pop-up Vice. Wolf bespoke lomo photo booth ethnic cliche." + )) + + override def config = super.config ++ customConfig.toMap.toMap + + TypedPipe.from(inSrc) + .flatMap(_.split("[^\\w]+")) + .map(_.toLowerCase -> 1) + .group + // force the number of reducers to two, to test with/without estimation + .withReducers(2) + .sum + .write(counts) +} + +class SimpleFileNotFoundJob(args: Args, customConfig: Config) extends Job(args) { + import HipJob._ + + val inSrc = TextLine("file.txt") + + override def config = super.config ++ customConfig.toMap.toMap + + TypedPipe.from(inSrc) + .flatMap(_.split("[^\\w]+")) + .map(_.toLowerCase -> 1) + .group + // force the number of reducers to two, to test with/without estimation + .withReducers(2) + .sum + .write(counts) +} + class GroupAllJob(args: Args, customConfig: Config) extends Job(args) { import HipJob._ @@ -140,7 +179,6 @@ class ReducerEstimatorTest extends WordSpec with Matchers with HadoopSharedPlatf conf.get(EstimatorConfig.originalNumReducers) should contain ("2") } .run() - } "run with correct number of reducers when overriding set values" in { @@ -178,7 +216,34 @@ class ReducerEstimatorTest extends WordSpec with Matchers with HadoopSharedPlatf conf.getNumReducers should contain (10) } .run() + } + + "ignore memory source in input size estimation" in { + val customConfig = Config.empty.addReducerEstimator(classOf[InputSizeReducerEstimator]) + + (InputSizeReducerEstimator.BytesPerReducer -> (1L << 10).toString) + + (Config.ReducerEstimatorOverride -> "true") + + HadoopPlatformJobTest(new SimpleMemoryJob(_, customConfig), cluster) + .inspectCompletedFlow { flow => + val steps = flow.getFlowSteps.asScala + steps should have size 1 + + val conf = Config.fromHadoop(steps.head.getConfig) + conf.getNumReducers should contain (2) + conf.get(EstimatorConfig.originalNumReducers) should contain ("2") + } + .run() + } + + "throw FileNotFoundException during estimation" in { + val customConfig = Config.empty.addReducerEstimator(classOf[InputSizeReducerEstimator]) + + (InputSizeReducerEstimator.BytesPerReducer -> (1L << 10).toString) + + (Config.ReducerEstimatorOverride -> "true") + HadoopPlatformJobTest(new SimpleFileNotFoundJob(_, customConfig), cluster) + .runExpectFailure { case error: FlowException => + error.getCause.getClass should be(classOf[FileNotFoundException]) + } } } From d7c27087855b165f8515c0b6a3ce8a35b5bf6916 Mon Sep 17 00:00:00 2001 From: Anton Panasenko Date: Wed, 8 Mar 2017 18:02:30 -0800 Subject: [PATCH 12/16] Use CombinedWritableSequenceFile for VKVS and make separate Combined*SequenceFileScheme --- build.sbt | 2 +- .../commons/scheme/KeyValueByteScheme.java | 5 +++-- .../scheme/CombinedSequenceFileScheme.scala | 17 +++++++++++++++++ 3 files changed, 21 insertions(+), 3 deletions(-) create mode 100644 scalding-commons/src/main/scala/com/twitter/scalding/commons/scheme/CombinedSequenceFileScheme.scala diff --git a/build.sbt b/build.sbt index d448c83dda..68add93060 100644 --- a/build.sbt +++ b/build.sbt @@ -21,7 +21,7 @@ val avroVersion = "1.7.4" val bijectionVersion = "0.9.5" val cascadingAvroVersion = "2.1.2" val chillVersion = "0.8.4" -val elephantbirdVersion = "4.8" +val elephantbirdVersion = "4.15" val hadoopLzoVersion = "0.4.19" val hadoopVersion = "2.5.0" val hbaseVersion = "0.94.10" diff --git a/scalding-commons/src/main/java/com/twitter/scalding/commons/scheme/KeyValueByteScheme.java b/scalding-commons/src/main/java/com/twitter/scalding/commons/scheme/KeyValueByteScheme.java index bf95160efb..a436c75e6d 100644 --- a/scalding-commons/src/main/java/com/twitter/scalding/commons/scheme/KeyValueByteScheme.java +++ b/scalding-commons/src/main/java/com/twitter/scalding/commons/scheme/KeyValueByteScheme.java @@ -11,15 +11,16 @@ import cascading.flow.FlowProcess; import cascading.scheme.SinkCall; import cascading.scheme.SourceCall; -import cascading.scheme.hadoop.WritableSequenceFile; import cascading.tuple.Fields; import cascading.tuple.Tuple; import cascading.tuple.TupleEntry; +import com.twitter.elephantbird.cascading2.scheme.CombinedWritableSequenceFile; + /** * */ -public class KeyValueByteScheme extends WritableSequenceFile { +public class KeyValueByteScheme extends CombinedWritableSequenceFile { public KeyValueByteScheme(Fields fields) { super(fields, BytesWritable.class, BytesWritable.class); } diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/scheme/CombinedSequenceFileScheme.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/scheme/CombinedSequenceFileScheme.scala new file mode 100644 index 0000000000..ee22414ec7 --- /dev/null +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/scheme/CombinedSequenceFileScheme.scala @@ -0,0 +1,17 @@ +package com.twitter.scalding.commons.scheme + +import cascading.scheme.Scheme +import com.twitter.elephantbird.cascading2.scheme.{CombinedSequenceFile, CombinedWritableSequenceFile} +import com.twitter.scalding.{HadoopSchemeInstance, SequenceFileScheme, WritableSequenceFileScheme} + + +trait CombinedSequenceFileScheme extends SequenceFileScheme { + // TODO Cascading doesn't support local mode yet + override def hdfsScheme = HadoopSchemeInstance(new CombinedSequenceFile(fields).asInstanceOf[Scheme[_, _, _, _, _]]) +} + +trait CombinedWritableSequenceFileScheme extends WritableSequenceFileScheme { + // TODO Cascading doesn't support local mode yet + override def hdfsScheme = + HadoopSchemeInstance(new CombinedWritableSequenceFile(fields, keyType, valueType).asInstanceOf[Scheme[_, _, _, _, _]]) +} \ No newline at end of file From c5f3922287b6220dcf792b50e6c7a2a76e0161df Mon Sep 17 00:00:00 2001 From: "Flavio W. Brasil" Date: Tue, 4 Apr 2017 18:28:28 -0700 Subject: [PATCH 13/16] add explicit return types to implicit methods and values (#1660) --- .../com/twitter/scalding/RangedArgs.scala | 2 +- .../twitter/scalding/avro/SchemaType.scala | 22 +++++----- .../commons/source/LzoGenericScheme.scala | 2 +- .../source/VersionedKeyValSource.scala | 4 +- .../commons/VersionedKeyValSourceTest.scala | 4 +- .../BijectedOrderedSerialization.scala | 4 +- .../com/twitter/scalding/CumulativeSum.scala | 2 +- .../twitter/scalding/FieldConversions.scala | 26 ++++++------ .../scalding/GeneratedTupleAdders.scala | 42 +++++++++---------- .../main/scala/com/twitter/scalding/Job.scala | 8 ++-- .../com/twitter/scalding/Operations.scala | 2 +- .../twitter/scalding/ReduceOperations.scala | 4 +- .../com/twitter/scalding/TupleConverter.scala | 2 +- .../com/twitter/scalding/TuplePacker.scala | 4 +- .../com/twitter/scalding/TupleUnpacker.scala | 2 +- .../com/twitter/scalding/bdd/BddDsl.scala | 2 +- .../bdd/PipeOperationsConversions.scala | 32 +++++++------- .../com/twitter/scalding/bdd/TBddDsl.scala | 2 +- .../bdd/TypedPipeOperationsConversions.scala | 8 ++-- .../twitter/scalding/mathematics/Matrix.scala | 6 +-- .../scalding/mathematics/SizeHint.scala | 2 +- .../serialization/WrappedSerialization.scala | 2 +- .../typed/GeneratedFlattenGroup.scala | 16 +++---- .../com/twitter/scalding/typed/Grouped.scala | 4 +- .../com/twitter/scalding/typed/Sketched.scala | 5 ++- .../twitter/scalding/typed/TypedPipe.scala | 4 +- .../scala/com/twitter/scalding/CoreTest.scala | 6 +-- .../twitter/scalding/ExecutionUtilTest.scala | 6 +-- .../scala/com/twitter/scalding/KryoTest.scala | 4 +- .../scalding/ReduceOperationsTest.scala | 2 +- .../com/twitter/scalding/SourceSpec.scala | 4 +- .../twitter/scalding/TypedDelimitedTest.scala | 6 +-- .../twitter/scalding/TypedFieldsTest.scala | 2 +- .../com/twitter/scalding/TypedPipeTest.scala | 4 +- .../TypedSketchJoinJobForEmptyKeysTest.scala | 2 +- .../mathematics/Matrix2OptimizationTest.scala | 8 ++-- .../typed/BijectedSourceSinkTest.scala | 2 +- .../typed/NoStackLineNumberTest.scala | 4 +- .../scalding/typed/TypedPipeDiffTest.scala | 10 ++--- .../scala/com/twitter/scalding/RichDate.scala | 2 +- .../scala/com/twitter/scalding/DateTest.scala | 2 +- .../scalding/platform/PlatformTest.scala | 6 +-- .../com/twitter/scalding/ReplImplicits.scala | 12 +++--- .../WriterReaderProperties.scala | 4 +- .../macros/MacroOrderingProperties.scala | 28 ++++++------- .../scalding/thrift/macros/PlatformTest.scala | 2 +- 46 files changed, 165 insertions(+), 164 deletions(-) diff --git a/scalding-args/src/main/scala/com/twitter/scalding/RangedArgs.scala b/scalding-args/src/main/scala/com/twitter/scalding/RangedArgs.scala index 74fbaa7809..65cb5585ea 100644 --- a/scalding-args/src/main/scala/com/twitter/scalding/RangedArgs.scala +++ b/scalding-args/src/main/scala/com/twitter/scalding/RangedArgs.scala @@ -17,7 +17,7 @@ limitations under the License. package com.twitter.scalding object RangedArgs { - implicit def rangedFromArgs(args: Args) = new RangedArgs(args) + implicit def rangedFromArgs(args: Args): RangedArgs = new RangedArgs(args) } case class Range[T](lower: T, upper: T)(implicit ord: Ordering[T]) { diff --git a/scalding-avro/src/main/scala/com/twitter/scalding/avro/SchemaType.scala b/scalding-avro/src/main/scala/com/twitter/scalding/avro/SchemaType.scala index b053cdf2f5..e24e71aa51 100644 --- a/scalding-avro/src/main/scala/com/twitter/scalding/avro/SchemaType.scala +++ b/scalding-avro/src/main/scala/com/twitter/scalding/avro/SchemaType.scala @@ -28,50 +28,50 @@ object AvroSchemaType { // primitive types - implicit def BooleanSchema = new AvroSchemaType[Boolean] { + implicit def BooleanSchema: AvroSchemaType[Boolean] = new AvroSchemaType[Boolean] { def schema = Schema.create(Schema.Type.BOOLEAN) } - implicit def ByteBufferSchema = new AvroSchemaType[ByteBuffer] { + implicit def ByteBufferSchema: AvroSchemaType[ByteBuffer] = new AvroSchemaType[ByteBuffer] { def schema = Schema.create(Schema.Type.BYTES) } - implicit def DoubleSchema = new AvroSchemaType[Double] { + implicit def DoubleSchema: AvroSchemaType[Double] = new AvroSchemaType[Double] { def schema = Schema.create(Schema.Type.DOUBLE) } - implicit def FloatSchema = new AvroSchemaType[Float] { + implicit def FloatSchema: AvroSchemaType[Float] = new AvroSchemaType[Float] { def schema = Schema.create(Schema.Type.FLOAT) } - implicit def IntSchema = new AvroSchemaType[Int] { + implicit def IntSchema: AvroSchemaType[Int] = new AvroSchemaType[Int] { def schema = Schema.create(Schema.Type.INT) } - implicit def LongSchema = new AvroSchemaType[Long] { + implicit def LongSchema: AvroSchemaType[Long] = new AvroSchemaType[Long] { def schema = Schema.create(Schema.Type.LONG) } - implicit def StringSchema = new AvroSchemaType[String] { + implicit def StringSchema: AvroSchemaType[String] = new AvroSchemaType[String] { def schema = Schema.create(Schema.Type.STRING) } // collections - implicit def CollectionSchema[CC[x] <: Iterable[x], T](implicit sch: AvroSchemaType[T]) = new AvroSchemaType[CC[T]] { + implicit def CollectionSchema[CC[x] <: Iterable[x], T](implicit sch: AvroSchemaType[T]): AvroSchemaType[CC[T]] = new AvroSchemaType[CC[T]] { def schema = Schema.createArray(sch.schema) } - implicit def ArraySchema[CC[x] <: Array[x], T](implicit sch: AvroSchemaType[T]) = new AvroSchemaType[CC[T]] { + implicit def ArraySchema[CC[x] <: Array[x], T](implicit sch: AvroSchemaType[T]): AvroSchemaType[CC[T]] { val schema: Schema } = new AvroSchemaType[CC[T]] { val schema = Schema.createArray(sch.schema) } //maps - implicit def MapSchema[CC[String, x] <: Map[String, x], T](implicit sch: AvroSchemaType[T]) = new AvroSchemaType[CC[String, T]] { + implicit def MapSchema[CC[String, x] <: Map[String, x], T](implicit sch: AvroSchemaType[T]): AvroSchemaType[CC[String, T]] = new AvroSchemaType[CC[String, T]] { def schema = Schema.createMap(sch.schema) } // Avro SpecificRecord - implicit def SpecificRecordSchema[T <: SpecificRecord](implicit mf: Manifest[T]) = new AvroSchemaType[T] { + implicit def SpecificRecordSchema[T <: SpecificRecord](implicit mf: Manifest[T]): AvroSchemaType[T] = new AvroSchemaType[T] { def schema = mf.runtimeClass.newInstance.asInstanceOf[SpecificRecord].getSchema } diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericScheme.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericScheme.scala index 25a6c4e0cf..09ad893905 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericScheme.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericScheme.scala @@ -41,7 +41,7 @@ private[source] object ExternalizerSerializer { import com.twitter.bijection.Inversion.attemptWhen import com.twitter.bijection.codec.Base64 - implicit val baseInj = JavaSerializationInjection[Externalizer[T]] + implicit val baseInj: Injection[Externalizer[T], Array[Byte]] = JavaSerializationInjection[Externalizer[T]] implicit val unwrap: Injection[GZippedBase64String, String] = // this does not catch cases where it's Base64 but not compressed 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 72f37a167b..0a5878c762 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 @@ -211,10 +211,10 @@ class VersionedKeyValSource[K, V](val path: String, val sourceVersion: Option[Lo object RichPipeEx extends java.io.Serializable { implicit def pipeToRichPipeEx(pipe: Pipe): RichPipeEx = new RichPipeEx(pipe) - implicit def typedPipeToRichPipeEx[K: Ordering, V: Monoid](pipe: TypedPipe[(K, V)]) = + implicit def typedPipeToRichPipeEx[K: Ordering, V: Monoid](pipe: TypedPipe[(K, V)]): TypedRichPipeEx[K, V] = new TypedRichPipeEx(pipe) implicit def keyedListLikeToRichPipeEx[K: Ordering, V: Monoid, T[K, +V] <: KeyedListLike[K, V, T]]( - kll: KeyedListLike[K, V, T]) = typedPipeToRichPipeEx(kll.toTypedPipe) + kll: KeyedListLike[K, V, T]): TypedRichPipeEx[K, V] = typedPipeToRichPipeEx(kll.toTypedPipe) } class TypedRichPipeEx[K: Ordering, V: Monoid](pipe: TypedPipe[(K, V)]) extends java.io.Serializable { 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 815f3fa13f..174ad17186 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 @@ -30,7 +30,7 @@ class TypedWriteIncrementalJob(args: Args) extends Job(args) { import RichPipeEx._ val pipe = TypedPipe.from(TypedTsv[Int]("input")) - implicit val inj = Injection.connect[(Int, Int), (Array[Byte], Array[Byte])] + implicit val inj: Injection[(Int, Int), (Array[Byte], Array[Byte])] = Injection.connect[(Int, Int), (Array[Byte], Array[Byte])] pipe .map{ k => (k, k) } @@ -41,7 +41,7 @@ class MoreComplexTypedWriteIncrementalJob(args: Args) extends Job(args) { import RichPipeEx._ val pipe = TypedPipe.from(TypedTsv[Int]("input")) - implicit val inj = Injection.connect[(Int, Int), (Array[Byte], Array[Byte])] + implicit val inj: Injection[(Int, Int), (Array[Byte], Array[Byte])] = Injection.connect[(Int, Int), (Array[Byte], Array[Byte])] pipe .map{ k => (k, k) } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/BijectedOrderedSerialization.scala b/scalding-core/src/main/scala/com/twitter/scalding/BijectedOrderedSerialization.scala index 067fb4e78b..b5ad813408 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/BijectedOrderedSerialization.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/BijectedOrderedSerialization.scala @@ -19,10 +19,10 @@ import com.twitter.scalding.serialization.OrderedSerialization import com.twitter.bijection.{ ImplicitBijection, Injection } object BijectedOrderedSerialization { - implicit def fromBijection[T, U](implicit bij: ImplicitBijection[T, U], ordSer: OrderedSerialization[U]) = + implicit def fromBijection[T, U](implicit bij: ImplicitBijection[T, U], ordSer: OrderedSerialization[U]): OrderedSerialization[T] = OrderedSerialization.viaTransform[T, U](bij.apply(_), bij.invert(_)) - implicit def fromInjection[T, U](implicit bij: Injection[T, U], ordSer: OrderedSerialization[U]) = + implicit def fromInjection[T, U](implicit bij: Injection[T, U], ordSer: OrderedSerialization[U]): OrderedSerialization[T] = OrderedSerialization.viaTryTransform[T, U](bij.apply(_), bij.invert(_)) } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/CumulativeSum.scala b/scalding-core/src/main/scala/com/twitter/scalding/CumulativeSum.scala index 2513854721..b26882c732 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/CumulativeSum.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/CumulativeSum.scala @@ -30,7 +30,7 @@ import com.twitter.algebird._ * than 2 entries. */ object CumulativeSum { - implicit def toCumulativeSum[K, U, V](pipe: TypedPipe[(K, (U, V))]) = + implicit def toCumulativeSum[K, U, V](pipe: TypedPipe[(K, (U, V))]): CumulativeSumExtension[K, U, V] = new CumulativeSumExtension(pipe) class CumulativeSumExtension[K, U, V]( diff --git a/scalding-core/src/main/scala/com/twitter/scalding/FieldConversions.scala b/scalding-core/src/main/scala/com/twitter/scalding/FieldConversions.scala index 41a72d566d..44179cdc33 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/FieldConversions.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/FieldConversions.scala @@ -49,7 +49,7 @@ trait LowPriorityFieldConversions { * Lists are handled by an implicit in FieldConversions, which have * higher priority. */ - implicit def productToFields(f: Product) = { + implicit def productToFields(f: Product): Fields = { val fields = new Fields(f.productIterator.map { anyToFieldArg }.toSeq: _*) f.productIterator.foreach { _ match { @@ -111,22 +111,22 @@ trait FieldConversions extends LowPriorityFieldConversions { } //Single entry fields: - implicit def unitToFields(u: Unit) = Fields.NONE // linter:ignore - implicit def intToFields(x: Int) = new Fields(new java.lang.Integer(x)) - implicit def integerToFields(x: java.lang.Integer) = new Fields(x) - implicit def stringToFields(x: String) = new Fields(x) - implicit def enumValueToFields(x: Enumeration#Value) = new Fields(x.toString) + implicit def unitToFields(u: Unit): Fields = Fields.NONE // linter:ignore + implicit def intToFields(x: Int): Fields = new Fields(new java.lang.Integer(x)) + implicit def integerToFields(x: java.lang.Integer): Fields = new Fields(x) + implicit def stringToFields(x: String): Fields = new Fields(x) + implicit def enumValueToFields(x: Enumeration#Value): Fields = new Fields(x.toString) /** * '* means Fields.ALL, otherwise we take the .name */ - implicit def symbolToFields(x: Symbol) = { + implicit def symbolToFields(x: Symbol): Fields = { if (x == '*) { Fields.ALL } else { new Fields(x.name) } } - implicit def fieldToFields(f: Field[_]) = RichFields(f) + implicit def fieldToFields(f: Field[_]): RichFields = RichFields(f) @tailrec final def newSymbol(avoid: Set[Symbol], guess: Symbol, trial: Int = 0): Symbol = { @@ -171,18 +171,18 @@ trait FieldConversions extends LowPriorityFieldConversions { implicit def fromEnum[T <: Enumeration](enumeration: T): Fields = new Fields(enumeration.values.toList.map { _.toString }: _*) - implicit def fields[T <: TraversableOnce[Symbol]](f: T) = new Fields(f.toSeq.map(_.name): _*) - implicit def strFields[T <: TraversableOnce[String]](f: T) = new Fields(f.toSeq: _*) - implicit def intFields[T <: TraversableOnce[Int]](f: T) = { + implicit def fields[T <: TraversableOnce[Symbol]](f: T): Fields = new Fields(f.toSeq.map(_.name): _*) + implicit def strFields[T <: TraversableOnce[String]](f: T): Fields = new Fields(f.toSeq: _*) + implicit def intFields[T <: TraversableOnce[Int]](f: T): Fields = { new Fields(f.toSeq.map { new java.lang.Integer(_) }: _*) } - implicit def fieldFields[T <: TraversableOnce[Field[_]]](f: T) = RichFields(f.toSeq) + implicit def fieldFields[T <: TraversableOnce[Field[_]]](f: T): RichFields = RichFields(f.toSeq) /** * Useful to convert f : Any* to Fields. This handles mixed cases ("hey", 'you). * Not sure we should be this flexible, but given that Cascading will throw an * exception before scheduling the job, I guess this is okay. */ - implicit def parseAnySeqToFields[T <: TraversableOnce[Any]](anyf: T) = { + implicit def parseAnySeqToFields[T <: TraversableOnce[Any]](anyf: T): Fields = { val fields = new Fields(anyf.toSeq.map { anyToFieldArg }: _*) anyf.foreach { _ match { diff --git a/scalding-core/src/main/scala/com/twitter/scalding/GeneratedTupleAdders.scala b/scalding-core/src/main/scala/com/twitter/scalding/GeneratedTupleAdders.scala index 3d5e5a576f..a3353666aa 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/GeneratedTupleAdders.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/GeneratedTupleAdders.scala @@ -95,7 +95,7 @@ trait GeneratedTupleAdders { } } - implicit def tup1ToAdder[A](tup: Tuple1[A]) = new Tuple1Adder(tup) + implicit def tup1ToAdder[A](tup: Tuple1[A]): Tuple1Adder[A] = new Tuple1Adder(tup) class Tuple2Adder[A, B](tup: Tuple2[A, B]) { def :+[C](other: C) = { @@ -186,7 +186,7 @@ trait GeneratedTupleAdders { } } - implicit def tup2ToAdder[A, B](tup: Tuple2[A, B]) = new Tuple2Adder(tup) + implicit def tup2ToAdder[A, B](tup: Tuple2[A, B]): Tuple2Adder[A, B] = new Tuple2Adder(tup) class Tuple3Adder[A, B, C](tup: Tuple3[A, B, C]) { def :+[D](other: D) = { @@ -273,7 +273,7 @@ trait GeneratedTupleAdders { } } - implicit def tup3ToAdder[A, B, C](tup: Tuple3[A, B, C]) = new Tuple3Adder(tup) + implicit def tup3ToAdder[A, B, C](tup: Tuple3[A, B, C]): Tuple3Adder[A, B, C] = new Tuple3Adder(tup) class Tuple4Adder[A, B, C, D](tup: Tuple4[A, B, C, D]) { def :+[E](other: E) = { @@ -356,7 +356,7 @@ trait GeneratedTupleAdders { } } - implicit def tup4ToAdder[A, B, C, D](tup: Tuple4[A, B, C, D]) = new Tuple4Adder(tup) + implicit def tup4ToAdder[A, B, C, D](tup: Tuple4[A, B, C, D]): Tuple4Adder[A, B, C, D] = new Tuple4Adder(tup) class Tuple5Adder[A, B, C, D, E](tup: Tuple5[A, B, C, D, E]) { def :+[F](other: F) = { @@ -435,7 +435,7 @@ trait GeneratedTupleAdders { } } - implicit def tup5ToAdder[A, B, C, D, E](tup: Tuple5[A, B, C, D, E]) = new Tuple5Adder(tup) + implicit def tup5ToAdder[A, B, C, D, E](tup: Tuple5[A, B, C, D, E]): Tuple5Adder[A, B, C, D, E] = new Tuple5Adder(tup) class Tuple6Adder[A, B, C, D, E, F](tup: Tuple6[A, B, C, D, E, F]) { def :+[G](other: G) = { @@ -510,7 +510,7 @@ trait GeneratedTupleAdders { } } - implicit def tup6ToAdder[A, B, C, D, E, F](tup: Tuple6[A, B, C, D, E, F]) = new Tuple6Adder(tup) + implicit def tup6ToAdder[A, B, C, D, E, F](tup: Tuple6[A, B, C, D, E, F]): Tuple6Adder[A, B, C, D, E, F] = new Tuple6Adder(tup) class Tuple7Adder[A, B, C, D, E, F, G](tup: Tuple7[A, B, C, D, E, F, G]) { def :+[H](other: H) = { @@ -581,7 +581,7 @@ trait GeneratedTupleAdders { } } - implicit def tup7ToAdder[A, B, C, D, E, F, G](tup: Tuple7[A, B, C, D, E, F, G]) = new Tuple7Adder(tup) + implicit def tup7ToAdder[A, B, C, D, E, F, G](tup: Tuple7[A, B, C, D, E, F, G]): Tuple7Adder[A, B, C, D, E, F, G] = new Tuple7Adder(tup) class Tuple8Adder[A, B, C, D, E, F, G, H](tup: Tuple8[A, B, C, D, E, F, G, H]) { def :+[I](other: I) = { @@ -648,7 +648,7 @@ trait GeneratedTupleAdders { } } - implicit def tup8ToAdder[A, B, C, D, E, F, G, H](tup: Tuple8[A, B, C, D, E, F, G, H]) = new Tuple8Adder(tup) + implicit def tup8ToAdder[A, B, C, D, E, F, G, H](tup: Tuple8[A, B, C, D, E, F, G, H]): Tuple8Adder[A, B, C, D, E, F, G, H] = new Tuple8Adder(tup) class Tuple9Adder[A, B, C, D, E, F, G, H, I](tup: Tuple9[A, B, C, D, E, F, G, H, I]) { def :+[J](other: J) = { @@ -711,7 +711,7 @@ trait GeneratedTupleAdders { } } - implicit def tup9ToAdder[A, B, C, D, E, F, G, H, I](tup: Tuple9[A, B, C, D, E, F, G, H, I]) = new Tuple9Adder(tup) + implicit def tup9ToAdder[A, B, C, D, E, F, G, H, I](tup: Tuple9[A, B, C, D, E, F, G, H, I]): Tuple9Adder[A, B, C, D, E, F, G, H, I] = new Tuple9Adder(tup) class Tuple10Adder[A, B, C, D, E, F, G, H, I, J](tup: Tuple10[A, B, C, D, E, F, G, H, I, J]) { def :+[K](other: K) = { @@ -770,7 +770,7 @@ trait GeneratedTupleAdders { } } - implicit def tup10ToAdder[A, B, C, D, E, F, G, H, I, J](tup: Tuple10[A, B, C, D, E, F, G, H, I, J]) = new Tuple10Adder(tup) + implicit def tup10ToAdder[A, B, C, D, E, F, G, H, I, J](tup: Tuple10[A, B, C, D, E, F, G, H, I, J]): Tuple10Adder[A, B, C, D, E, F, G, H, I, J] = new Tuple10Adder(tup) class Tuple11Adder[A, B, C, D, E, F, G, H, I, J, K](tup: Tuple11[A, B, C, D, E, F, G, H, I, J, K]) { def :+[L](other: L) = { @@ -825,7 +825,7 @@ trait GeneratedTupleAdders { } } - implicit def tup11ToAdder[A, B, C, D, E, F, G, H, I, J, K](tup: Tuple11[A, B, C, D, E, F, G, H, I, J, K]) = new Tuple11Adder(tup) + implicit def tup11ToAdder[A, B, C, D, E, F, G, H, I, J, K](tup: Tuple11[A, B, C, D, E, F, G, H, I, J, K]): Tuple11Adder[A, B, C, D, E, F, G, H, I, J, K] = new Tuple11Adder(tup) class Tuple12Adder[A, B, C, D, E, F, G, H, I, J, K, L](tup: Tuple12[A, B, C, D, E, F, G, H, I, J, K, L]) { def :+[M](other: M) = { @@ -876,7 +876,7 @@ trait GeneratedTupleAdders { } } - implicit def tup12ToAdder[A, B, C, D, E, F, G, H, I, J, K, L](tup: Tuple12[A, B, C, D, E, F, G, H, I, J, K, L]) = new Tuple12Adder(tup) + implicit def tup12ToAdder[A, B, C, D, E, F, G, H, I, J, K, L](tup: Tuple12[A, B, C, D, E, F, G, H, I, J, K, L]): Tuple12Adder[A, B, C, D, E, F, G, H, I, J, K, L] = new Tuple12Adder(tup) class Tuple13Adder[A, B, C, D, E, F, G, H, I, J, K, L, M](tup: Tuple13[A, B, C, D, E, F, G, H, I, J, K, L, M]) { def :+[N](other: N) = { @@ -923,7 +923,7 @@ trait GeneratedTupleAdders { } } - implicit def tup13ToAdder[A, B, C, D, E, F, G, H, I, J, K, L, M](tup: Tuple13[A, B, C, D, E, F, G, H, I, J, K, L, M]) = new Tuple13Adder(tup) + implicit def tup13ToAdder[A, B, C, D, E, F, G, H, I, J, K, L, M](tup: Tuple13[A, B, C, D, E, F, G, H, I, J, K, L, M]): Tuple13Adder[A, B, C, D, E, F, G, H, I, J, K, L, M] = new Tuple13Adder(tup) class Tuple14Adder[A, B, C, D, E, F, G, H, I, J, K, L, M, N](tup: Tuple14[A, B, C, D, E, F, G, H, I, J, K, L, M, N]) { def :+[O](other: O) = { @@ -966,7 +966,7 @@ trait GeneratedTupleAdders { } } - implicit def tup14ToAdder[A, B, C, D, E, F, G, H, I, J, K, L, M, N](tup: Tuple14[A, B, C, D, E, F, G, H, I, J, K, L, M, N]) = new Tuple14Adder(tup) + implicit def tup14ToAdder[A, B, C, D, E, F, G, H, I, J, K, L, M, N](tup: Tuple14[A, B, C, D, E, F, G, H, I, J, K, L, M, N]): Tuple14Adder[A, B, C, D, E, F, G, H, I, J, K, L, M, N] = new Tuple14Adder(tup) class Tuple15Adder[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O](tup: Tuple15[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O]) { def :+[P](other: P) = { @@ -1005,7 +1005,7 @@ trait GeneratedTupleAdders { } } - implicit def tup15ToAdder[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O](tup: Tuple15[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O]) = new Tuple15Adder(tup) + implicit def tup15ToAdder[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O](tup: Tuple15[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O]): Tuple15Adder[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O] = new Tuple15Adder(tup) class Tuple16Adder[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P](tup: Tuple16[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P]) { def :+[Q](other: Q) = { @@ -1040,7 +1040,7 @@ trait GeneratedTupleAdders { } } - implicit def tup16ToAdder[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P](tup: Tuple16[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P]) = new Tuple16Adder(tup) + implicit def tup16ToAdder[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P](tup: Tuple16[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P]): Tuple16Adder[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P] = new Tuple16Adder(tup) class Tuple17Adder[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q](tup: Tuple17[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q]) { def :+[R](other: R) = { @@ -1071,7 +1071,7 @@ trait GeneratedTupleAdders { } } - implicit def tup17ToAdder[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q](tup: Tuple17[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q]) = new Tuple17Adder(tup) + implicit def tup17ToAdder[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q](tup: Tuple17[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q]): Tuple17Adder[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q] = new Tuple17Adder(tup) class Tuple18Adder[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R](tup: Tuple18[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R]) { def :+[S](other: S) = { @@ -1098,7 +1098,7 @@ trait GeneratedTupleAdders { } } - implicit def tup18ToAdder[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R](tup: Tuple18[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R]) = new Tuple18Adder(tup) + implicit def tup18ToAdder[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R](tup: Tuple18[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R]): Tuple18Adder[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R] = new Tuple18Adder(tup) class Tuple19Adder[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S](tup: Tuple19[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S]) { def :+[T](other: T) = { @@ -1121,7 +1121,7 @@ trait GeneratedTupleAdders { } } - implicit def tup19ToAdder[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S](tup: Tuple19[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S]) = new Tuple19Adder(tup) + implicit def tup19ToAdder[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S](tup: Tuple19[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S]): Tuple19Adder[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S] = new Tuple19Adder(tup) class Tuple20Adder[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T](tup: Tuple20[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T]) { def :+[U](other: U) = { @@ -1140,7 +1140,7 @@ trait GeneratedTupleAdders { } } - implicit def tup20ToAdder[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T](tup: Tuple20[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T]) = new Tuple20Adder(tup) + implicit def tup20ToAdder[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T](tup: Tuple20[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T]): Tuple20Adder[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T] = new Tuple20Adder(tup) class Tuple21Adder[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U](tup: Tuple21[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U]) { def :+[V](other: V) = { @@ -1155,7 +1155,7 @@ trait GeneratedTupleAdders { } } - implicit def tup21ToAdder[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U](tup: Tuple21[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U]) = new Tuple21Adder(tup) + implicit def tup21ToAdder[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U](tup: Tuple21[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U]): Tuple21Adder[A, B, C, D, E, F, G, H, I, J, K, L, M, N, O, P, Q, R, S, T, U] = new Tuple21Adder(tup) } // end of autogenerated 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 355ddffe6b..85d347d1c6 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Job.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Job.scala @@ -116,7 +116,7 @@ class Job(val args: Args) extends FieldConversions with java.io.Serializable { //This is the FlowDef used by all Sources this job creates @transient - implicit protected val flowDef = { + implicit protected val flowDef: FlowDef = { val fd = new FlowDef fd.setName(name) fd @@ -265,7 +265,7 @@ class Job(val args: Args) extends FieldConversions with java.io.Serializable { } // Print custom counters unless --scalding.nocounters is used or there are no custom stats if (!args.boolean("scalding.nocounters")) { - implicit val statProvider = statsData + implicit val statProvider: CascadingStats = statsData val jobStats = Stats.getAllCustomCounters if (!jobStats.isEmpty) { println("Dumping custom counters:") @@ -389,7 +389,7 @@ trait DefaultDateRangeJob extends Job { // Optionally take --tz argument, or use Pacific time. Derived classes may // override defaultTimeZone to change the default. def defaultTimeZone = PACIFIC - implicit lazy val tz = args.optional("tz") match { + implicit lazy val tz: java.util.TimeZone = args.optional("tz") match { case Some(tzn) => java.util.TimeZone.getTimeZone(tzn) case None => defaultTimeZone } @@ -410,7 +410,7 @@ trait DefaultDateRangeJob extends Job { (s, e) } - implicit lazy val dateRange = DateRange(startDate, if (period > 0) startDate + Days(period) - Millisecs(1) else endDate) + implicit lazy val dateRange: DateRange = DateRange(startDate, if (period > 0) startDate + Days(period) - Millisecs(1) else endDate) override def next: Option[Job] = if (period > 0) { diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Operations.scala b/scalding-core/src/main/scala/com/twitter/scalding/Operations.scala index 6f95c95077..e3e46124b7 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Operations.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Operations.scala @@ -257,7 +257,7 @@ package com.twitter.scalding { override def operate(flowProcess: FlowProcess[_], functionCall: FunctionCall[MapsideCache[K, V]]): Unit = { val cache = functionCall.getContext - implicit val sg = boxedSemigroup.get + implicit val sg: Semigroup[V] = boxedSemigroup.get val res: Map[K, V] = mergeTraversableOnce(lockedFn.get(functionCall.getArguments)) val evicted = cache.putAll(res) add(evicted, functionCall) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/ReduceOperations.scala b/scalding-core/src/main/scala/com/twitter/scalding/ReduceOperations.scala index 3b601b78d5..376e0daf2e 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/ReduceOperations.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/ReduceOperations.scala @@ -110,7 +110,7 @@ trait ReduceOperations[+Self <: ReduceOperations[Self]] extends java.io.Serializ //bits = log(m) == 2 *log(104/errPercent) = 2log(104) - 2*log(errPercent) def log2(x: Double) = scala.math.log(x) / scala.math.log(2.0) val bits = 2 * scala.math.ceil(log2(104) - log2(errPercent)).toInt - implicit val hmm = new HyperLogLogMonoid(bits) + implicit val hmm: HyperLogLogMonoid = new HyperLogLogMonoid(bits) mapPlusMap(f) { (t: T) => hmm.create(t) } (fn) } @@ -391,7 +391,7 @@ trait ReduceOperations[+Self <: ReduceOperations[Self]] extends java.io.Serializ def sortedTake[T](f: (Fields, Fields), k: Int)(implicit conv: TupleConverter[T], ord: Ordering[T]): Self = { assert(f._2.size == 1, "output field size must be 1") - implicit val mon = new PriorityQueueMonoid[T](k) + implicit val mon: PriorityQueueMonoid[T] = new PriorityQueueMonoid[T](k) mapPlusMap(f) { (tup: T) => mon.build(tup) } { (lout: PriorityQueue[T]) => lout.iterator.asScala.toList.sorted } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/TupleConverter.scala b/scalding-core/src/main/scala/com/twitter/scalding/TupleConverter.scala index 0a0f2c12d9..b6cf65cac2 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/TupleConverter.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/TupleConverter.scala @@ -44,7 +44,7 @@ trait TupleConverter[@specialized(Int, Long, Float, Double) T] extends java.io.S } trait LowPriorityTupleConverters extends java.io.Serializable { - implicit def singleConverter[@specialized(Int, Long, Float, Double) A](implicit g: TupleGetter[A]) = + implicit def singleConverter[@specialized(Int, Long, Float, Double) A](implicit g: TupleGetter[A]): TupleConverter[A] = new TupleConverter[A] { def apply(tup: TupleEntry) = g.get(tup.getTuple, 0) def arity = 1 diff --git a/scalding-core/src/main/scala/com/twitter/scalding/TuplePacker.scala b/scalding-core/src/main/scala/com/twitter/scalding/TuplePacker.scala index 43c539716e..326c96aad0 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/TuplePacker.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/TuplePacker.scala @@ -38,11 +38,11 @@ trait TuplePacker[T] extends java.io.Serializable { object TuplePacker extends CaseClassPackers trait CaseClassPackers extends LowPriorityTuplePackers { - implicit def caseClassPacker[T <: Product](implicit mf: Manifest[T]) = new OrderedTuplePacker[T] + implicit def caseClassPacker[T <: Product](implicit mf: Manifest[T]): OrderedTuplePacker[T] = new OrderedTuplePacker[T] } trait LowPriorityTuplePackers extends java.io.Serializable { - implicit def genericTuplePacker[T: Manifest] = new ReflectionTuplePacker[T] + implicit def genericTuplePacker[T: Manifest]: ReflectionTuplePacker[T] = new ReflectionTuplePacker[T] } /** diff --git a/scalding-core/src/main/scala/com/twitter/scalding/TupleUnpacker.scala b/scalding-core/src/main/scala/com/twitter/scalding/TupleUnpacker.scala index fbef771c81..ce6e367860 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/TupleUnpacker.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/TupleUnpacker.scala @@ -35,7 +35,7 @@ trait TupleUnpacker[T] extends java.io.Serializable { } trait LowPriorityTupleUnpackers { - implicit def genericUnpacker[T: Manifest] = new ReflectionTupleUnpacker[T] + implicit def genericUnpacker[T: Manifest]: ReflectionTupleUnpacker[T] = new ReflectionTupleUnpacker[T] } /** diff --git a/scalding-core/src/main/scala/com/twitter/scalding/bdd/BddDsl.scala b/scalding-core/src/main/scala/com/twitter/scalding/bdd/BddDsl.scala index f0c1da4798..2f3c4b57d8 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/bdd/BddDsl.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/bdd/BddDsl.scala @@ -21,7 +21,7 @@ trait BddDsl extends FieldConversions with PipeOperationsConversions { jobTest.source[T](source, data)(setter) } - implicit def fromSimpleTypeDataToSourceWithoutSchema[T](data: Iterable[T])(implicit setter: TupleSetter[T]) = + implicit def fromSimpleTypeDataToSourceWithoutSchema[T](data: Iterable[T])(implicit setter: TupleSetter[T]): SimpleTypeTestSourceWithoutSchema[T] = new SimpleTypeTestSourceWithoutSchema(data)(setter) class TestSource(data: TestSourceWithoutSchema, schema: Fields) { diff --git a/scalding-core/src/main/scala/com/twitter/scalding/bdd/PipeOperationsConversions.scala b/scalding-core/src/main/scala/com/twitter/scalding/bdd/PipeOperationsConversions.scala index 04b336f8dc..ce7d064572 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/bdd/PipeOperationsConversions.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/bdd/PipeOperationsConversions.scala @@ -40,27 +40,27 @@ trait PipeOperationsConversions { def apply(pipes: List[RichPipe]): Pipe = op(pipes.map(_.pipe).toList) } - implicit val fromSingleRichPipeFunctionToOperation = (op: RichPipe => RichPipe) => new OnePipeOperation(op(_).pipe) - implicit val fromSingleRichPipeToPipeFunctionToOperation = (op: RichPipe => Pipe) => new OnePipeOperation(op(_)) + implicit val fromSingleRichPipeFunctionToOperation: (RichPipe => RichPipe) => OnePipeOperation = (op: RichPipe => RichPipe) => new OnePipeOperation(op(_).pipe) + implicit val fromSingleRichPipeToPipeFunctionToOperation: (RichPipe => Pipe) => OnePipeOperation = (op: RichPipe => Pipe) => new OnePipeOperation(op(_)) - implicit val fromTwoRichPipesFunctionToOperation = (op: (RichPipe, RichPipe) => RichPipe) => new TwoPipesOperation(op(_, _).pipe) - implicit val fromTwoRichPipesToRichPipeFunctionToOperation = (op: (RichPipe, RichPipe) => Pipe) => new TwoPipesOperation(op(_, _)) + implicit val fromTwoRichPipesFunctionToOperation: ((RichPipe, RichPipe) => RichPipe) => TwoPipesOperation = (op: (RichPipe, RichPipe) => RichPipe) => new TwoPipesOperation(op(_, _).pipe) + implicit val fromTwoRichPipesToRichPipeFunctionToOperation: ((RichPipe, RichPipe) => Pipe) => TwoPipesOperation = (op: (RichPipe, RichPipe) => Pipe) => new TwoPipesOperation(op(_, _)) - implicit val fromThreeRichPipesFunctionToOperation = (op: (RichPipe, RichPipe, RichPipe) => RichPipe) => new ThreePipesOperation(op(_, _, _).pipe) - implicit val fromThreeRichPipesToPipeFunctionToOperation = (op: (RichPipe, RichPipe, RichPipe) => Pipe) => new ThreePipesOperation(op(_, _, _)) + implicit val fromThreeRichPipesFunctionToOperation: ((RichPipe, RichPipe, RichPipe) => RichPipe) => ThreePipesOperation = (op: (RichPipe, RichPipe, RichPipe) => RichPipe) => new ThreePipesOperation(op(_, _, _).pipe) + implicit val fromThreeRichPipesToPipeFunctionToOperation: ((RichPipe, RichPipe, RichPipe) => Pipe) => ThreePipesOperation = (op: (RichPipe, RichPipe, RichPipe) => Pipe) => new ThreePipesOperation(op(_, _, _)) - implicit val fromRichPipeListFunctionToOperation = (op: List[RichPipe] => RichPipe) => new ListRichPipesOperation(op(_).pipe) - implicit val fromRichPipeListToPipeFunctionToOperation = (op: List[RichPipe] => Pipe) => new ListRichPipesOperation(op(_)) + implicit val fromRichPipeListFunctionToOperation: (List[RichPipe] => RichPipe) => ListRichPipesOperation = (op: List[RichPipe] => RichPipe) => new ListRichPipesOperation(op(_).pipe) + implicit val fromRichPipeListToPipeFunctionToOperation: (List[RichPipe] => Pipe) => ListRichPipesOperation = (op: List[RichPipe] => Pipe) => new ListRichPipesOperation(op(_)) - implicit val fromSinglePipeFunctionToOperation = (op: Pipe => RichPipe) => new OnePipeOperation(op(_).pipe) - implicit val fromSinglePipeToRichPipeFunctionToOperation = (op: Pipe => Pipe) => new OnePipeOperation(op(_)) + implicit val fromSinglePipeFunctionToOperation: (Pipe => RichPipe) => OnePipeOperation = (op: Pipe => RichPipe) => new OnePipeOperation(op(_).pipe) + implicit val fromSinglePipeToRichPipeFunctionToOperation: (Pipe => Pipe) => OnePipeOperation = (op: Pipe => Pipe) => new OnePipeOperation(op(_)) - implicit val fromTwoPipeFunctionToOperation = (op: (Pipe, Pipe) => RichPipe) => new TwoPipesOperation(op(_, _).pipe) - implicit val fromTwoRichPipeToPipeFunctionToOperation = (op: (Pipe, Pipe) => Pipe) => new TwoPipesOperation(op(_, _)) + implicit val fromTwoPipeFunctionToOperation: ((Pipe, Pipe) => RichPipe) => TwoPipesOperation = (op: (Pipe, Pipe) => RichPipe) => new TwoPipesOperation(op(_, _).pipe) + implicit val fromTwoRichPipeToPipeFunctionToOperation: ((Pipe, Pipe) => Pipe) => TwoPipesOperation = (op: (Pipe, Pipe) => Pipe) => new TwoPipesOperation(op(_, _)) - implicit val fromThreePipeFunctionToOperation = (op: (Pipe, Pipe, Pipe) => RichPipe) => new ThreePipesOperation(op(_, _, _).pipe) - implicit val fromThreeRichPipeToPipeFunctionToOperation = (op: (Pipe, Pipe, Pipe) => Pipe) => new ThreePipesOperation(op(_, _, _)) + implicit val fromThreePipeFunctionToOperation: ((Pipe, Pipe, Pipe) => RichPipe) => ThreePipesOperation = (op: (Pipe, Pipe, Pipe) => RichPipe) => new ThreePipesOperation(op(_, _, _).pipe) + implicit val fromThreeRichPipeToPipeFunctionToOperation: ((Pipe, Pipe, Pipe) => Pipe) => ThreePipesOperation = (op: (Pipe, Pipe, Pipe) => Pipe) => new ThreePipesOperation(op(_, _, _)) - implicit val fromListPipeFunctionToOperation = (op: List[Pipe] => RichPipe) => new ListPipesOperation(op(_).pipe) - implicit val fromListRichPipeToPipeFunctionToOperation = (op: List[Pipe] => Pipe) => new ListPipesOperation(op(_)) + implicit val fromListPipeFunctionToOperation: (List[Pipe] => RichPipe) => ListPipesOperation = (op: List[Pipe] => RichPipe) => new ListPipesOperation(op(_).pipe) + implicit val fromListRichPipeToPipeFunctionToOperation: (List[Pipe] => Pipe) => ListPipesOperation = (op: List[Pipe] => Pipe) => new ListPipesOperation(op(_)) } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/bdd/TBddDsl.scala b/scalding-core/src/main/scala/com/twitter/scalding/bdd/TBddDsl.scala index 7e4cc79e8d..7c1ace3474 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/bdd/TBddDsl.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/bdd/TBddDsl.scala @@ -29,7 +29,7 @@ trait TBddDsl extends FieldConversions with TypedPipeOperationsConversions { jobTest.source[T](source, data) } - implicit def fromSimpleTypeToTypedSource[T](data: Iterable[T]) = + implicit def fromSimpleTypeToTypedSource[T](data: Iterable[T]): SimpleTypedTestSource[T] = new SimpleTypedTestSource(data) case class TestCaseGiven1[TypeIn](source: TypedTestSource[TypeIn]) { diff --git a/scalding-core/src/main/scala/com/twitter/scalding/bdd/TypedPipeOperationsConversions.scala b/scalding-core/src/main/scala/com/twitter/scalding/bdd/TypedPipeOperationsConversions.scala index b332348152..409019f1fb 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/bdd/TypedPipeOperationsConversions.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/bdd/TypedPipeOperationsConversions.scala @@ -45,15 +45,15 @@ trait TypedPipeOperationsConversions { override def apply(pipes: List[TypedPipe[_]]): TypedPipe[TypeOut] = op(pipes) } - implicit def fromSingleTypedPipeFunctionToOperation[TypeIn, TypeOut](op: TypedPipe[TypeIn] => TypedPipe[TypeOut]) = + implicit def fromSingleTypedPipeFunctionToOperation[TypeIn, TypeOut](op: TypedPipe[TypeIn] => TypedPipe[TypeOut]): OneTypedPipeOperation[TypeIn, TypeOut] = new OneTypedPipeOperation[TypeIn, TypeOut](op) - implicit def fromTwoTypedPipesFunctionToOperation[TypeIn1, TypeIn2, TypeOut](op: (TypedPipe[TypeIn1], TypedPipe[TypeIn2]) => TypedPipe[TypeOut]) = + implicit def fromTwoTypedPipesFunctionToOperation[TypeIn1, TypeIn2, TypeOut](op: (TypedPipe[TypeIn1], TypedPipe[TypeIn2]) => TypedPipe[TypeOut]): TwoTypedPipesOperation[TypeIn1, TypeIn2, TypeOut] = new TwoTypedPipesOperation[TypeIn1, TypeIn2, TypeOut](op) - implicit def fromThreeTypedPipesFunctionToOperation[TypeIn1, TypeIn2, TypeIn3, TypeOut](op: (TypedPipe[TypeIn1], TypedPipe[TypeIn2], TypedPipe[TypeIn3]) => TypedPipe[TypeOut]) = + implicit def fromThreeTypedPipesFunctionToOperation[TypeIn1, TypeIn2, TypeIn3, TypeOut](op: (TypedPipe[TypeIn1], TypedPipe[TypeIn2], TypedPipe[TypeIn3]) => TypedPipe[TypeOut]): ThreeTypedPipesOperation[TypeIn1, TypeIn2, TypeIn3, TypeOut] = new ThreeTypedPipesOperation[TypeIn1, TypeIn2, TypeIn3, TypeOut](op) - implicit def fromListOfTypedPipesFunctionToOperation[TypeOut](op: List[TypedPipe[_]] => TypedPipe[TypeOut]) = + implicit def fromListOfTypedPipesFunctionToOperation[TypeOut](op: List[TypedPipe[_]] => TypedPipe[TypeOut]): ListOfTypedPipesOperations[TypeOut] = new ListOfTypedPipesOperations[TypeOut](op) } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix.scala b/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix.scala index f94a88626c..be2e0f3d8c 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/mathematics/Matrix.scala @@ -173,8 +173,8 @@ class MatrixMappableExtensions[T](mappable: Mappable[T])(implicit fd: FlowDef, m object Matrix { // If this function is implicit, you can use the PipeExtensions methods on pipe - implicit def pipeExtensions[P <% Pipe](p: P) = new MatrixPipeExtensions(p) - implicit def mappableExtensions[T](mt: Mappable[T])(implicit fd: FlowDef, mode: Mode) = + implicit def pipeExtensions[P <% Pipe](p: P): MatrixPipeExtensions = new MatrixPipeExtensions(p) + implicit def mappableExtensions[T](mt: Mappable[T])(implicit fd: FlowDef, mode: Mode): MatrixMappableExtensions[T] = new MatrixMappableExtensions(mt)(fd, mode) def filterOutZeros[ValT](fSym: Symbol, group: Monoid[ValT])(fpipe: Pipe): Pipe = { @@ -189,7 +189,7 @@ object Matrix { vct.map { tup => (tup._1, tup._2 - avg) } } - implicit def literalToScalar[ValT](v: ValT) = new LiteralScalar(v) + implicit def literalToScalar[ValT](v: ValT): LiteralScalar[ValT] = new LiteralScalar(v) // Converts to Matrix for addition implicit def diagonalToMatrix[RowT, ValT](diag: DiagonalMatrix[RowT, ValT]): Matrix[RowT, RowT, ValT] = { diff --git a/scalding-core/src/main/scala/com/twitter/scalding/mathematics/SizeHint.scala b/scalding-core/src/main/scala/com/twitter/scalding/mathematics/SizeHint.scala index ce461ebd4b..a164be7650 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/mathematics/SizeHint.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/mathematics/SizeHint.scala @@ -16,7 +16,7 @@ limitations under the License. package com.twitter.scalding.mathematics object SizeHint { - implicit val ordering = SizeHintOrdering + implicit val ordering: Ordering[SizeHint] = SizeHintOrdering // Return a sparsity assuming all the diagonal is present, but nothing else def asDiagonal(h: SizeHint): SizeHint = { def make(r: BigInt, c: BigInt) = { diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/WrappedSerialization.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/WrappedSerialization.scala index abd2ba0cbd..a3759980de 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/WrappedSerialization.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/WrappedSerialization.scala @@ -87,7 +87,7 @@ object WrappedSerialization { type ClassSerialization[T] = (Class[T], Serialization[T]) private def getSerializer[U]: Injection[Externalizer[U], String] = { - implicit val initialInj = JavaSerializationInjection[Externalizer[U]] + implicit val initialInj: Injection[Externalizer[U], Array[Byte]] = JavaSerializationInjection[Externalizer[U]] Injection.connect[Externalizer[U], Array[Byte], Base64String, String] } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/GeneratedFlattenGroup.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/GeneratedFlattenGroup.scala index a78eafeaee..458c100dce 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/GeneratedFlattenGroup.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/GeneratedFlattenGroup.scala @@ -20,7 +20,7 @@ object FlattenGroup { def flattenValueTuple: KLL[KEY, (A, B, C)] = nested.mapValues { tup => FlattenGroup.flattenNestedTuple(tup) } } - implicit def toFlattenLeftJoin3[KEY, KLL[KLL_K, +KLL_V] <: KeyedListLike[KLL_K, KLL_V, KLL], A, B, C](nested: KLL[KEY, ((A, B), C)]) = new FlattenLeftJoin3(nested) + implicit def toFlattenLeftJoin3[KEY, KLL[KLL_K, +KLL_V] <: KeyedListLike[KLL_K, KLL_V, KLL], A, B, C](nested: KLL[KEY, ((A, B), C)]): FlattenGroup.FlattenLeftJoin3[KEY, KLL, A, B, C] = new FlattenLeftJoin3(nested) def flattenNestedTuple[A, B, C, D](nested: (((A, B), C), D)): (A, B, C, D) = { val (((a, b), c), d) = nested @@ -31,7 +31,7 @@ object FlattenGroup { def flattenValueTuple: KLL[KEY, (A, B, C, D)] = nested.mapValues { tup => FlattenGroup.flattenNestedTuple(tup) } } - implicit def toFlattenLeftJoin4[KEY, KLL[KLL_K, +KLL_V] <: KeyedListLike[KLL_K, KLL_V, KLL], A, B, C, D](nested: KLL[KEY, (((A, B), C), D)]) = new FlattenLeftJoin4(nested) + implicit def toFlattenLeftJoin4[KEY, KLL[KLL_K, +KLL_V] <: KeyedListLike[KLL_K, KLL_V, KLL], A, B, C, D](nested: KLL[KEY, (((A, B), C), D)]): FlattenGroup.FlattenLeftJoin4[KEY, KLL, A, B, C, D] = new FlattenLeftJoin4(nested) def flattenNestedTuple[A, B, C, D, E](nested: ((((A, B), C), D), E)): (A, B, C, D, E) = { val ((((a, b), c), d), e) = nested @@ -42,7 +42,7 @@ object FlattenGroup { def flattenValueTuple: KLL[KEY, (A, B, C, D, E)] = nested.mapValues { tup => FlattenGroup.flattenNestedTuple(tup) } } - implicit def toFlattenLeftJoin5[KEY, KLL[KLL_K, +KLL_V] <: KeyedListLike[KLL_K, KLL_V, KLL], A, B, C, D, E](nested: KLL[KEY, ((((A, B), C), D), E)]) = new FlattenLeftJoin5(nested) + implicit def toFlattenLeftJoin5[KEY, KLL[KLL_K, +KLL_V] <: KeyedListLike[KLL_K, KLL_V, KLL], A, B, C, D, E](nested: KLL[KEY, ((((A, B), C), D), E)]): FlattenGroup.FlattenLeftJoin5[KEY, KLL, A, B, C, D, E] = new FlattenLeftJoin5(nested) def flattenNestedTuple[A, B, C, D, E, F](nested: (((((A, B), C), D), E), F)): (A, B, C, D, E, F) = { val (((((a, b), c), d), e), f) = nested @@ -53,7 +53,7 @@ object FlattenGroup { def flattenValueTuple: KLL[KEY, (A, B, C, D, E, F)] = nested.mapValues { tup => FlattenGroup.flattenNestedTuple(tup) } } - implicit def toFlattenLeftJoin6[KEY, KLL[KLL_K, +KLL_V] <: KeyedListLike[KLL_K, KLL_V, KLL], A, B, C, D, E, F](nested: KLL[KEY, (((((A, B), C), D), E), F)]) = new FlattenLeftJoin6(nested) + implicit def toFlattenLeftJoin6[KEY, KLL[KLL_K, +KLL_V] <: KeyedListLike[KLL_K, KLL_V, KLL], A, B, C, D, E, F](nested: KLL[KEY, (((((A, B), C), D), E), F)]): FlattenGroup.FlattenLeftJoin6[KEY, KLL, A, B, C, D, E, F] = new FlattenLeftJoin6(nested) def flattenNestedTuple[A, B, C, D, E, F, G](nested: ((((((A, B), C), D), E), F), G)): (A, B, C, D, E, F, G) = { val ((((((a, b), c), d), e), f), g) = nested @@ -147,7 +147,7 @@ object FlattenGroup { def flattenValueTuple: KLL[KEY, (Option[A], Option[B], Option[C])] = nested.mapValues { tup => FlattenGroup.flattenNestedOptionTuple(tup) } } - implicit def toFlattenOuterJoin3[KEY, KLL[KLL_K, +KLL_V] <: KeyedListLike[KLL_K, KLL_V, KLL], A, B, C](nested: KLL[KEY, (Option[(Option[A], Option[B])], Option[C])]) = new FlattenOuterJoin3(nested) + implicit def toFlattenOuterJoin3[KEY, KLL[KLL_K, +KLL_V] <: KeyedListLike[KLL_K, KLL_V, KLL], A, B, C](nested: KLL[KEY, (Option[(Option[A], Option[B])], Option[C])]): FlattenGroup.FlattenOuterJoin3[KEY, KLL, A, B, C] = new FlattenOuterJoin3(nested) def flattenNestedOptionTuple[A, B, C, D](nested: (Option[(Option[(Option[A], Option[B])], Option[C])], Option[D])): (Option[A], Option[B], Option[C], Option[D]) = { val (rest1, d) = nested @@ -160,7 +160,7 @@ object FlattenGroup { def flattenValueTuple: KLL[KEY, (Option[A], Option[B], Option[C], Option[D])] = nested.mapValues { tup => FlattenGroup.flattenNestedOptionTuple(tup) } } - implicit def toFlattenOuterJoin4[KEY, KLL[KLL_K, +KLL_V] <: KeyedListLike[KLL_K, KLL_V, KLL], A, B, C, D](nested: KLL[KEY, (Option[(Option[(Option[A], Option[B])], Option[C])], Option[D])]) = new FlattenOuterJoin4(nested) + implicit def toFlattenOuterJoin4[KEY, KLL[KLL_K, +KLL_V] <: KeyedListLike[KLL_K, KLL_V, KLL], A, B, C, D](nested: KLL[KEY, (Option[(Option[(Option[A], Option[B])], Option[C])], Option[D])]): FlattenGroup.FlattenOuterJoin4[KEY, KLL, A, B, C, D] = new FlattenOuterJoin4(nested) def flattenNestedOptionTuple[A, B, C, D, E](nested: (Option[(Option[(Option[(Option[A], Option[B])], Option[C])], Option[D])], Option[E])): (Option[A], Option[B], Option[C], Option[D], Option[E]) = { val (rest1, e) = nested @@ -174,7 +174,7 @@ object FlattenGroup { def flattenValueTuple: KLL[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E])] = nested.mapValues { tup => FlattenGroup.flattenNestedOptionTuple(tup) } } - implicit def toFlattenOuterJoin5[KEY, KLL[KLL_K, +KLL_V] <: KeyedListLike[KLL_K, KLL_V, KLL], A, B, C, D, E](nested: KLL[KEY, (Option[(Option[(Option[(Option[A], Option[B])], Option[C])], Option[D])], Option[E])]) = new FlattenOuterJoin5(nested) + implicit def toFlattenOuterJoin5[KEY, KLL[KLL_K, +KLL_V] <: KeyedListLike[KLL_K, KLL_V, KLL], A, B, C, D, E](nested: KLL[KEY, (Option[(Option[(Option[(Option[A], Option[B])], Option[C])], Option[D])], Option[E])]): FlattenGroup.FlattenOuterJoin5[KEY, KLL, A, B, C, D, E] = new FlattenOuterJoin5(nested) def flattenNestedOptionTuple[A, B, C, D, E, F](nested: (Option[(Option[(Option[(Option[(Option[A], Option[B])], Option[C])], Option[D])], Option[E])], Option[F])): (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F]) = { val (rest1, f) = nested @@ -189,7 +189,7 @@ object FlattenGroup { def flattenValueTuple: KLL[KEY, (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F])] = nested.mapValues { tup => FlattenGroup.flattenNestedOptionTuple(tup) } } - implicit def toFlattenOuterJoin6[KEY, KLL[KLL_K, +KLL_V] <: KeyedListLike[KLL_K, KLL_V, KLL], A, B, C, D, E, F](nested: KLL[KEY, (Option[(Option[(Option[(Option[(Option[A], Option[B])], Option[C])], Option[D])], Option[E])], Option[F])]) = new FlattenOuterJoin6(nested) + implicit def toFlattenOuterJoin6[KEY, KLL[KLL_K, +KLL_V] <: KeyedListLike[KLL_K, KLL_V, KLL], A, B, C, D, E, F](nested: KLL[KEY, (Option[(Option[(Option[(Option[(Option[A], Option[B])], Option[C])], Option[D])], Option[E])], Option[F])]): FlattenGroup.FlattenOuterJoin6[KEY, KLL, A, B, C, D, E, F] = new FlattenOuterJoin6(nested) def flattenNestedOptionTuple[A, B, C, D, E, F, G](nested: (Option[(Option[(Option[(Option[(Option[(Option[A], Option[B])], Option[C])], Option[D])], Option[E])], Option[F])], Option[G])): (Option[A], Option[B], Option[C], Option[D], Option[E], Option[F], Option[G]) = { val (rest1, g) = nested diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/Grouped.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/Grouped.scala index 9b6a88fcf3..e9f1b2f1cc 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/Grouped.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/Grouped.scala @@ -322,7 +322,7 @@ case class UnsortedIdentityReduce[K, V1]( // If you care which items you take, you should sort by a random number // or the value itself. val fakeOrdering: Ordering[V1] = Ordering.by { v: V1 => v.hashCode } - implicit val mon = new PriorityQueueMonoid[V1](n)(fakeOrdering) + implicit val mon: PriorityQueueMonoid[V1] = new PriorityQueueMonoid[V1](n)(fakeOrdering) // Do the heap-sort on the mappers: val pretake: TypedPipe[(K, V1)] = mapped.mapValues { v: V1 => mon.build(v) } .sumByLocalKeys @@ -408,7 +408,7 @@ case class IdentityValueSortedReduce[K, V1]( // This means don't take anything, which is legal, but strange filterKeys(_ => false) } else { - implicit val mon = new PriorityQueueMonoid[V1](n)(valueSort.asInstanceOf[Ordering[V1]]) + implicit val mon: PriorityQueueMonoid[V1] = new PriorityQueueMonoid[V1](n)(valueSort.asInstanceOf[Ordering[V1]]) // Do the heap-sort on the mappers: val pretake: TypedPipe[(K, V1)] = mapped.mapValues { v: V1 => mon.build(v) } .sumByLocalKeys diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/Sketched.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/Sketched.scala index 3c0f86f0df..027c619929 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/Sketched.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/Sketched.scala @@ -18,6 +18,7 @@ package com.twitter.scalding.typed import com.twitter.algebird.{ Bytes, CMS, CMSHasherImplicits, Batched } import com.twitter.scalding.serialization.macros.impl.BinaryOrdering._ import com.twitter.scalding.serialization.{ OrderedSerialization, OrderedSerialization2 } +import com.twitter.algebird.CMSMonoid import scala.language.experimental.macros @@ -40,10 +41,10 @@ case class Sketched[K, V](pipe: TypedPipe[(K, V)], def reducers = Some(numReducers) - private lazy implicit val cms = CMS.monoid[Bytes](eps, delta, seed) + private lazy implicit val cms: CMSMonoid[Bytes] = CMS.monoid[Bytes](eps, delta, seed) lazy val sketch: TypedPipe[CMS[Bytes]] = { // every 10k items, compact into a CMS to prevent very slow mappers - lazy implicit val batchedSG = Batched.compactingSemigroup[CMS[Bytes]](10000) + lazy implicit val batchedSG: com.twitter.algebird.Semigroup[Batched[CMS[Bytes]]] = Batched.compactingSemigroup[CMS[Bytes]](10000) pipe .map { case (k, _) => ((), Batched(cms.create(Bytes(serialize(k))))) } 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 5e11853df4..a29af24beb 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 @@ -265,7 +265,7 @@ trait TypedPipe[+T] extends Serializable { implicit val ordT: Ordering[U] = ord.asInstanceOf[Ordering[U]] // Semigroup to handle duplicates for a given key might have different values. - implicit val sg = new Semigroup[T] { + implicit val sg: Semigroup[T] = new Semigroup[T] { def plus(a: T, b: T) = b } @@ -473,7 +473,7 @@ trait TypedPipe[+T] extends Serializable { */ def sum[U >: T](implicit plus: Semigroup[U]): ValuePipe[U] = { // every 1000 items, compact. - lazy implicit val batchedSG = Batched.compactingSemigroup[U](1000) + lazy implicit val batchedSG: Semigroup[Batched[U]] = Batched.compactingSemigroup[U](1000) ComputedValue(map { t => ((), Batched[U](t)) } .sumByLocalKeys // remove the Batched before going to the reducers 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..dee964672f 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/CoreTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/CoreTest.scala @@ -1823,9 +1823,9 @@ class CounterJobTest extends WordSpec with Matchers { object DailySuffixTsvJob { val strd1 = "2014-05-01" val strd2 = "2014-05-02" - implicit val tz = DateOps.UTC - implicit val parser = DateParser.default - implicit val dr = DateRange(RichDate(strd1), RichDate(strd2)) + implicit val tz: java.util.TimeZone = DateOps.UTC + implicit val parser: DateParser = DateParser.default + implicit val dr: DateRange = DateRange(RichDate(strd1), RichDate(strd2)) def source(str: String) = DailySuffixTsv(str) } diff --git a/scalding-core/src/test/scala/com/twitter/scalding/ExecutionUtilTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/ExecutionUtilTest.scala index 872ba8f917..b31118e1f9 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/ExecutionUtilTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/ExecutionUtilTest.scala @@ -5,9 +5,9 @@ import org.scalatest.{ Matchers, WordSpec } class ExecutionUtilTest extends WordSpec with Matchers { import ExecutionUtil._ - implicit val tz = DateOps.UTC - implicit val dp = DateParser.default - implicit val dateRange = DateRange.parse("2015-01-01", "2015-01-10") + implicit val tz: java.util.TimeZone = DateOps.UTC + implicit val dp: DateParser = DateParser.default + implicit val dateRange: DateRange = DateRange.parse("2015-01-01", "2015-01-10") def run[T](e: Execution[T]) = e.waitFor(Config.default, Local(true)) diff --git a/scalding-core/src/test/scala/com/twitter/scalding/KryoTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/KryoTest.scala index 7f2977d545..f2d474013d 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/KryoTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/KryoTest.scala @@ -95,7 +95,7 @@ class KryoTest extends WordSpec with Matchers { "KryoSerializers and KryoDeserializers" should { "round trip any non-array object" in { import HyperLogLog._ - implicit val hllmon = new HyperLogLogMonoid(4) + implicit val hllmon: HyperLogLogMonoid = new HyperLogLogMonoid(4) val test = List(1, 2, "hey", (1, 2), Args("--this is --a --b --test 34"), ("hey", "you"), ("slightly", 1L, "longer", 42, "tuple"), @@ -145,7 +145,7 @@ class KryoTest extends WordSpec with Matchers { } "handle Date, RichDate and DateRange" in { import DateOps._ - implicit val tz = PACIFIC + implicit val tz: java.util.TimeZone = PACIFIC val myDate: RichDate = "1999-12-30T14" val simpleDate: java.util.Date = myDate.value val myDateRange = DateRange("2012-01-02", "2012-06-09") diff --git a/scalding-core/src/test/scala/com/twitter/scalding/ReduceOperationsTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/ReduceOperationsTest.scala index 9e9e5e323b..d8f8d2675f 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/ReduceOperationsTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/ReduceOperationsTest.scala @@ -71,7 +71,7 @@ class SortedTakeJob(args: Args) extends Job(args) { } class ApproximateUniqueCountJob(args: Args) extends Job(args) { - implicit def utf8ToBytes(s: String) = com.twitter.bijection.Injection.utf8(s) + implicit def utf8ToBytes(s: String): Array[Byte] = com.twitter.bijection.Injection.utf8(s) try { Tsv("input0", ('category, 'model, 'os)).read diff --git a/scalding-core/src/test/scala/com/twitter/scalding/SourceSpec.scala b/scalding-core/src/test/scala/com/twitter/scalding/SourceSpec.scala index 3d603ae44c..87ae70d9c4 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/SourceSpec.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/SourceSpec.scala @@ -26,8 +26,8 @@ class SourceSpec extends WordSpec with Matchers { "A case class Source" should { "inherit equality properly from TimePathedSource" in { - implicit val tz = DateOps.UTC - implicit val parser = DateParser.default + implicit val tz: java.util.TimeZone = DateOps.UTC + implicit val parser: DateParser = DateParser.default val d1 = RichDate("2012-02-01") val d2 = RichDate("2012-02-02") diff --git a/scalding-core/src/test/scala/com/twitter/scalding/TypedDelimitedTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/TypedDelimitedTest.scala index cd5e563f81..6cfc271250 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/TypedDelimitedTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/TypedDelimitedTest.scala @@ -53,9 +53,9 @@ class TypedOsvJob(args: Args) extends Job(args) { object DailySuffixTypedTsvJob { val strd1 = "2014-05-01" val strd2 = "2014-05-02" - implicit val tz = DateOps.UTC - implicit val parser = DateParser.default - implicit val dr1 = DateRange(RichDate(strd1), RichDate(strd2)) + implicit val tz: java.util.TimeZone = DateOps.UTC + implicit val parser: DateParser = DateParser.default + implicit val dr1: DateRange = DateRange(RichDate(strd1), RichDate(strd2)) def source(str: String) = DailySuffixTypedTsv[(String, Int)](str) diff --git a/scalding-core/src/test/scala/com/twitter/scalding/TypedFieldsTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/TypedFieldsTest.scala index 39829c3dfa..679437faae 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/TypedFieldsTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/TypedFieldsTest.scala @@ -78,7 +78,7 @@ class UntypedFieldsJob(args: Args) extends Job(args) { class TypedFieldsJob(args: Args) extends Job(args) { - implicit val ordering = new Ordering[Opaque] { + implicit val ordering: Ordering[Opaque] = new Ordering[Opaque] { def compare(a: Opaque, b: Opaque) = a.str compare b.str } diff --git a/scalding-core/src/test/scala/com/twitter/scalding/TypedPipeTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/TypedPipeTest.scala index 54c225e79b..4edbda8e17 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/TypedPipeTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/TypedPipeTest.scala @@ -1365,7 +1365,7 @@ class TypedSketchJoinJob(args: Args) extends Job(args) { val zero = TypedPipe.from(TypedText.tsv[(Int, Int)]("input0")) val one = TypedPipe.from(TypedText.tsv[(Int, Int)]("input1")) - implicit def serialize(k: Int) = k.toString.getBytes + implicit def serialize(k: Int): Array[Byte] = k.toString.getBytes zero .sketch(args("reducers").toInt) @@ -1384,7 +1384,7 @@ class TypedSketchLeftJoinJob(args: Args) extends Job(args) { val zero = TypedPipe.from(TypedText.tsv[(Int, Int)]("input0")) val one = TypedPipe.from(TypedText.tsv[(Int, Int)]("input1")) - implicit def serialize(k: Int) = k.toString.getBytes + implicit def serialize(k: Int): Array[Byte] = k.toString.getBytes zero .sketch(args("reducers").toInt) diff --git a/scalding-core/src/test/scala/com/twitter/scalding/TypedSketchJoinJobForEmptyKeysTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/TypedSketchJoinJobForEmptyKeysTest.scala index d9c03bda3c..241790a6d1 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/TypedSketchJoinJobForEmptyKeysTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/TypedSketchJoinJobForEmptyKeysTest.scala @@ -7,7 +7,7 @@ class TypedSketchJoinJobForEmptyKeys(args: Args) extends Job(args) { val leftTypedPipe = TypedPipe.from(List((1, 1111))) val rightTypedPipe = TypedPipe.from(List((3, 3333), (4, 4444))) - implicit def serialize(k: Int) = k.toString.getBytes + implicit def serialize(k: Int): Array[Byte] = k.toString.getBytes leftTypedPipe .sketch(1) .leftJoin(rightTypedPipe) diff --git a/scalding-core/src/test/scala/com/twitter/scalding/mathematics/Matrix2OptimizationTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/mathematics/Matrix2OptimizationTest.scala index a0c20a745e..1cb68472c2 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/mathematics/Matrix2OptimizationTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/mathematics/Matrix2OptimizationTest.scala @@ -36,8 +36,8 @@ class Matrix2OptimizationSpec extends WordSpec with Matchers { import Dsl._ import com.twitter.scalding.Test - implicit val mode = Test(Map()) - implicit val fd = new FlowDef + implicit val mode: Test = Test(Map()) + implicit val fd: FlowDef = new FlowDef val globM = TypedPipe.from(IterableSource(List((1, 2, 3.0), (2, 2, 4.0)))) @@ -190,8 +190,8 @@ class Matrix2OptimizationSpec extends WordSpec with Matchers { object Matrix2Props extends Properties("Matrix2") { import com.twitter.scalding.Test - implicit val mode = Test(Map()) - implicit val fd = new FlowDef + implicit val mode: Test = Test(Map()) + implicit val fd: FlowDef = new FlowDef val globM = TypedPipe.from(IterableSource(List((1, 2, 3.0), (2, 2, 4.0)))) implicit val ring: Ring[Double] = Ring.doubleRing diff --git a/scalding-core/src/test/scala/com/twitter/scalding/typed/BijectedSourceSinkTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/typed/BijectedSourceSinkTest.scala index a24d9772dc..3497cf42f1 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/typed/BijectedSourceSinkTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/typed/BijectedSourceSinkTest.scala @@ -27,7 +27,7 @@ private[typed] object LongIntPacker { class MutatedSourceJob(args: Args) extends Job(args) { import com.twitter.bijection._ - implicit val bij = new AbstractBijection[Long, (Int, Int)] { + implicit val bij: AbstractBijection[Long, (Int, Int)] = new AbstractBijection[Long, (Int, Int)] { override def apply(x: Long) = (LongIntPacker.l(x), LongIntPacker.r(x)) override def invert(y: (Int, Int)) = LongIntPacker.lr(y._1, y._2) } diff --git a/scalding-core/src/test/scala/com/twitter/scalding/typed/NoStackLineNumberTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/typed/NoStackLineNumberTest.scala index 4db59794a4..a4a67be95a 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/typed/NoStackLineNumberTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/typed/NoStackLineNumberTest.scala @@ -30,8 +30,8 @@ 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 fd: FlowDef = new FlowDef + implicit val m: Hdfs = new Hdfs(false, new Configuration) val pipeFut = com.twitter.example.scalding.typed.InAnotherPackage.buildF.map { tp => tp.toPipe('a, 'b) diff --git a/scalding-core/src/test/scala/com/twitter/scalding/typed/TypedPipeDiffTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/typed/TypedPipeDiffTest.scala index ed15643f49..a63faa56d7 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/typed/TypedPipeDiffTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/typed/TypedPipeDiffTest.scala @@ -141,7 +141,7 @@ object TypedPipeDiffLaws { } class TypedPipeDiffLaws extends PropSpec with PropertyChecks with Checkers { - override implicit val generatorDrivenConfig = PropertyCheckConfiguration(minSuccessful = 5) + override implicit val generatorDrivenConfig: PropertyCheckConfiguration = PropertyCheckConfiguration(minSuccessful = 5) property("diffLaws") { check(TypedPipeDiffLaws.diffLaw[Int]) @@ -150,7 +150,7 @@ class TypedPipeDiffLaws extends PropSpec with PropertyChecks with Checkers { property("diffArrayLaws") { - implicit val arbNoOrdering = Arbitrary { + implicit val arbNoOrdering: Arbitrary[Array[NoOrdering]] = Arbitrary { for { strs <- Arbitrary.arbitrary[Array[String]] } yield { @@ -158,7 +158,7 @@ class TypedPipeDiffLaws extends PropSpec with PropertyChecks with Checkers { } } - implicit val arbNoOrderingHashCollision = Arbitrary { + implicit val arbNoOrderingHashCollision: Arbitrary[Array[NoOrderingHashCollisions]] = Arbitrary { for { strs <- Arbitrary.arbitrary[Array[String]] } yield { @@ -181,7 +181,7 @@ class TypedPipeDiffLaws extends PropSpec with PropertyChecks with Checkers { property("diffByGroupLaws") { - implicit val arbNoOrdering = Arbitrary { + implicit val arbNoOrdering: Arbitrary[NoOrdering] = Arbitrary { for { name <- Arbitrary.arbitrary[String] } yield { @@ -189,7 +189,7 @@ class TypedPipeDiffLaws extends PropSpec with PropertyChecks with Checkers { } } - implicit val arbNoOrderingHashCollision = Arbitrary { + implicit val arbNoOrderingHashCollision: Arbitrary[NoOrderingHashCollisions] = Arbitrary { for { name <- Arbitrary.arbitrary[String] } yield { diff --git a/scalding-date/src/main/scala/com/twitter/scalding/RichDate.scala b/scalding-date/src/main/scala/com/twitter/scalding/RichDate.scala index 6aab75a071..d2d7ba57b3 100644 --- a/scalding-date/src/main/scala/com/twitter/scalding/RichDate.scala +++ b/scalding-date/src/main/scala/com/twitter/scalding/RichDate.scala @@ -28,7 +28,7 @@ import java.util.TimeZone */ object RichDate { // Implicits to Java types: - implicit def toDate(rd: RichDate) = rd.value + implicit def toDate(rd: RichDate): Date = rd.value implicit def toCalendar(rd: RichDate)(implicit tz: TimeZone): Calendar = { val cal = Calendar.getInstance(tz) cal.setTime(rd.value) diff --git a/scalding-date/src/test/scala/com/twitter/scalding/DateTest.scala b/scalding-date/src/test/scala/com/twitter/scalding/DateTest.scala index d3924f5642..2c33162ccd 100644 --- a/scalding-date/src/test/scala/com/twitter/scalding/DateTest.scala +++ b/scalding-date/src/test/scala/com/twitter/scalding/DateTest.scala @@ -20,7 +20,7 @@ import java.util.Calendar import java.util.TimeZone class DateTest extends WordSpec { - implicit val tz = DateOps.PACIFIC + implicit val tz: TimeZone = DateOps.PACIFIC implicit def dateParser: DateParser = DateParser.default 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 c2c42584cf..7f84da1fbf 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 @@ -110,8 +110,8 @@ object MultipleGroupByJobData { class MultipleGroupByJob(args: Args) extends Job(args) { import com.twitter.scalding.serialization._ import MultipleGroupByJobData._ - implicit val stringOrdSer = new StringOrderedSerialization() - implicit val stringTup2OrdSer = new OrderedSerialization2(stringOrdSer, stringOrdSer) + implicit val stringOrdSer: OrderedSerialization[String] = new StringOrderedSerialization() + implicit val stringTup2OrdSer: OrderedSerialization[(String, String)] = new OrderedSerialization2(stringOrdSer, stringOrdSer) val otherStream = TypedPipe.from(data).map{ k => (k, k) }.group TypedPipe.from(data) @@ -284,7 +284,7 @@ class GroupedLimitJobWithSteps(args: Args) extends Job(args) { } object OrderedSerializationTest { - implicit val genASGK = Arbitrary { + implicit val genASGK: Arbitrary[NestedCaseClass] = Arbitrary { for { ts <- Arbitrary.arbitrary[Long] b <- Gen.nonEmptyListOf(Gen.alphaNumChar).map (_.mkString) 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..33006b447d 100644 --- a/scalding-repl/src/main/scala/com/twitter/scalding/ReplImplicits.scala +++ b/scalding-repl/src/main/scala/com/twitter/scalding/ReplImplicits.scala @@ -282,7 +282,7 @@ object ReplImplicits extends FieldConversions { * Convert KeyedListLike to enriched ShellTypedPipe * (e.g. allows .snapshot to be called on Grouped, CoGrouped, etc) */ - implicit def keyedListLikeToShellTypedPipe[K, V, T[K, +V] <: KeyedListLike[K, V, T]](kll: KeyedListLike[K, V, T])(implicit state: BaseReplState) = + implicit def keyedListLikeToShellTypedPipe[K, V, T[K, +V] <: KeyedListLike[K, V, T]](kll: KeyedListLike[K, V, T])(implicit state: BaseReplState): ShellTypedPipe[(K, V)] = new ShellTypedPipe(kll.toTypedPipe)(state) /** @@ -309,12 +309,12 @@ object ReplState extends BaseReplState */ object ReplImplicitContext { /** Implicit execution context for using the Execution monad */ - implicit val executionContext = ConcurrentExecutionContext.global + implicit val executionContext: scala.concurrent.ExecutionContextExecutor = ConcurrentExecutionContext.global /** Implicit repl state used for ShellPipes */ - implicit def stateImpl = ReplState + implicit def stateImpl: ReplState.type = ReplState /** Implicit flowDef for this Scalding shell session. */ - implicit def flowDefImpl = ReplState.flowDef + implicit def flowDefImpl: FlowDef = ReplState.flowDef /** Defaults to running in local mode if no mode is specified. */ - implicit def modeImpl = ReplState.mode - implicit def configImpl = ReplState.config + implicit def modeImpl: Mode = ReplState.mode + implicit def configImpl: Config = ReplState.config } diff --git a/scalding-serialization/src/test/scala/com/twitter/scalding/serialization/WriterReaderProperties.scala b/scalding-serialization/src/test/scala/com/twitter/scalding/serialization/WriterReaderProperties.scala index a4e9fab9fd..8705541535 100644 --- a/scalding-serialization/src/test/scala/com/twitter/scalding/serialization/WriterReaderProperties.scala +++ b/scalding-serialization/src/test/scala/com/twitter/scalding/serialization/WriterReaderProperties.scala @@ -63,8 +63,8 @@ object WriterReaderProperties extends Properties("WriterReaderProperties") { def writerReaderCollection[T: Writer: Reader, C <: Iterable[T]: Arbitrary: Equiv](implicit cbf: CanBuildFrom[Nothing, T, C]): Prop = { - implicit val cwriter = Writer.collection[T, C] - implicit val creader = Reader.collection[T, C] + implicit val cwriter: Writer[C] = Writer.collection[T, C] + implicit val creader: Reader[C] = Reader.collection[T, C] writerReader(implicitly[Arbitrary[C]].arbitrary) } diff --git a/scalding-serialization/src/test/scala/com/twitter/scalding/serialization/macros/MacroOrderingProperties.scala b/scalding-serialization/src/test/scala/com/twitter/scalding/serialization/macros/MacroOrderingProperties.scala index 6cc94ce61d..37be34a32b 100644 --- a/scalding-serialization/src/test/scala/com/twitter/scalding/serialization/macros/MacroOrderingProperties.scala +++ b/scalding-serialization/src/test/scala/com/twitter/scalding/serialization/macros/MacroOrderingProperties.scala @@ -316,24 +316,24 @@ class MacroOrderingProperties extends FunSuite with PropertyChecks with Matchers check[Boolean] } test("Test out jl.Boolean") { - implicit val a = arbMap { b: Boolean => java.lang.Boolean.valueOf(b) } + implicit val a: Arbitrary[java.lang.Boolean] = arbMap { b: Boolean => java.lang.Boolean.valueOf(b) } check[java.lang.Boolean] } test("Test out Byte") { check[Byte] } test("Test out jl.Byte") { - implicit val a = arbMap { b: Byte => java.lang.Byte.valueOf(b) } + implicit val a: Arbitrary[java.lang.Byte] = arbMap { b: Byte => java.lang.Byte.valueOf(b) } check[java.lang.Byte] checkCollisions[java.lang.Byte] } test("Test out Short") { check[Short] } test("Test out jl.Short") { - implicit val a = arbMap { b: Short => java.lang.Short.valueOf(b) } + implicit val a: Arbitrary[java.lang.Short] = arbMap { b: Short => java.lang.Short.valueOf(b) } check[java.lang.Short] checkCollisions[java.lang.Short] } test("Test out Char") { check[Char] } test("Test out jl.Char") { - implicit val a = arbMap { b: Char => java.lang.Character.valueOf(b) } + implicit val a: Arbitrary[java.lang.Character] = arbMap { b: Char => java.lang.Character.valueOf(b) } check[java.lang.Character] checkCollisions[java.lang.Character] } @@ -368,26 +368,26 @@ class MacroOrderingProperties extends FunSuite with PropertyChecks with Matchers } test("Test out jl.Integer") { - implicit val a = arbMap { b: Int => java.lang.Integer.valueOf(b) } + implicit val a: Arbitrary[java.lang.Integer] = arbMap { b: Int => java.lang.Integer.valueOf(b) } check[java.lang.Integer] checkCollisions[java.lang.Integer] } test("Test out Float") { check[Float] } test("Test out jl.Float") { - implicit val a = arbMap { b: Float => java.lang.Float.valueOf(b) } + implicit val a: Arbitrary[java.lang.Float] = arbMap { b: Float => java.lang.Float.valueOf(b) } check[java.lang.Float] checkCollisions[java.lang.Float] } test("Test out Long") { check[Long] } test("Test out jl.Long") { - implicit val a = arbMap { b: Long => java.lang.Long.valueOf(b) } + implicit val a: Arbitrary[java.lang.Long] = arbMap { b: Long => java.lang.Long.valueOf(b) } check[java.lang.Long] checkCollisions[java.lang.Long] } test("Test out Double") { check[Double] } test("Test out jl.Double") { - implicit val a = arbMap { b: Double => java.lang.Double.valueOf(b) } + implicit val a: Arbitrary[java.lang.Double] = arbMap { b: Double => java.lang.Double.valueOf(b) } check[java.lang.Double] checkCollisions[java.lang.Double] } @@ -412,27 +412,27 @@ class MacroOrderingProperties extends FunSuite with PropertyChecks with Matchers checkCollisions[List[Float]] } test("Test out Queue[Int]") { - implicit val isa = collectionArb[Queue, Int] + implicit val isa: Arbitrary[Queue[Int]] = collectionArb[Queue, Int] primitiveOrderedBufferSupplier[Queue[Int]] check[Queue[Int]] checkCollisions[Queue[Int]] } test("Test out IndexedSeq[Int]") { - implicit val isa = collectionArb[IndexedSeq, Int] + implicit val isa: Arbitrary[IndexedSeq[Int]] = collectionArb[IndexedSeq, Int] primitiveOrderedBufferSupplier[IndexedSeq[Int]] check[IndexedSeq[Int]] checkCollisions[IndexedSeq[Int]] } test("Test out HashSet[Int]") { import scala.collection.immutable.HashSet - implicit val isa = collectionArb[HashSet, Int] + implicit val isa: Arbitrary[HashSet[Int]] = collectionArb[HashSet, Int] primitiveOrderedBufferSupplier[HashSet[Int]] check[HashSet[Int]] checkCollisions[HashSet[Int]] } test("Test out ListSet[Int]") { import scala.collection.immutable.ListSet - implicit val isa = collectionArb[ListSet, Int] + implicit val isa: Arbitrary[ListSet[Int]] = collectionArb[ListSet, Int] primitiveOrderedBufferSupplier[ListSet[Int]] check[ListSet[Int]] checkCollisions[ListSet[Int]] @@ -526,14 +526,14 @@ class MacroOrderingProperties extends FunSuite with PropertyChecks with Matchers } test("Test out HashMap[Long, Long]") { import scala.collection.immutable.HashMap - implicit val isa = Arbitrary(implicitly[Arbitrary[List[(Long, Long)]]].arbitrary.map(HashMap(_: _*))) + implicit val isa: Arbitrary[HashMap[Long, Long]] = Arbitrary(implicitly[Arbitrary[List[(Long, Long)]]].arbitrary.map(HashMap(_: _*))) primitiveOrderedBufferSupplier[HashMap[Long, Long]] check[HashMap[Long, Long]] checkCollisions[HashMap[Long, Long]] } test("Test out ListMap[Long, Long]") { import scala.collection.immutable.ListMap - implicit val isa = Arbitrary(implicitly[Arbitrary[List[(Long, Long)]]].arbitrary.map(ListMap(_: _*))) + implicit val isa: Arbitrary[ListMap[Long, Long]] = Arbitrary(implicitly[Arbitrary[List[(Long, Long)]]].arbitrary.map(ListMap(_: _*))) primitiveOrderedBufferSupplier[ListMap[Long, Long]] check[ListMap[Long, Long]] checkCollisions[ListMap[Long, Long]] diff --git a/scalding-thrift-macros/src/test/scala/com/twitter/scalding/thrift/macros/PlatformTest.scala b/scalding-thrift-macros/src/test/scala/com/twitter/scalding/thrift/macros/PlatformTest.scala index afc8bea413..74e7010e47 100644 --- a/scalding-thrift-macros/src/test/scala/com/twitter/scalding/thrift/macros/PlatformTest.scala +++ b/scalding-thrift-macros/src/test/scala/com/twitter/scalding/thrift/macros/PlatformTest.scala @@ -43,7 +43,7 @@ class PlatformTest extends WordSpec with Matchers with HadoopSharedPlatformTest import ScroogeGenerators._ - implicit def arbitraryInstanceProvider[T: Arbitrary] = new InstanceProvider[T] { + implicit def arbitraryInstanceProvider[T: Arbitrary]: InstanceProvider[T] = new InstanceProvider[T] { def g(idx: Int) = ScroogeGenerators.dataProvider[T](idx) } From d46948f22e097b5cc961d184789e0fcd5d95fc25 Mon Sep 17 00:00:00 2001 From: Shunsuke Otani Date: Wed, 5 Apr 2017 10:29:16 +0900 Subject: [PATCH 14/16] Simplify match statement and use collection.breakOut (#1661) * Simplify match statement * Use collection.breakOut --- .../com/twitter/scalding/ExecutionContext.scala | 4 ++-- .../com/twitter/scalding/FieldConversions.scala | 12 ++++-------- 2 files changed, 6 insertions(+), 10 deletions(-) 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 8c0497866a..23644b23b5 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/ExecutionContext.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/ExecutionContext.scala @@ -151,10 +151,10 @@ object ExecutionContext { private val LOG: Logger = LoggerFactory.getLogger(ExecutionContext.getClass) private[scalding] def getDesc[T](baseFlowStep: BaseFlowStep[T]): Seq[String] = { - baseFlowStep.getGraph.vertexSet.asScala.toSeq.flatMap(_ match { + baseFlowStep.getGraph.vertexSet.asScala.flatMap { case pipe: Pipe => RichPipe.getPipeDescriptions(pipe) case _ => List() // no descriptions - }) + }(collection.breakOut) } /* * implicit val ec = ExecutionContext.newContext(config) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/FieldConversions.scala b/scalding-core/src/main/scala/com/twitter/scalding/FieldConversions.scala index 44179cdc33..9b91675160 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/FieldConversions.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/FieldConversions.scala @@ -52,10 +52,8 @@ trait LowPriorityFieldConversions { implicit def productToFields(f: Product): Fields = { val fields = new Fields(f.productIterator.map { anyToFieldArg }.toSeq: _*) f.productIterator.foreach { - _ match { - case field: Field[_] => fields.setComparator(field.id, field.ord) - case _ => - } + case field: Field[_] => fields.setComparator(field.id, field.ord) + case _ => } fields } @@ -185,10 +183,8 @@ trait FieldConversions extends LowPriorityFieldConversions { implicit def parseAnySeqToFields[T <: TraversableOnce[Any]](anyf: T): Fields = { val fields = new Fields(anyf.toSeq.map { anyToFieldArg }: _*) anyf.foreach { - _ match { - case field: Field[_] => fields.setComparator(field.id, field.ord) - case _ => - } + case field: Field[_] => fields.setComparator(field.id, field.ord) + case _ => } fields } From 68519c108ddea3dabaf31b624c4c314cf2e2b208 Mon Sep 17 00:00:00 2001 From: Jackson Davis Date: Fri, 7 Apr 2017 10:56:11 -0700 Subject: [PATCH 15/16] Use reflection over Jobs to find serialized classes (#1654) * Use reflection over Jobs to find serialized classes Using scala reflection, we can look at the types of TypedPipe/Grouped etc to identify classes being serialized and automatically assign them compact cascading tokens instead of writing full names. * Job-type-reflection: 2.10 cross-compiling changes PSA stringToTermName is deprecated in 2.11, to might need to revert some of this for 2.12 support * Job-type-reflection: fix private[this] case and test for val from trait * Review fixes * Don't assign a token to a class that gets one from a Serialization * Exclude java + scala arrays & primitives * Review fixes * Register cascading tokenized classes in KryoHadoop Also added a new customRegistrar method so users can add additional default serializers before the tokenized classes registered --- .../scala/com/twitter/scalding/Args.scala | 5 ++ .../scalding/CascadingTokenUpdater.scala | 19 ++++- .../main/scala/com/twitter/scalding/Job.scala | 7 ++ .../scala/com/twitter/scalding/JobTest.scala | 2 +- .../scalding/ReferencedClassFinder.scala | 76 +++++++++++++++++++ .../scalding/serialization/KryoHadoop.scala | 38 ++++++++-- .../scalding/ReferencedClassFinderTest.scala | 71 +++++++++++++++++ 7 files changed, 210 insertions(+), 8 deletions(-) create mode 100644 scalding-core/src/main/scala/com/twitter/scalding/ReferencedClassFinder.scala create mode 100644 scalding-core/src/test/scala/com/twitter/scalding/ReferencedClassFinderTest.scala diff --git a/scalding-args/src/main/scala/com/twitter/scalding/Args.scala b/scalding-args/src/main/scala/com/twitter/scalding/Args.scala index 6d71596b2a..2c487322ae 100644 --- a/scalding-args/src/main/scala/com/twitter/scalding/Args.scala +++ b/scalding-args/src/main/scala/com/twitter/scalding/Args.scala @@ -61,6 +61,11 @@ object Args { case e: NumberFormatException => false } } + + /** + * By default, scalding will use reflection to try and identify classes to tokenize. Set to false to disable + */ + val jobClassReflection = "scalding.job.classreflection" } class Args(val m: Map[String, List[String]]) extends java.io.Serializable { diff --git a/scalding-core/src/main/scala/com/twitter/scalding/CascadingTokenUpdater.scala b/scalding-core/src/main/scala/com/twitter/scalding/CascadingTokenUpdater.scala index f8f0a5da82..d642484568 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/CascadingTokenUpdater.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/CascadingTokenUpdater.scala @@ -15,6 +15,8 @@ limitations under the License. */ package com.twitter.scalding +import cascading.tuple.hadoop.SerializationToken + object CascadingTokenUpdater { private final val lowestAllowed = 128 // cascading rules @@ -55,8 +57,23 @@ object CascadingTokenUpdater { def update(config: Config, clazzes: Set[Class[_]]): Config = { val toks = config.getCascadingSerializationTokens + + val serializations = config.get(Config.IoSerializationsKey).getOrElse("") + val fromSerializations: Seq[String] = if (serializations.isEmpty) + Seq.empty + else + for { + serialization <- serializations.split(",") + clazz = Class.forName(serialization) + tokenAnnotation = clazz.getAnnotation(classOf[SerializationToken]) + if tokenAnnotation != null + className <- tokenAnnotation.classNames() + } yield { + className + } + // We don't want to assign tokens to classes already in the map - val newClasses: Iterable[String] = clazzes.map { _.getName } -- toks.values + val newClasses: Iterable[String] = clazzes.map { _.getName } -- fromSerializations -- toks.values config + (Config.CascadingSerializationTokens -> toksToString(toks ++ assignTokens(firstAvailableToken(toks), newClasses))) } 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 85d347d1c6..e64a637620 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Job.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Job.scala @@ -188,6 +188,7 @@ class Job(val args: Args) extends FieldConversions with java.io.Serializable { defaultComparator.map(init.setDefaultComparator) .getOrElse(init) .setSerialization(Right(classOf[serialization.KryoHadoop]), ioSerializations) + .addCascadingClassSerializationTokens(reflectedClasses) .setScaldingVersion .setCascadingAppName(name) .setCascadingAppId(name) @@ -197,6 +198,12 @@ class Job(val args: Args) extends FieldConversions with java.io.Serializable { .toMap.toMap[AnyRef, AnyRef] // linter:ignore the second one is to lift from String -> AnyRef } + def reflectedClasses: Set[Class[_]] = { + if (args.optional(Args.jobClassReflection).map(_.toBoolean).getOrElse(true)) { + ReferencedClassFinder.findReferencedClasses(getClass) + } else Set.empty + } + /** * This is here so that Mappable.toIterator can find an implicit config */ 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 70fb598846..ffc807a829 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/JobTest.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/JobTest.scala @@ -178,7 +178,7 @@ 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[scalding] def initJob(useHadoop: Boolean, job: Option[JobConf] = None): Job = { // Create a global mode to use for testing. val testMode: TestMode = if (useHadoop) { diff --git a/scalding-core/src/main/scala/com/twitter/scalding/ReferencedClassFinder.scala b/scalding-core/src/main/scala/com/twitter/scalding/ReferencedClassFinder.scala new file mode 100644 index 0000000000..0631a5c4c4 --- /dev/null +++ b/scalding-core/src/main/scala/com/twitter/scalding/ReferencedClassFinder.scala @@ -0,0 +1,76 @@ +package com.twitter.scalding + +import com.twitter.scalding.typed.CoGroupable + +import scala.reflect.runtime.universe + +object ReferencedClassFinder { + private val baseContainers = List( + classOf[Execution[_]], + classOf[TypedPipe[_]], + classOf[TypedSink[_]], + classOf[TypedSource[_]], + classOf[CoGroupable[_, _]], + classOf[KeyedList[_, _]]) + + /** + * Add the given type, as well as all referenced types to the cascading tokens list. + * note, for maximal efficiency, you should also register those types with the kryo + * instantiator being used. + */ + def addCascadingTokensFrom(c: Class[_], config: Config): Config = { + CascadingTokenUpdater.update(config, findReferencedClasses(c) + c) + } + + /** + * Reflect over a scalding job to try and identify types it uses so they can be tokenized by cascading. + * Since scala reflection is broken with the Hadoop InterfaceAudiance annotation (see + * https://issues.scala-lang.org/browse/SI-10129), we can't iterate over scalaType.members, so we instead use java + * reflection to iterate over fields to find the ones we care about, and then look those up in scala reflection to + * find the full un-erased type signatures, and try to find types from those. + * + * Note: this not guaranteed to find every used type. Eg, it can't find types used in a step that isn't + * referred to in a field + */ + def findReferencedClasses(outerClass: Class[_]): Set[Class[_]] = { + val scalaPackage = Package.getPackage("scala") + val mirror = universe.runtimeMirror(outerClass.getClassLoader) + val scalaType = mirror.classSymbol(outerClass).toType + (for { + field <- outerClass.getDeclaredFields + if baseContainers.exists(_.isAssignableFrom(field.getType)) + scalaSignature = scalaType.member(universe.stringToTermName(field.getName)).typeSignature + clazz <- getClassesForType(scalaSignature) + /* The scala root package contains a lot of shady stuff, eg compile-time wrappers (scala.Int/Array etc), + * which reflection will present as type parameters. Skip the whole package - chill-hadoop already ensures most + * of the ones we care about (eg tuples) get tokenized in cascading. + */ + if !(clazz.isPrimitive || clazz.isArray || clazz.getPackage.equals(scalaPackage)) + } yield { + clazz + }).toSet + } + + private def getClassesForType(typeSignature: universe.Type): Seq[Class[_]] = typeSignature match { + case universe.TypeRef(_, _, args) => + args.flatMap { generic => + //If the wrapped type is a Tuple, recurse into its types + if (generic.typeSymbol.fullName.startsWith("scala.Tuple")) { + getClassesForType(generic) + } else { + getClassOpt(generic.typeSymbol.fullName) + } + } + //.member returns the accessor method for the variable unless the field is private[this], so inspect the return type + case universe.NullaryMethodType(resultType) => getClassesForType(resultType) + case _ => Nil + } + + private def getClassOpt(name: String): Option[Class[_]] = { + try { + Some(Class.forName(name)) + } catch { + case _: ClassNotFoundException => None + } + } +} diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala index fa3ca48bc6..49c48c67d0 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/KryoHadoop.scala @@ -17,14 +17,10 @@ package com.twitter.scalding.serialization import com.esotericsoftware.kryo.Kryo import com.esotericsoftware.kryo.serializers.FieldSerializer - -import com.twitter.scalding.DateRange -import com.twitter.scalding.RichDate -import com.twitter.scalding.Args - +import com.twitter.scalding.{ Args, CascadingTokenUpdater, DateRange, RichDate, Config => ScaldingConfig } import com.twitter.chill.algebird._ import com.twitter.chill.config.Config -import com.twitter.chill.{ SingletonSerializer, ScalaKryoInstantiator, KryoInstantiator } +import com.twitter.chill.{ IKryoRegistrar, KryoInstantiator, ScalaKryoInstantiator, SingletonSerializer } class KryoHadoop(@transient config: Config) extends KryoInstantiator { // keeping track of references is costly for memory, and often triggers OOM on Hadoop @@ -89,6 +85,36 @@ class KryoHadoop(@transient config: Config) extends KryoInstantiator { val classLoader = Thread.currentThread.getContextClassLoader newK.setClassLoader(classLoader) + customRegistrar(newK) + + /** + * Register any cascading tokenized classes not already registered + */ + val tokenizedClasses = CascadingTokenUpdater.parseTokens(config.get(ScaldingConfig.CascadingSerializationTokens)).values + for { + className <- tokenizedClasses + clazz <- getClassOpt(className) + if !newK.alreadyRegistered(clazz) + } { + newK.register(clazz) + } + newK } + + private def getClassOpt(name: String): Option[Class[_]] = { + try { + Some(Class.forName(name)) + } catch { + case _: ClassNotFoundException => None + } + } + + /** + * If you override KryoHadoop, prefer to add registrations here instead of overriding [[newKryo]]. + * That way, any additional default serializers will be used for registering cascading tokenized classes. + */ + def customRegistrar: IKryoRegistrar = new IKryoRegistrar { + override def apply(k: Kryo): Unit = {} + } } diff --git a/scalding-core/src/test/scala/com/twitter/scalding/ReferencedClassFinderTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/ReferencedClassFinderTest.scala new file mode 100644 index 0000000000..7acebcbc3d --- /dev/null +++ b/scalding-core/src/test/scala/com/twitter/scalding/ReferencedClassFinderTest.scala @@ -0,0 +1,71 @@ +package com.twitter.scalding + +import com.twitter.chill._ +import com.twitter.chill.config.{ ConfiguredInstantiator, ScalaMapConfig } +import org.apache.hadoop.io.BytesWritable +import org.scalatest.{ Matchers, WordSpec } + +case class C1(a: Int) +case class C2(b: Int) +case class C3(c: Int) +case class C4(d: Int) + +trait TraitType { + val tp2 = TypedPipe.from(List(C4(0), C4(1))) +} + +class ReferencedClassFinderExample(args: Args) extends Job(args) with TraitType { + case class C5(e: Int) + + val tp = TypedPipe.from(List(C1(1), C1(1), C1(2), C1(3), C1(5))) + val grouped = tp.groupBy(c => C2(c.a))(new Ordering[C2] { + override def compare(a: C2, b: C2) = b.b - a.b + }) + // Verify that we can inspect private[this] fields + private[this] val withTuple = grouped.toList.mapValues(list => C3(list.length)) + // Verify that we don't assign a >= 128 token to a class that has a < 128 token + val bw = TypedPipe.from(List(new BytesWritable(Array[Byte](0, 1, 2)))) + // Verify we don't tokenize scala's array & primitive wrappers. + val ints = TypedPipe.from(List(0, 1, 2)) + val arr = TypedPipe.from(List(Array(0L), Array(1L), Array(2L))) + // Inner classes don't work because fullName doesn't have the $ for the inner class, but verify we catch the + // ClassNotFoundException + val innerClass = TypedPipe.from(List(C5(2), C5(3), C5(5), C5(8))) + + withTuple.write(TypedTsv[(C2, C3)](args("output"))) +} + +class ReferencedClassFinderTest extends WordSpec with Matchers { + "JobClassFinder" should { + "Identify and tokenize used case classes" in { + val job = JobTest(new ReferencedClassFinderExample(_)) + .arg("output", "outputFile") + .sink[(C2, C3)](TypedTsv[(C2, C3)]("outputFile")){ _: Any => Unit }.initJob(false) + val config = Config.tryFrom(job.config).get + val tokenizedClasses = config.getCascadingSerializationTokens.values.toSet + val kryoRegisteredClasses = config.getKryoRegisteredClasses + + tokenizedClasses should contain(classOf[C1].getName) + tokenizedClasses should contain(classOf[C2].getName) + tokenizedClasses should contain(classOf[C3].getName) + tokenizedClasses should contain(classOf[C4].getName) + kryoRegisteredClasses should contain(classOf[C1]) + kryoRegisteredClasses should contain(classOf[C2]) + kryoRegisteredClasses should contain(classOf[C3]) + kryoRegisteredClasses should contain(classOf[C4]) + + tokenizedClasses should not contain (classOf[BytesWritable].getName) + kryoRegisteredClasses should not contain (classOf[BytesWritable]) + // classOf[Int] will return the primitive int, so manually pass in scala's wrapper + tokenizedClasses should not contain ("scala.Int") + tokenizedClasses should not contain ("scala.Array") + } + + "Run successfully" in { + JobTest(new ReferencedClassFinderExample(_)) + .arg("output", "outputFile") + .sink[(C2, C3)](TypedTsv[(C2, C3)]("outputFile")){ _: Any => Unit } + .runHadoop + } + } +} \ No newline at end of file From 5d5dd548174b4b4f83f27831d93bf09a0affb736 Mon Sep 17 00:00:00 2001 From: "P. Oscar Boykin" Date: Fri, 7 Apr 2017 08:54:22 -1000 Subject: [PATCH 16/16] Make a new method private (#1662) --- .../scalding/commons/scheme/CombinedSequenceFileScheme.scala | 5 ++--- scalding-core/src/main/scala/com/twitter/scalding/Job.scala | 4 ++-- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/scheme/CombinedSequenceFileScheme.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/scheme/CombinedSequenceFileScheme.scala index ee22414ec7..a987ca0e11 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/scheme/CombinedSequenceFileScheme.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/scheme/CombinedSequenceFileScheme.scala @@ -1,9 +1,8 @@ package com.twitter.scalding.commons.scheme import cascading.scheme.Scheme -import com.twitter.elephantbird.cascading2.scheme.{CombinedSequenceFile, CombinedWritableSequenceFile} -import com.twitter.scalding.{HadoopSchemeInstance, SequenceFileScheme, WritableSequenceFileScheme} - +import com.twitter.elephantbird.cascading2.scheme.{ CombinedSequenceFile, CombinedWritableSequenceFile } +import com.twitter.scalding.{ HadoopSchemeInstance, SequenceFileScheme, WritableSequenceFileScheme } trait CombinedSequenceFileScheme extends SequenceFileScheme { // TODO Cascading doesn't support local mode yet 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 e64a637620..5642251b01 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Job.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Job.scala @@ -198,11 +198,11 @@ class Job(val args: Args) extends FieldConversions with java.io.Serializable { .toMap.toMap[AnyRef, AnyRef] // linter:ignore the second one is to lift from String -> AnyRef } - def reflectedClasses: Set[Class[_]] = { + private def reflectedClasses: Set[Class[_]] = if (args.optional(Args.jobClassReflection).map(_.toBoolean).getOrElse(true)) { ReferencedClassFinder.findReferencedClasses(getClass) } else Set.empty - } + /** * This is here so that Mappable.toIterator can find an implicit config