diff --git a/.gitignore b/.gitignore index a8990a326..f213f9276 100644 --- a/.gitignore +++ b/.gitignore @@ -14,3 +14,4 @@ project/plugins/src_managed/ */.settings */.cache */.classpath +vagrant/ diff --git a/CHANGES.md b/CHANGES.md index 1e29374a8..64ae18618 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,5 +1,8 @@ # summingbird # +## 0.4.1 +* Release tool failure fix from 0.4.0 release. + ## 0.4.0 * Add/use the multiplier option: https://github.com/twitter/summingbird/pull/481 * add variations on pipeFactory to allow a map function before time extrac...: https://github.com/twitter/summingbird/pull/482 diff --git a/README.md b/README.md index 4841eb045..2aa37ad26 100644 --- a/README.md +++ b/README.md @@ -102,7 +102,7 @@ Follow [@summingbird](https://twitter.com/summingbird) on Twitter for updates. ## Maven -Summingbird modules are published on maven central. The current groupid and version for all modules is, respectively, `"com.twitter"` and `0.4.0`. +Summingbird modules are published on maven central. The current groupid and version for all modules is, respectively, `"com.twitter"` and `0.4.1`. Current published artifacts are diff --git a/project/Build.scala b/project/Build.scala index eb6f549d8..6d7303635 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -18,9 +18,11 @@ object SummingbirdBuild extends Build { case version if version startsWith "2.10" => "org.specs2" %% "specs2" % "1.13" % "test" } - val sharedSettings = Project.defaultSettings ++ Seq( + val extraSettings = Project.defaultSettings ++ mimaDefaultSettings + + val sharedSettings = extraSettings ++ Seq( organization := "com.twitter", - version := "0.4.0", + version := "0.4.1", scalaVersion := "2.9.3", crossScalaVersions := Seq("2.9.3", "2.10.0"), libraryDependencies ++= Seq( @@ -110,22 +112,25 @@ object SummingbirdBuild extends Build { ).aggregate( summingbirdCore, summingbirdBatch, + summingbirdBatchHadoop, summingbirdOnline, summingbirdClient, summingbirdStorm, + summingbirdStormTest, summingbirdScalding, + summingbirdScaldingTest, summingbirdBuilder, summingbirdChill, summingbirdExample ) val dfsDatastoresVersion = "1.3.4" - val bijectionVersion = "0.6.0" - val algebirdVersion = "0.3.1" - val scaldingVersion = "0.9.0rc4" - val storehausVersion = "0.8.0" + val bijectionVersion = "0.6.2" + val algebirdVersion = "0.5.0" + val scaldingVersion = "0.9.0rc15" + val storehausVersion = "0.9.0" val utilVersion = "6.3.8" - val chillVersion = "0.3.5" + val chillVersion = "0.3.6" val tormentaVersion = "0.7.0" lazy val slf4jVersion = "1.6.6" @@ -139,7 +144,7 @@ object SummingbirdBuild extends Build { def youngestForwardCompatible(subProj: String) = Some(subProj) .filterNot(unreleasedModules.contains(_)) - .map { s => "com.twitter" % ("summingbird-" + s + "_2.9.3") % "0.2.4" } + .map { s => "com.twitter" % ("summingbird-" + s + "_2.9.3") % "0.4.1" } def module(name: String) = { val id = "summingbird-%s".format(name) @@ -152,7 +157,8 @@ object SummingbirdBuild extends Build { lazy val summingbirdBatch = module("batch").settings( libraryDependencies ++= Seq( "com.twitter" %% "algebird-core" % algebirdVersion, - "com.twitter" %% "bijection-core" % bijectionVersion + "com.twitter" %% "bijection-core" % bijectionVersion, + "com.twitter" %% "scalding-date" % scaldingVersion ) ) @@ -190,8 +196,7 @@ object SummingbirdBuild extends Build { withCross("com.twitter" %% "util-core" % utilVersion) ) ).dependsOn( - summingbirdCore % "test->test;compile->compile", - summingbirdBatch + summingbirdCore % "test->test;compile->compile" ) lazy val summingbirdStorm = module("storm").settings( @@ -216,6 +221,22 @@ object SummingbirdBuild extends Build { summingbirdBatch ) + lazy val summingbirdStormTest = module("storm-test").settings( + parallelExecution in Test := false, + libraryDependencies ++= Seq( + "com.twitter" %% "algebird-core" % algebirdVersion, + "com.twitter" %% "bijection-core" % bijectionVersion, + "com.twitter" %% "storehaus-core" % storehausVersion, + "com.twitter" %% "storehaus-algebra" % storehausVersion, + "com.twitter" %% "tormenta-core" % tormentaVersion, + withCross("com.twitter" %% "util-core" % utilVersion), + "storm" % "storm" % "0.9.0-wip15" % "provided" + ) + ).dependsOn( + summingbirdCore % "test->test;compile->compile", + summingbirdStorm + ) + lazy val summingbirdScalding = module("scalding").settings( libraryDependencies ++= Seq( "com.backtype" % "dfs-datastores" % dfsDatastoresVersion, @@ -234,6 +255,30 @@ object SummingbirdBuild extends Build { ).dependsOn( summingbirdCore % "test->test;compile->compile", summingbirdChill, + summingbirdBatchHadoop, + summingbirdBatch + ) + + lazy val summingbirdScaldingTest = module("scalding-test").settings( + libraryDependencies ++= Seq( + "org.scalacheck" %% "scalacheck" % "1.10.0" + ) + ).dependsOn( + summingbirdCore % "test->test;compile->compile", + summingbirdChill, + summingbirdBatchHadoop, + summingbirdScalding + ) + + lazy val summingbirdBatchHadoop = module("batch-hadoop").settings( + libraryDependencies ++= Seq( + "com.backtype" % "dfs-datastores" % dfsDatastoresVersion, + "com.twitter" %% "algebird-core" % algebirdVersion, + "com.twitter" %% "bijection-json" % bijectionVersion, + "com.twitter" %% "scalding-date" % scaldingVersion + ) + ).dependsOn( + summingbirdCore % "test->test;compile->compile", summingbirdBatch ) diff --git a/project/plugins.sbt b/project/plugins.sbt index a97c98c69..d7bdf720f 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -1,3 +1,5 @@ +resolvers += Resolver.url("sbt-plugin-releases", new URL("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases/"))(Resolver.ivyStylePatterns) + resolvers ++= Seq( "jgit-repo" at "http://download.eclipse.org/jgit/maven", "sonatype-releases" at "http://oss.sonatype.org/content/repositories/releases" diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/ScaldingConfig.scala b/summingbird-batch-hadoop/src/main/scala/com/twitter/summingbird/batch/BatchConfig.scala similarity index 94% rename from summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/ScaldingConfig.scala rename to summingbird-batch-hadoop/src/main/scala/com/twitter/summingbird/batch/BatchConfig.scala index b709692a5..39e869f9c 100644 --- a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/ScaldingConfig.scala +++ b/summingbird-batch-hadoop/src/main/scala/com/twitter/summingbird/batch/BatchConfig.scala @@ -14,13 +14,13 @@ limitations under the License. */ -package com.twitter.summingbird.scalding +package com.twitter.summingbird.batch import scala.collection.JavaConverters._ import org.apache.hadoop.conf.Configuration import com.twitter.summingbird.{WrappingConfig, ReadableMap} -object ScaldingConfig { +object BatchConfig { def apply(backingConfig: Configuration) = WrappingConfig(new WrappedHadoopConfig(backingConfig)) } diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/ConfigBijection.scala b/summingbird-batch-hadoop/src/main/scala/com/twitter/summingbird/batch/ConfigBijection.scala similarity index 94% rename from summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/ConfigBijection.scala rename to summingbird-batch-hadoop/src/main/scala/com/twitter/summingbird/batch/ConfigBijection.scala index 25b7c24da..a8f03aece 100644 --- a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/ConfigBijection.scala +++ b/summingbird-batch-hadoop/src/main/scala/com/twitter/summingbird/batch/ConfigBijection.scala @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package com.twitter.summingbird.scalding +package com.twitter.summingbird.batch import com.twitter.bijection.Bijection import java.util.{ Map => JMap } @@ -27,7 +27,7 @@ import scala.collection.JavaConverters._ * @author Ashu Singhal */ -object ConfigBijection { +private[summingbird] object ConfigBijection { implicit val fromMap: Bijection[Map[String, AnyRef], Configuration] = new Bijection[Map[String, AnyRef], Configuration] { override def apply(config: Map[String, AnyRef]) = { diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/IteratorSums.scala b/summingbird-batch-hadoop/src/main/scala/com/twitter/summingbird/batch/IteratorSums.scala similarity index 97% rename from summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/IteratorSums.scala rename to summingbird-batch-hadoop/src/main/scala/com/twitter/summingbird/batch/IteratorSums.scala index b73ae78a6..2a99cdb2e 100644 --- a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/IteratorSums.scala +++ b/summingbird-batch-hadoop/src/main/scala/com/twitter/summingbird/batch/IteratorSums.scala @@ -14,12 +14,12 @@ limitations under the License. */ -package com.twitter.summingbird.scalding +package com.twitter.summingbird.batch import com.twitter.algebird.{Semigroup, StatefulSummer} import scala.collection.mutable.ArrayBuffer -object IteratorSums extends java.io.Serializable { +private[summingbird] object IteratorSums extends java.io.Serializable { def sumWith[T](it: Iterator[T], summer: StatefulSummer[T]): Iterator[T] = // this is for MAXIMUM speed. Any ideas to speed it up, say so + benchmark diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/JobState.scala b/summingbird-batch-hadoop/src/main/scala/com/twitter/summingbird/batch/JobState.scala similarity index 96% rename from summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/JobState.scala rename to summingbird-batch-hadoop/src/main/scala/com/twitter/summingbird/batch/JobState.scala index abbbfc813..7110ab7e4 100644 --- a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/JobState.scala +++ b/summingbird-batch-hadoop/src/main/scala/com/twitter/summingbird/batch/JobState.scala @@ -14,7 +14,7 @@ limitations under the License. */ -package com.twitter.summingbird.scalding +package com.twitter.summingbird.batch /** * Job state models the memory of when the next job should diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/option/FlatMapShards.scala b/summingbird-batch-hadoop/src/main/scala/com/twitter/summingbird/batch/option/FlatMapShards.scala similarity index 93% rename from summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/option/FlatMapShards.scala rename to summingbird-batch-hadoop/src/main/scala/com/twitter/summingbird/batch/option/FlatMapShards.scala index e2b396921..172feaa33 100644 --- a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/option/FlatMapShards.scala +++ b/summingbird-batch-hadoop/src/main/scala/com/twitter/summingbird/batch/option/FlatMapShards.scala @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package com.twitter.summingbird.scalding.option +package com.twitter.summingbird.batch.option object FlatMapShards { val default = FlatMapShards(0) diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/option/Reducers.scala b/summingbird-batch-hadoop/src/main/scala/com/twitter/summingbird/batch/option/Reducers.scala similarity index 93% rename from summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/option/Reducers.scala rename to summingbird-batch-hadoop/src/main/scala/com/twitter/summingbird/batch/option/Reducers.scala index 367cb8fbc..3921b39ea 100644 --- a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/option/Reducers.scala +++ b/summingbird-batch-hadoop/src/main/scala/com/twitter/summingbird/batch/option/Reducers.scala @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package com.twitter.summingbird.scalding.option +package com.twitter.summingbird.batch.option object Reducers { val default = Reducers(-1) diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/state/FileVersionTracking.scala b/summingbird-batch-hadoop/src/main/scala/com/twitter/summingbird/batch/state/FileVersionTracking.scala similarity index 92% rename from summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/state/FileVersionTracking.scala rename to summingbird-batch-hadoop/src/main/scala/com/twitter/summingbird/batch/state/FileVersionTracking.scala index 17f9b19af..6ae98c3fc 100644 --- a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/state/FileVersionTracking.scala +++ b/summingbird-batch-hadoop/src/main/scala/com/twitter/summingbird/batch/state/FileVersionTracking.scala @@ -14,7 +14,7 @@ limitations under the License. */ -package com.twitter.summingbird.scalding.state +package com.twitter.summingbird.batch.state import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.fs.Path @@ -24,14 +24,14 @@ import scala.util.{Try, Success, Failure} import org.slf4j.LoggerFactory -object FileVersionTracking { +private[summingbird] object FileVersionTracking { @transient private val logger = LoggerFactory.getLogger(classOf[FileVersionTracking]) val FINISHED_VERSION_SUFFIX = ".version" implicit def path(strPath: String): Path = new Path(strPath) def path(basePath: String, fileName: String): Path = new Path(basePath, fileName) } -case class FileVersionTracking(root: String, fs: FileSystem) { +private[summingbird] case class FileVersionTracking(root: String, fs: FileSystem) { import FileVersionTracking._ fs.mkdirs(root) diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/state/HDFSState.scala b/summingbird-batch-hadoop/src/main/scala/com/twitter/summingbird/batch/state/HDFSState.scala similarity index 91% rename from summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/state/HDFSState.scala rename to summingbird-batch-hadoop/src/main/scala/com/twitter/summingbird/batch/state/HDFSState.scala index 15ee05a8a..b44feae2e 100644 --- a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/state/HDFSState.scala +++ b/summingbird-batch-hadoop/src/main/scala/com/twitter/summingbird/batch/state/HDFSState.scala @@ -13,22 +13,26 @@ See the License for the specific language governing permissions and limitations under the License. */ -package com.twitter.summingbird.scalding.state - -import com.twitter.algebird.ExclusiveLower -import com.twitter.algebird.InclusiveLower -import com.twitter.algebird.InclusiveUpper -import com.twitter.algebird.Lower -import com.twitter.algebird.Upper -import com.twitter.algebird.{ ExclusiveUpper, Intersection, Interval } +package com.twitter.summingbird.batch.state + +import com.twitter.algebird.{ + ExclusiveLower, + ExclusiveUpper, + InclusiveLower, + InclusiveUpper, + Intersection, + Interval, + Lower, + Upper +} import com.twitter.bijection.Conversion.asMethod import com.twitter.summingbird.batch.{ BatchID, Batcher, Timestamp } -import com.twitter.summingbird.scalding.{ PrepareState, RunningState, WaitingState, Scalding } +import com.twitter.summingbird.batch.{ PrepareState, RunningState, WaitingState } import java.util.concurrent.atomic.AtomicBoolean import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.FileSystem -import Scalding.dateRangeInjection +// import Scalding.dateRangeInjection import org.slf4j.LoggerFactory /** @@ -141,7 +145,7 @@ class HDFSState(config: HDFSState.Config)(implicit batcher: Batcher) override def fail(err: Throwable) = throw err } - private class Running(succeedPart: Intersection[Timestamp], bool: AtomicBoolean) + private class Running(succeedPart: Interval.GenIntersection[Timestamp], bool: AtomicBoolean) extends RunningState[Interval[Timestamp]] { def setStopped = assert( bool.compareAndSet(true, false), diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/store/HDFSMetadata.scala b/summingbird-batch-hadoop/src/main/scala/com/twitter/summingbird/batch/store/HDFSMetadata.scala similarity index 96% rename from summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/store/HDFSMetadata.scala rename to summingbird-batch-hadoop/src/main/scala/com/twitter/summingbird/batch/store/HDFSMetadata.scala index ab6a8a253..4aef5ff58 100644 --- a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/store/HDFSMetadata.scala +++ b/summingbird-batch-hadoop/src/main/scala/com/twitter/summingbird/batch/store/HDFSMetadata.scala @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package com.twitter.summingbird.scalding.store +package com.twitter.summingbird.batch.store import com.backtype.hadoop.datastores.{ VersionedStore => BacktypeVersionedStore } import com.twitter.bijection.json.{ JsonInjection, JsonNodeInjection } @@ -54,7 +54,7 @@ import scala.util.control.Exception.allCatch // pull req to scalding-commons that adds this generic capability to // VersionedKeyValSource. -object HDFSMetadata { +private[summingbird] object HDFSMetadata { val METADATA_FILE = "_summingbird.json" def apply(conf: Configuration, rootPath: String): HDFSMetadata = @@ -132,7 +132,7 @@ class HDFSMetadata(conf: Configuration, rootPath: String) { /** Refers to a specific version on disk. Allows reading and writing metadata to specific locations */ -class HDFSVersionMetadata private[store] (val version: Long, conf: Configuration, val path: Path) { +private[summingbird] class HDFSVersionMetadata private[store] (val version: Long, conf: Configuration, val path: Path) { private def getString: Try[String] = Try { val fs = FileSystem.get(conf) diff --git a/summingbird-scalding/src/test/scala/com/twitter/summingbird/scalding/HDFSStateLaws.scala b/summingbird-batch-hadoop/src/test/scala/com/twitter/summingbird/batch/HDFSStateLaws.scala similarity index 94% rename from summingbird-scalding/src/test/scala/com/twitter/summingbird/scalding/HDFSStateLaws.scala rename to summingbird-batch-hadoop/src/test/scala/com/twitter/summingbird/batch/HDFSStateLaws.scala index ea5e0f051..415458b83 100644 --- a/summingbird-scalding/src/test/scala/com/twitter/summingbird/scalding/HDFSStateLaws.scala +++ b/summingbird-batch-hadoop/src/test/scala/com/twitter/summingbird/batch/HDFSStateLaws.scala @@ -14,13 +14,13 @@ limitations under the License. */ -package com.twitter.summingbird.scalding +package com.twitter.summingbird.batch import com.twitter.algebird.Interval import com.twitter.scalding.{DateParser, RichDate} import com.twitter.summingbird._ import com.twitter.summingbird.batch._ -import com.twitter.summingbird.scalding.state.HDFSState +import com.twitter.summingbird.batch.state.HDFSState import java.io.File import java.util.{ TimeZone, UUID } import org.apache.hadoop.conf.Configuration @@ -84,13 +84,15 @@ object HDFSStateLaws extends Specification { val config = HDFSState.Config(path, new Configuration, startDate, numBatches) // Not aligned with batch size - val partialIncompleteInterval: Interval[Timestamp] = Interval.leftClosedRightOpen( + val partialIncompleteInterval: Interval[Timestamp] = Interval.leftClosedRightOpen[Timestamp]( batcher.earliestTimeOf(batcher.batchOf(startDate.get)), - RichDate("2012-12-26T10:40").value) + RichDate("2012-12-26T10:40").value + ).right.get - val partialCompleteInterval: Interval[Timestamp] = Interval.leftClosedRightOpen( + val partialCompleteInterval: Interval[Timestamp] = Interval.leftClosedRightOpen[Timestamp]( batcher.earliestTimeOf(batcher.batchOf(startDate.get)), - RichDate("2012-12-26T11:30").value) + RichDate("2012-12-26T11:30").value + ).right.get val runningState = HDFSState(config).begin.willAccept(partialIncompleteInterval) val waitingState = runningState match { diff --git a/summingbird-scalding/src/test/scala/com/twitter/summingbird/scalding/IteratorSumLaws.scala b/summingbird-batch-hadoop/src/test/scala/com/twitter/summingbird/batch/IteratorSumLaws.scala similarity index 98% rename from summingbird-scalding/src/test/scala/com/twitter/summingbird/scalding/IteratorSumLaws.scala rename to summingbird-batch-hadoop/src/test/scala/com/twitter/summingbird/batch/IteratorSumLaws.scala index b2fe52ff2..babf86ce7 100644 --- a/summingbird-scalding/src/test/scala/com/twitter/summingbird/scalding/IteratorSumLaws.scala +++ b/summingbird-batch-hadoop/src/test/scala/com/twitter/summingbird/batch/IteratorSumLaws.scala @@ -14,7 +14,7 @@ limitations under the License. */ -package com.twitter.summingbird.scalding +package com.twitter.summingbird.batch import org.scalacheck._ import Gen._ diff --git a/summingbird-batch/src/main/scala/com/twitter/summingbird/batch/BatchID.scala b/summingbird-batch/src/main/scala/com/twitter/summingbird/batch/BatchID.scala index 4e7e9608a..835664afe 100644 --- a/summingbird-batch/src/main/scala/com/twitter/summingbird/batch/BatchID.scala +++ b/summingbird-batch/src/main/scala/com/twitter/summingbird/batch/BatchID.scala @@ -17,8 +17,16 @@ limitations under the License. package com.twitter.summingbird.batch import com.twitter.algebird.Monoid -import com.twitter.algebird.{ Universe, Empty, Interval, Intersection, - InclusiveLower, ExclusiveUpper, InclusiveUpper, ExclusiveLower } +import com.twitter.algebird.{ + Empty, + Interval, + Intersection, + InclusiveLower, + ExclusiveUpper, + InclusiveUpper, + ExclusiveLower, + Universe +} import com.twitter.bijection.{ Bijection, Injection } import scala.collection.Iterator.iterate @@ -41,6 +49,8 @@ object BatchID { implicit val equiv: Equiv[BatchID] = Equiv.by(_.id) def apply(long: Long) = new BatchID(long) + def apply(ts: Timestamp) = new BatchID(ts.milliSinceEpoch) + // Enables BatchID(someBatchID.toString) roundtripping def apply(str: String) = new BatchID(str.split("\\.")(1).toLong) @@ -66,14 +76,14 @@ object BatchID { } .flatMap { case (min, max, cnt) => if ((min + cnt) == (max + 1L)) { - Some(Interval.leftClosedRightOpen(min, max.next)) + Some(Interval.leftClosedRightOpen(min, max.next).right.get) } else { // These batches are not contiguous, not an interval None } } - .orElse(Some(Empty())) // there was nothing it iter + .orElse(Some(Empty[BatchID]())) // there was nothing it iter /** Returns all the BatchIDs that are contained in the interval */ diff --git a/summingbird-batch/src/main/scala/com/twitter/summingbird/batch/Batcher.scala b/summingbird-batch/src/main/scala/com/twitter/summingbird/batch/Batcher.scala index 943c3cb11..d21622360 100644 --- a/summingbird-batch/src/main/scala/com/twitter/summingbird/batch/Batcher.scala +++ b/summingbird-batch/src/main/scala/com/twitter/summingbird/batch/Batcher.scala @@ -17,7 +17,7 @@ limitations under the License. package com.twitter.summingbird.batch import com.twitter.algebird.{ Universe, Empty, Interval, Intersection, - InclusiveLower, ExclusiveUpper, InclusiveUpper, ExclusiveLower } + InclusiveLower, ExclusiveUpper, InclusiveUpper, ExclusiveLower, Lower, Upper } import scala.collection.immutable.SortedSet import java.util.{ Comparator, Date } @@ -69,6 +69,9 @@ object Batcher { override val currentBatch = BatchID(0L) def batchOf(t: Timestamp) = currentBatch def earliestTimeOf(batch: BatchID) = Timestamp.Min + + override def latestTimeOf(batch: BatchID) = Timestamp.Max + override def toInterval(b: BatchID): Interval[Timestamp] = if(b == BatchID(0)) Intersection( @@ -157,9 +160,23 @@ trait Batcher extends Serializable { def toInterval(b: BatchID): Interval[Timestamp] = Intersection(InclusiveLower(earliestTimeOf(b)), ExclusiveUpper(earliestTimeOf(b.next))) + def toTimestamp(b: Interval[BatchID]): Interval[Timestamp] = + b match { + case Empty() => Empty[Timestamp]() + case Universe() => Universe[Timestamp]() + case ExclusiveUpper(upper) => ExclusiveUpper(earliestTimeOf(upper)) + case InclusiveUpper(upper) => InclusiveUpper(latestTimeOf(upper)) + case InclusiveLower(lower) => InclusiveLower(earliestTimeOf(lower)) + case ExclusiveLower(lower) => ExclusiveLower(latestTimeOf(lower)) + case Intersection(low, high) => toTimestamp(low) && toTimestamp(high) + } + /** Returns the (inclusive) earliest time of the supplied batch. */ def earliestTimeOf(batch: BatchID): Timestamp + /** Returns the latest time in the given batch */ + def latestTimeOf(batch: BatchID): Timestamp = earliestTimeOf(batch.next).prev + /** Returns the current BatchID. */ def currentBatch: BatchID = batchOf(Timestamp.now) @@ -176,7 +193,7 @@ trait Batcher extends Serializable { */ def enclosedBy(batchID: BatchID, other: Batcher): Iterable[BatchID] = { val earliestInclusive = earliestTimeOf(batchID) - val latestInclusive = earliestTimeOf(batchID.next).prev + val latestInclusive = latestTimeOf(batchID) BatchID.range( other.batchOf(earliestInclusive), other.batchOf(latestInclusive) diff --git a/summingbird-batch/src/main/scala/com/twitter/summingbird/batch/Milliseconds.scala b/summingbird-batch/src/main/scala/com/twitter/summingbird/batch/Milliseconds.scala new file mode 100644 index 000000000..ce9dc1d88 --- /dev/null +++ b/summingbird-batch/src/main/scala/com/twitter/summingbird/batch/Milliseconds.scala @@ -0,0 +1,29 @@ +/* +Copyright 2013 Twitter, Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + +http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package com.twitter.summingbird.batch + +case class Milliseconds(toLong: Long) extends Ordered[Milliseconds] { + def compare(that: Milliseconds) = toLong.compare(that.toLong) + def prev = copy(toLong = toLong - 1) + def next = copy(toLong = toLong + 1) +} + +object Milliseconds { + val Max = Milliseconds(Long.MaxValue) + val Min = Milliseconds(Long.MinValue) + implicit val orderingOnTimestamp: Ordering[Milliseconds] = Ordering.by(_.toLong) +} \ No newline at end of file diff --git a/summingbird-batch/src/main/scala/com/twitter/summingbird/batch/PrunedSpace.scala b/summingbird-batch/src/main/scala/com/twitter/summingbird/batch/PrunedSpace.scala new file mode 100644 index 000000000..0d10a6578 --- /dev/null +++ b/summingbird-batch/src/main/scala/com/twitter/summingbird/batch/PrunedSpace.scala @@ -0,0 +1,31 @@ +/* +Copyright 2014 Twitter, Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + +http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package com.twitter.summingbird.batch + +trait PrunedSpace[-T] extends java.io.Serializable { + // expire (REMOVE) before writing, T is often (K, V) pair + def prune(item: T, writeTime: Timestamp): Boolean +} + +object PrunedSpace extends java.io.Serializable { + val neverPruned: PrunedSpace[Any] = + new PrunedSpace[Any] { def prune(item: Any, writeTime: Timestamp) = false } + + def apply[T](pruneFn: (T, Timestamp) => Boolean) = new PrunedSpace[T] { + def prune(item: T, writeTime: Timestamp) = pruneFn(item, writeTime) + } +} \ No newline at end of file diff --git a/summingbird-batch/src/main/scala/com/twitter/summingbird/batch/TimeBoundedKeySpace.scala b/summingbird-batch/src/main/scala/com/twitter/summingbird/batch/TimeBoundedKeySpace.scala new file mode 100644 index 000000000..23881ddb2 --- /dev/null +++ b/summingbird-batch/src/main/scala/com/twitter/summingbird/batch/TimeBoundedKeySpace.scala @@ -0,0 +1,47 @@ +/* +Copyright 2014 Twitter, Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + +http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package com.twitter.summingbird.batch + +import com.twitter.algebird.{Interval, InclusiveUpper, Empty} + +trait TimeBoundedKeySpace[-K] extends java.io.Serializable { + // if this is true, this key is never updated during the passed interval + // use case: for time series dashboards. The value is kept, but we + // see that we don't need to look for more updates to the key + def isFrozen(key: K, period: Interval[Timestamp]): Boolean +} + +object TimeBoundedKeySpace extends java.io.Serializable { + val neverFrozen: TimeBoundedKeySpace[Any] = + new TimeBoundedKeySpace[Any] { def isFrozen(key: Any, period: Interval[Timestamp]) = false } + + def apply[K](fn: (K, Interval[Timestamp]) => Boolean) = new TimeBoundedKeySpace[K] { + def isFrozen(key: K, period: Interval[Timestamp]) = fn(key, period) + } + + /** + * This is a common case of being able to compute a time after which + * no more writes occur. + */ + def freezesAt[K](ftime: K => Timestamp): TimeBoundedKeySpace[K] = + new TimeBoundedKeySpace[K] { + def isFrozen(key: K, period: Interval[Timestamp]) = { + val liquid = InclusiveUpper(ftime(key)) + (liquid && period) == Empty[Timestamp]() + } + } +} diff --git a/summingbird-batch/src/main/scala/com/twitter/summingbird/batch/TimeStamp.scala b/summingbird-batch/src/main/scala/com/twitter/summingbird/batch/TimeStamp.scala index a5ce7ee59..12708bd35 100644 --- a/summingbird-batch/src/main/scala/com/twitter/summingbird/batch/TimeStamp.scala +++ b/summingbird-batch/src/main/scala/com/twitter/summingbird/batch/TimeStamp.scala @@ -19,13 +19,18 @@ package com.twitter.summingbird.batch import com.twitter.algebird.Monoid import com.twitter.bijection.Bijection import java.util.Date - +import com.twitter.scalding.RichDate case class Timestamp(milliSinceEpoch: Long) extends Ordered[Timestamp] { def compare(that: Timestamp) = milliSinceEpoch.compare(that.milliSinceEpoch) def prev = copy(milliSinceEpoch = milliSinceEpoch - 1) def next = copy(milliSinceEpoch = milliSinceEpoch + 1) def toDate = new Date(milliSinceEpoch) + def toRichDate = new RichDate(milliSinceEpoch) + def -(other: Milliseconds) = Timestamp(milliSinceEpoch - other.toLong) + def +(other: Milliseconds) = Timestamp(milliSinceEpoch + other.toLong) + // Delta between two timestamps + def -(other: Timestamp): Milliseconds = Milliseconds(milliSinceEpoch - other.milliSinceEpoch) def incrementMillis(millis: Long) = Timestamp(milliSinceEpoch + millis) def incrementSeconds(seconds: Long) = Timestamp(milliSinceEpoch + (seconds*1000L)) def incrementMinutes(minutes: Long) = Timestamp(milliSinceEpoch + (minutes*1000*60)) diff --git a/summingbird-batch/src/test/scala/com/twitter/summingbird/batch/BatchLaws.scala b/summingbird-batch/src/test/scala/com/twitter/summingbird/batch/BatchLaws.scala index 43d4f6ff2..409517855 100644 --- a/summingbird-batch/src/test/scala/com/twitter/summingbird/batch/BatchLaws.scala +++ b/summingbird-batch/src/test/scala/com/twitter/summingbird/batch/BatchLaws.scala @@ -53,7 +53,10 @@ object BatchLaws extends Properties("BatchID") { property("range, toInterval and toIterable should be equivalent") = forAll { (b1: BatchID, diff: SmallLong) => val b2 = b1 + (diff.get) - val interval = Interval.leftClosedRightOpen(b1, b2.next) + val interval = Interval.leftClosedRightOpen(b1, b2.next) match { + case Left(i) => i + case Right(i) => i + } (BatchID.toInterval(BatchID.range(b1, b2)) == Some(interval)) && BatchID.toIterable(interval).toList == BatchID.range(b1, b2).toList } diff --git a/summingbird-batch/src/test/scala/com/twitter/summingbird/batch/BatcherLaws.scala b/summingbird-batch/src/test/scala/com/twitter/summingbird/batch/BatcherLaws.scala index 1226c5657..32bda6c81 100644 --- a/summingbird-batch/src/test/scala/com/twitter/summingbird/batch/BatcherLaws.scala +++ b/summingbird-batch/src/test/scala/com/twitter/summingbird/batch/BatcherLaws.scala @@ -57,11 +57,31 @@ object BatcherLaws extends Properties("Batcher") { list == List(b) } + def batchIntervalTransformToTs(batcher: Batcher, intervalGenerator: (BatchID, BatchID) => Interval[BatchID]) = + forAll { (tsA: Timestamp, tsB: Timestamp, deltaMs: Long) => + val (tsLower, tsUpper) = if (tsA < tsB) (tsA, tsB) else (tsB, tsA) + + val deltaBounded = Milliseconds(deltaMs % 1000 * 86000 * 365L) + val int = intervalGenerator(batcher.batchOf(tsLower), batcher.batchOf(tsUpper)) + + val tsInterval = batcher.toTimestamp(int) + + val generatedTS = tsLower + deltaBounded + val generatedBatch = batcher.batchOf(generatedTS) + + // Granularity of the batcher interval is bigger + // So we can't correctly do both intersections together + int.contains(generatedBatch) == tsInterval.contains(generatedTS) + } + def batcherLaws(batcher: Batcher) = earliestIs_<=(batcher) && batchesAreWeakOrderings(batcher) && batchesIncreaseByAtMostOne(batcher) && - batchesCoveredByIdent(batcher) + batchesCoveredByIdent(batcher) && + batchIntervalTransformToTs(batcher, Interval.leftOpenRightClosed(_, _)) && + batchIntervalTransformToTs(batcher, Interval.leftClosedRightOpen(_, _)) + property("UnitBatcher should always return the same batch") = { val batcher = Batcher.unit @@ -140,4 +160,5 @@ object BatcherLaws extends Properties("Batcher") { int.contains(hourlyBatcher.earliestTimeOf(maxBatch.next).prev) } } + } diff --git a/summingbird-builder/src/main/scala/com/twitter/summingbird/Predef.scala b/summingbird-builder/src/main/scala/com/twitter/summingbird/Predef.scala index 9ae344623..906f2706b 100644 --- a/summingbird-builder/src/main/scala/com/twitter/summingbird/Predef.scala +++ b/summingbird-builder/src/main/scala/com/twitter/summingbird/Predef.scala @@ -50,7 +50,7 @@ object Predef { // Offline stores type VersionedStore[K, V] = - com.twitter.summingbird.scalding.VersionedBatchStore[K, V, K, (BatchID, V)] + com.twitter.summingbird.scalding.store.VersionedBatchStore[K, V, K, (BatchID, V)] val VersionedStore = com.twitter.summingbird.scalding.store.VersionedStore // Common options diff --git a/summingbird-builder/src/main/scala/com/twitter/summingbird/builder/CompletedBuilder.scala b/summingbird-builder/src/main/scala/com/twitter/summingbird/builder/CompletedBuilder.scala index 941c45e32..f3dc5bc99 100644 --- a/summingbird-builder/src/main/scala/com/twitter/summingbird/builder/CompletedBuilder.scala +++ b/summingbird-builder/src/main/scala/com/twitter/summingbird/builder/CompletedBuilder.scala @@ -17,14 +17,14 @@ limitations under the License. package com.twitter.summingbird.builder import com.twitter.bijection.Injection -import com.twitter.chill.java.IterableRegistrar import com.twitter.chill.{ InjectionDefaultRegistrar, InjectionRegistrar, IKryoRegistrar } +import com.twitter.chill.java.IterableRegistrar import com.twitter.storehaus.algebra.MergeableStore.enrich +import com.twitter.summingbird.{ Env, KeyedProducer, Options, Platform, Summer } import com.twitter.summingbird.batch.Batcher -import com.twitter.summingbird.{ Env, KeyedProducer, Options, Summer, Platform } +import com.twitter.summingbird.option.CacheSize import com.twitter.summingbird.scalding.Scalding import com.twitter.summingbird.storm.Storm -import com.twitter.summingbird.util.CacheSize import java.io.Serializable diff --git a/summingbird-builder/src/main/scala/com/twitter/summingbird/builder/SourceBuilder.scala b/summingbird-builder/src/main/scala/com/twitter/summingbird/builder/SourceBuilder.scala index b5535fe59..7a83ea460 100644 --- a/summingbird-builder/src/main/scala/com/twitter/summingbird/builder/SourceBuilder.scala +++ b/summingbird-builder/src/main/scala/com/twitter/summingbird/builder/SourceBuilder.scala @@ -23,7 +23,9 @@ import com.twitter.chill.java.IterableRegistrar import com.twitter.storehaus.algebra.MergeableStore import com.twitter.summingbird._ import com.twitter.summingbird.batch.{BatchID, Batcher} -import com.twitter.summingbird.scalding.{Scalding, ScaldingService, ScaldingEnv, BatchedScaldingStore, ScaldingSink} +import com.twitter.summingbird.option.CacheSize +import com.twitter.summingbird.scalding.{Scalding, Service, ScaldingEnv, Sink} +import com.twitter.summingbird.scalding.batch.BatchedStore import com.twitter.summingbird.service.CompoundService import com.twitter.summingbird.sink.{CompoundSink, BatchedSinkFromOffline} import com.twitter.summingbird.source.EventSource @@ -31,7 +33,6 @@ import com.twitter.summingbird.store.CompoundStore import com.twitter.summingbird.storm.{ MergeableStoreSupplier, StoreWrapper, Storm, StormEnv, StormSource, StormSink } -import com.twitter.summingbird.util.CacheSize import java.io.Serializable import java.util.Date @@ -141,7 +142,7 @@ case class SourceBuilder[T: Manifest] private ( * Complete this builder instance with a BatchStore. At this point, * the Summingbird job can be executed on Hadoop. */ - def groupAndSumTo[K, V](store: BatchedScaldingStore[K, V])( + def groupAndSumTo[K, V](store: BatchedStore[K, V])( implicit ev: T <:< (K, V), env: Env, keyMf: Manifest[K], diff --git a/summingbird-builder/src/main/scala/com/twitter/summingbird/builder/package.scala b/summingbird-builder/src/main/scala/com/twitter/summingbird/builder/package.scala deleted file mode 100644 index b913ffdbb..000000000 --- a/summingbird-builder/src/main/scala/com/twitter/summingbird/builder/package.scala +++ /dev/null @@ -1,107 +0,0 @@ -/* -Copyright 2013 Twitter, Inc. - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - -http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package com.twitter.summingbird - -/** - * We put typedefs and vals here to make working with the (deprecated) Builder - * API nicer. - * - * @author Oscar Boykin - * @author Sam Ritchie - * @author Ashu Singhal - */ -package object builder { - - @deprecated("Use com.twitter.summingbird.option.MonoidIsCommutative", "0.1.0") - val MonoidIsCommutative = option.MonoidIsCommutative - - @deprecated("Use com.twitter.summingbird.option.MonoidIsCommutative", "0.1.0") - type MonoidIsCommutative = option.MonoidIsCommutative - - /** - * Storm flatMap option types and objects. - */ - @deprecated("Use com.twitter.summingbird.storm.option.SpoutParallelism", "0.1.0") - type SpoutParallelism = storm.option.SpoutParallelism - - @deprecated("Use com.twitter.summingbird.storm.option.SpoutParallelism", "0.1.0") - val SpoutParallelism = storm.option.SpoutParallelism - - @deprecated("Use com.twitter.summingbird.storm.option.FlatMapParallelism", "0.1.0") - type FlatMapParallelism = storm.option.FlatMapParallelism - - @deprecated("Use com.twitter.summingbird.storm.option.FlatMapParallelism", "0.1.0") - val FlatMapParallelism = storm.option.FlatMapParallelism - - @deprecated("Use com.twitter.summingbird.storm.option.FlatMapStormMetrics", "0.1.0") - type FlatMapStormMetrics = storm.option.FlatMapStormMetrics - - @deprecated("Use com.twitter.summingbird.storm.option.FlatMapStormMetrics", "0.1.0") - val FlatMapStormMetrics = storm.option.FlatMapStormMetrics - - /** - * Sink objects and types. - */ - @deprecated("Use com.twitter.summingbird.storm.option.SinkParallelism", "0.1.0") - type SinkParallelism = storm.option.SinkParallelism - - @deprecated("Use com.twitter.summingbird.storm.option.SinkParallelism", "0.1.0") - val SinkParallelism = storm.option.SinkParallelism - - @deprecated("Use com.twitter.summingbird.storm.option.OnlineSuccessHandler", "0.1.0") - type OnlineSuccessHandler = storm.option.OnlineSuccessHandler - - @deprecated("Use com.twitter.summingbird.storm.option.OnlineSuccessHandler", "0.1.0") - val OnlineSuccessHandler = storm.option.OnlineSuccessHandler - - @deprecated("Use com.twitter.summingbird.storm.option.IncludeSuccessHandler", "0.1.0") - type IncludeSuccessHandler = storm.option.IncludeSuccessHandler - - @deprecated("Use com.twitter.summingbird.storm.option.IncludeSuccessHandler", "0.1.0") - val IncludeSuccessHandler = storm.option.IncludeSuccessHandler - - @deprecated("Use com.twitter.summingbird.storm.option.OnlineExceptionHandler", "0.1.0") - type OnlineExceptionHandler = storm.option.OnlineExceptionHandler - - @deprecated("Use com.twitter.summingbird.storm.option.OnlineExceptionHandler", "0.1.0") - val OnlineExceptionHandler = storm.option.OnlineExceptionHandler - - @deprecated("Use com.twitter.summingbird.storm.option.SummerStormMetrics", "0.1.0") - type SinkStormMetrics = storm.option.SummerStormMetrics - - @deprecated("Use com.twitter.summingbird.storm.option.SummerStormMetrics", "0.1.0") - val SinkStormMetrics = storm.option.SummerStormMetrics - - @deprecated("Use com.twitter.summingbird.storm.option.MaxWaitingFutures", "0.1.0") - type MaxWaitingFutures = storm.option.MaxWaitingFutures - - @deprecated("Use com.twitter.summingbird.storm.option.MaxWaitingFutures", "0.1.0") - val MaxWaitingFutures = storm.option.MaxWaitingFutures - - /** Scalding options here */ - @deprecated("Use com.twitter.summingbird.scalding.option.FlatMapShards", "0.1.0") - val FlatMapShards = scalding.option.FlatMapShards - - @deprecated("Use com.twitter.summingbird.scalding.option.FlatMapShards", "0.1.0") - type FlatMapShards = scalding.option.FlatMapShards - - @deprecated("Use com.twitter.summingbird.scalding.option.Reducers", "0.1.0") - val Reducers = scalding.option.Reducers - - @deprecated("Use com.twitter.summingbird.scalding.option.Reducers", "0.1.0") - type Reducers = scalding.option.Reducers -} diff --git a/summingbird-builder/src/main/scala/com/twitter/summingbird/scalding/ScaldingEnv.scala b/summingbird-builder/src/main/scala/com/twitter/summingbird/scalding/ScaldingEnv.scala index 37b06711a..6c2e7f429 100644 --- a/summingbird-builder/src/main/scala/com/twitter/summingbird/scalding/ScaldingEnv.scala +++ b/summingbird-builder/src/main/scala/com/twitter/summingbird/scalding/ScaldingEnv.scala @@ -17,10 +17,11 @@ limitations under the License. package com.twitter.summingbird.scalding import com.twitter.scalding.{Args, Hdfs, RichDate, DateParser} -import com.twitter.summingbird.scalding.store.HDFSMetadata +import com.twitter.summingbird.batch.store.HDFSMetadata import com.twitter.summingbird.{ Env, Summer, TailProducer, AbstractJob } import com.twitter.summingbird.batch.{ BatchID, Batcher, Timestamp } -import com.twitter.summingbird.builder.{ SourceBuilder, Reducers, CompletedBuilder } +import com.twitter.summingbird.builder.{ SourceBuilder, CompletedBuilder } +import com.twitter.summingbird.batch.option.Reducers import scala.collection.JavaConverters._ import org.apache.hadoop.conf.Configuration @@ -38,7 +39,7 @@ import java.util.TimeZone // the intermediate key-values for using a store as a service // in another job. // Prefer using .write in the -core API. -case class StoreIntermediateData[K, V](sink: ScaldingSink[(K,V)]) extends java.io.Serializable +case class StoreIntermediateData[K, V](sink: Sink[(K,V)]) extends java.io.Serializable // TODO (https://github.com/twitter/summingbird/issues/69): Add // documentation later describing command-line args. start-time, @@ -83,7 +84,7 @@ case class ScaldingEnv(override val jobName: String, inargs: Array[String]) // Used to insert a write just before the store so the store // can be used as a Service private def addDeltaWrite(snode: Summer[Scalding, Any, Any], - sink: ScaldingSink[(Any, Any)]): Summer[Scalding, Any, Any] = { + sink: Sink[(Any, Any)]): Summer[Scalding, Any, Any] = { val Summer(prod, store, monoid) = snode Summer(prod.write(sink), store, monoid) } @@ -115,24 +116,14 @@ case class ScaldingEnv(override val jobName: String, inargs: Array[String]) val scald = Scalding(name, opts) .withRegistrars(ajob.registrars ++ builder.registrar.getRegistrars.asScala) - .withConfigUpdater { - // Set these before the user settings, so that the user - // can change them if needed - - // Make sure we use block compression from mappers to reducers - _.put("mapred.output.compression.type", "BLOCK") - .put("io.compression.codec.lzo.compression.level", "3") - .put("mapred.output.compress", "true") - .put("mapred.compress.map.output", "true") - } .withConfigUpdater{ c => c.updated(ajob.transformConfig(c.toMap)) } - def getStatePath(ss: ScaldingStore[_, _]): Option[String] = + def getStatePath(ss: Store[_, _]): Option[String] = ss match { - case store: VersionedBatchStore[_, _, _, _] => Some(store.rootPath) - case initstore: InitialBatchedStore[_, _] => getStatePath(initstore.proxy) + case store: store.VersionedBatchStore[_, _, _, _] => Some(store.rootPath) + case initstore: store.InitialBatchedStore[_, _] => getStatePath(initstore.proxy) case _ => None } // VersionedState needs this diff --git a/summingbird-builder/src/main/scala/com/twitter/summingbird/scalding/VersionedState.scala b/summingbird-builder/src/main/scala/com/twitter/summingbird/scalding/VersionedState.scala index f4e867323..9a0f4fb9c 100644 --- a/summingbird-builder/src/main/scala/com/twitter/summingbird/scalding/VersionedState.scala +++ b/summingbird-builder/src/main/scala/com/twitter/summingbird/scalding/VersionedState.scala @@ -16,9 +16,21 @@ package com.twitter.summingbird.scalding -import com.twitter.algebird.{ Interval, Intersection, ExclusiveUpper, InclusiveUpper } -import com.twitter.summingbird.batch.{ Batcher, BatchID, Timestamp } -import com.twitter.summingbird.scalding.store.HDFSMetadata +import com.twitter.algebird.{ + InclusiveUpper, + Intersection, + Interval, + ExclusiveUpper +} +import com.twitter.summingbird.batch.{ + Batcher, + BatchID, + PrepareState, + RunningState, + Timestamp, + WaitingState +} +import com.twitter.summingbird.batch.store.HDFSMetadata import org.slf4j.LoggerFactory @@ -61,7 +73,7 @@ private[scalding] class VersionedState(meta: HDFSMetadata, startDate: Option[Tim * batch stored in the most recent metadata file to the current * time. */ - lazy val requested = { + lazy val requested: Interval[Timestamp] = { val beginning: BatchID = startDate.map(batcher.batchOf(_)) .orElse(newestCompleted) @@ -71,7 +83,7 @@ private[scalding] class VersionedState(meta: HDFSMetadata, startDate: Option[Tim Interval.leftClosedRightOpen( batcher.earliestTimeOf(beginning), batcher.earliestTimeOf(end) - ) + ).right.get } def willAccept(available: Interval[Timestamp]) = @@ -94,7 +106,7 @@ private[scalding] class VersionedState(meta: HDFSMetadata, startDate: Option[Tim } } - private class VersionedRunningState(succeedPart: Intersection[Timestamp]) + private class VersionedRunningState(succeedPart: Interval.GenIntersection[Timestamp]) extends RunningState[Interval[Timestamp]] { def nextTime: Timestamp = succeedPart match { diff --git a/summingbird-builder/src/main/scala/com/twitter/summingbird/service/Service.scala b/summingbird-builder/src/main/scala/com/twitter/summingbird/service/Service.scala index efa7ec937..9a7907f8c 100644 --- a/summingbird-builder/src/main/scala/com/twitter/summingbird/service/Service.scala +++ b/summingbird-builder/src/main/scala/com/twitter/summingbird/service/Service.scala @@ -17,21 +17,21 @@ limitations under the License. package com.twitter.summingbird.service import com.twitter.storehaus.ReadableStore -import com.twitter.summingbird.scalding.ScaldingService +import com.twitter.summingbird.scalding.Service /** * Pairing of an online and offline service for use with an * OptionalPlatform2[Scalding, Storm]. */ case class CompoundService[Key, Joined]( - offline: Option[ScaldingService[Key, Joined]], + offline: Option[Service[Key, Joined]], online: Option[() => ReadableStore[Key, Joined]] ) object CompoundService { - def apply[K, J](offline: ScaldingService[K, J], online: => ReadableStore[K, J]): CompoundService[K, J] = + def apply[K, J](offline: Service[K, J], online: => ReadableStore[K, J]): CompoundService[K, J] = CompoundService(Some(offline), Some(() => online)) - def fromOffline[K, J](offline: ScaldingService[K, J]): CompoundService[K, J] = + def fromOffline[K, J](offline: Service[K, J]): CompoundService[K, J] = CompoundService(Some(offline), None) def fromOnline[K, J](online: => ReadableStore[K, J]): CompoundService[K, J] = CompoundService(None, Some(() => online)) diff --git a/summingbird-builder/src/main/scala/com/twitter/summingbird/sink/Sink.scala b/summingbird-builder/src/main/scala/com/twitter/summingbird/sink/Sink.scala index a85e3947f..779e81813 100644 --- a/summingbird-builder/src/main/scala/com/twitter/summingbird/sink/Sink.scala +++ b/summingbird-builder/src/main/scala/com/twitter/summingbird/sink/Sink.scala @@ -18,9 +18,10 @@ package com.twitter.summingbird.sink import cascading.flow.FlowDef -import com.twitter.summingbird.batch.{ BatchID, Batcher } +import com.twitter.summingbird.batch.{ BatchID, Batcher, Timestamp} import com.twitter.scalding.{ Mode, TypedPipe } -import com.twitter.summingbird.scalding.{ScaldingEnv, BatchedScaldingSink} +import com.twitter.summingbird.scalding.ScaldingEnv +import com.twitter.summingbird.scalding.batch.BatchedSink /** * Represents a location to which intermediate results of the @@ -37,7 +38,7 @@ trait OfflineSink[Event] { } /** Wrapped for the new scalding sink API in terms of the above */ -class BatchedSinkFromOffline[T](override val batcher: Batcher, offline: OfflineSink[T]) extends BatchedScaldingSink[T] { +class BatchedSinkFromOffline[T](override val batcher: Batcher, offline: OfflineSink[T]) extends BatchedSink[T] { /** OfflineSink doesn't support reading */ def readStream(batchID: BatchID, mode: Mode) = None @@ -45,7 +46,7 @@ class BatchedSinkFromOffline[T](override val batcher: Batcher, offline: OfflineS /** Instances may choose to write out materialized streams * by implementing this. This is what readStream returns. */ - def writeStream(batchID: BatchID, stream: TypedPipe[(Long, T)])(implicit flowDef: FlowDef, mode: Mode): Unit = { + def writeStream(batchID: BatchID, stream: TypedPipe[(Timestamp, T)])(implicit flowDef: FlowDef, mode: Mode): Unit = { // strip the time offline.write(batchID, stream.values)(flowDef, mode) } diff --git a/summingbird-builder/src/main/scala/com/twitter/summingbird/store/CompoundStore.scala b/summingbird-builder/src/main/scala/com/twitter/summingbird/store/CompoundStore.scala index 9cbdebd14..d55b3d55f 100644 --- a/summingbird-builder/src/main/scala/com/twitter/summingbird/store/CompoundStore.scala +++ b/summingbird-builder/src/main/scala/com/twitter/summingbird/store/CompoundStore.scala @@ -20,7 +20,7 @@ import com.twitter.summingbird.online.Externalizer import com.twitter.storehaus.ReadableStore import com.twitter.storehaus.algebra.MergeableStore import com.twitter.summingbird.batch.BatchID -import com.twitter.summingbird.scalding.BatchedScaldingStore +import com.twitter.summingbird.scalding.batch.BatchedStore /** * Compound BatchStore and MergeableStore, used for building a summingbird job. @@ -30,11 +30,11 @@ import com.twitter.summingbird.scalding.BatchedScaldingStore */ class CompoundStore[K, V] private ( - @transient offline: Option[BatchedScaldingStore[K, V]], + @transient offline: Option[BatchedStore[K, V]], online: Option[() => MergeableStore[(K, BatchID), V]]) extends Serializable { private val offlineBox = Externalizer(offline) - def offlineStore: Option[BatchedScaldingStore[K, V]] = offlineBox.get + def offlineStore: Option[BatchedStore[K, V]] = offlineBox.get def onlineSupplier: Option[() => MergeableStore[(K, BatchID), V]] = online } @@ -42,10 +42,10 @@ object CompoundStore { def fromOnline[K, V](onlineSupplier: => MergeableStore[(K, BatchID), V]): CompoundStore[K, V] = new CompoundStore(None, Some(() => onlineSupplier)) - def fromOffline[K, V](store: BatchedScaldingStore[K, V]): CompoundStore[K, V] = + def fromOffline[K, V](store: BatchedStore[K, V]): CompoundStore[K, V] = new CompoundStore(Some(store), None) - def apply[K, V](offlineStore: BatchedScaldingStore[K, V], onlineSupplier: => MergeableStore[(K, BatchID), V]) + def apply[K, V](offlineStore: BatchedStore[K, V], onlineSupplier: => MergeableStore[(K, BatchID), V]) : CompoundStore[K, V] = new CompoundStore(Some(offlineStore), Some(() => onlineSupplier)) } diff --git a/summingbird-builder/src/test/scala/com/twitter/summingbird/builder/OptionsTest.scala b/summingbird-builder/src/test/scala/com/twitter/summingbird/builder/OptionsTest.scala index 65cd0f04f..75068ccc4 100644 --- a/summingbird-builder/src/test/scala/com/twitter/summingbird/builder/OptionsTest.scala +++ b/summingbird-builder/src/test/scala/com/twitter/summingbird/builder/OptionsTest.scala @@ -16,10 +16,12 @@ limitations under the License. package com.twitter.summingbird.builder -import com.twitter.summingbird.scalding.{Scalding, ConfigBijection, ScaldingEnv, InitialBatchedStore} +import com.twitter.summingbird.batch.ConfigBijection +import com.twitter.summingbird.scalding.{Scalding, ScaldingEnv} +import com.twitter.summingbird.scalding.store.InitialBatchedStore import com.twitter.summingbird._ import com.twitter.summingbird.batch.{Batcher, BatchID} -import com.twitter.summingbird.option._ +import com.twitter.summingbird.option.{MonoidIsCommutative => BMonoidIsCommutative, _} import com.twitter.summingbird.source.EventSource import com.twitter.summingbird.store.CompoundStore import org.specs2.mutable._ @@ -38,7 +40,7 @@ class TestJob1(env: Env) extends AbstractJob(env) { .withTime(new java.util.Date(_)) .map { e => (e % 2, e) } .groupAndSumTo(CompoundStore.fromOffline[Long, Long](new InitialBatchedStore(BatchID(12L), null))) - .set(MonoidIsCommutative(true)) + .set(BMonoidIsCommutative(true)) } catch { case t: Throwable => t.printStackTrace diff --git a/summingbird-core/src/main/scala/com/twitter/summingbird/graph/package.scala b/summingbird-core/src/main/scala/com/twitter/summingbird/graph/package.scala index 6ac52ad80..8b60005da 100644 --- a/summingbird-core/src/main/scala/com/twitter/summingbird/graph/package.scala +++ b/summingbird-core/src/main/scala/com/twitter/summingbird/graph/package.scala @@ -16,6 +16,8 @@ package com.twitter.summingbird +import scala.collection.mutable.{Map => MMap} + /** Collection of graph algorithms */ package object graph { type NeighborFn[T] = (T => Iterable[T]) @@ -62,22 +64,25 @@ package object graph { * Behavior is not defined if the graph is not a DAG (for now, it runs forever, may throw later) */ def dagDepth[T](nodes: Iterable[T])(nf: NeighborFn[T]): Map[T, Int] = { + val acc = MMap[T, Int]() @annotation.tailrec - def computeDepth(todo: Set[T], acc: Map[T, Int]): Map[T, Int] = - if(todo.isEmpty) acc - else { + def computeDepth(todo: Set[T]): Unit = + if(!todo.isEmpty) { def withParents(n: T) = (n :: (nf(n).toList)).filterNot(acc.contains(_)).distinct val (doneThisStep, rest) = todo.map { withParents(_) }.partition { _.size == 1 } - val newAcc = acc ++ (doneThisStep.flatten.map { n => + + acc ++= (doneThisStep.flatten.map { n => val depth = nf(n) //n is done now, so all it's neighbors must be too. .map { acc(_) + 1 } .reduceOption { _ max _ } .getOrElse(0) n -> depth }) - computeDepth(rest.flatten, newAcc) + computeDepth(rest.flatten) } - computeDepth(nodes.toSet, Map.empty) + + computeDepth(nodes.toSet) + acc.toMap } } diff --git a/summingbird-core/src/main/scala/com/twitter/summingbird/option/Commutativity.scala b/summingbird-core/src/main/scala/com/twitter/summingbird/option/Commutativity.scala index 0e479d0fa..68682d3ac 100644 --- a/summingbird-core/src/main/scala/com/twitter/summingbird/option/Commutativity.scala +++ b/summingbird-core/src/main/scala/com/twitter/summingbird/option/Commutativity.scala @@ -21,8 +21,8 @@ import java.io.Serializable // TODO: this functionality should be in algebird: // https://github.com/twitter/algebird/issues/128 sealed trait Commutativity extends Serializable -object NonCommutative extends Commutativity -object Commutative extends Commutativity +case object NonCommutative extends Commutativity +case object Commutative extends Commutativity /** A readable way to specify commutivity in a way * that works with the Class-based Options system. diff --git a/summingbird-example/src/main/scala/com/twitter/summingbird/example/StormRunner.scala b/summingbird-example/src/main/scala/com/twitter/summingbird/example/StormRunner.scala index d326e7f8f..acb752290 100644 --- a/summingbird-example/src/main/scala/com/twitter/summingbird/example/StormRunner.scala +++ b/summingbird-example/src/main/scala/com/twitter/summingbird/example/StormRunner.scala @@ -20,7 +20,7 @@ import com.twitter.summingbird.batch.BatchID import com.twitter.summingbird.Options import com.twitter.summingbird.option.CacheSize import com.twitter.summingbird.storm.{ StormStore, Storm, Executor, StormExecutionConfig} -import com.twitter.summingbird.storm.option.{FlatMapParallelism, SinkParallelism, SpoutParallelism} +import com.twitter.summingbird.storm.option.{FlatMapParallelism, SummerParallelism, SpoutParallelism} import backtype.storm.{Config => BTConfig} import com.twitter.scalding.Args import com.twitter.tormenta.spout.TwitterSpout @@ -121,7 +121,7 @@ object StormRunner { } override def getNamedOptions: Map[String, Options] = Map( - "DEFAULT" -> Options().set(SinkParallelism(2)) + "DEFAULT" -> Options().set(SummerParallelism(2)) .set(FlatMapParallelism(80)) .set(SpoutParallelism(16)) .set(CacheSize(100)) diff --git a/summingbird-online/src/main/scala/com/twitter/summingbird/online/AsyncCache.scala b/summingbird-online/src/main/scala/com/twitter/summingbird/online/AsyncCache.scala index 9bab8d6d8..3a4a7a978 100644 --- a/summingbird-online/src/main/scala/com/twitter/summingbird/online/AsyncCache.scala +++ b/summingbird-online/src/main/scala/com/twitter/summingbird/online/AsyncCache.scala @@ -27,3 +27,7 @@ trait AsyncCache[Key, Value] { def insert(vals: TraversableOnce[(Key, Value)]): Future[Map[Key, Value]] def cleanup: Future[Unit] = Future.Unit } + +trait CacheBuilder[Key, Value] extends Serializable { + def apply(sg: Semigroup[Value]): AsyncCache[Key, Value] +} \ No newline at end of file diff --git a/summingbird-online/src/main/scala/com/twitter/summingbird/online/FlatMapOperation.scala b/summingbird-online/src/main/scala/com/twitter/summingbird/online/FlatMapOperation.scala index e7fe7e70b..fe7ab262a 100644 --- a/summingbird-online/src/main/scala/com/twitter/summingbird/online/FlatMapOperation.scala +++ b/summingbird-online/src/main/scala/com/twitter/summingbird/online/FlatMapOperation.scala @@ -43,13 +43,13 @@ trait FlatMapOperation[-T, +U] extends Serializable with Closeable { // serialization. I think that Kryo mis-serializes that reference. new FlatMapOperation[T, V] { def apply(t: T) = self(t).flatMap { tr => - val next: Seq[Future[TraversableOnce[V]]] = tr.map { fmo.apply(_) }.toSeq + val next: Seq[Future[TraversableOnce[V]]] = tr.map { fmo.apply(_) }.toIndexedSeq Future.collect(next).map(_.flatten) // flatten the inner } override def maybeFlush = { self.maybeFlush.flatMap{ x: TraversableOnce[U] => - val z: Seq[Future[TraversableOnce[V]]] = x.map(fmo.apply(_)).toSeq + val z: IndexedSeq[Future[TraversableOnce[V]]] = x.map(fmo.apply(_)).toIndexedSeq val w: Future[Seq[V]] = Future.collect(z).map(_.flatten) for { ws <- w @@ -69,6 +69,12 @@ class FunctionFlatMapOperation[T, U](@transient fm: T => TraversableOnce[U]) def apply(t: T) = Future.value(boxed.get(t)) } +class GenericFlatMapOperation[T, U](@transient fm: T => Future[TraversableOnce[U]]) + extends FlatMapOperation[T, U] { + val boxed = Externalizer(fm) + def apply(t: T) = boxed.get(t) +} + class FunctionKeyFlatMapOperation[K1, K2, V](@transient fm: K1 => TraversableOnce[K2]) extends FlatMapOperation[(K1, V), (K2, V)] { val boxed = Externalizer(fm) @@ -91,6 +97,9 @@ object FlatMapOperation { def apply[T, U](fm: T => TraversableOnce[U]): FlatMapOperation[T, U] = new FunctionFlatMapOperation(fm) + def generic[T, U](fm: T => Future[TraversableOnce[U]]): FlatMapOperation[T, U] = + new GenericFlatMapOperation(fm) + def keyFlatMap[K1, K2, V](fm: K1 => TraversableOnce[K2]): FlatMapOperation[(K1, V), (K2, V)] = new FunctionKeyFlatMapOperation(fm) @@ -108,11 +117,9 @@ object FlatMapOperation { Future.value(Map.empty) else { // Do the lookup - val mres: Map[K, Future[Option[JoinedV]]] = - store.multiGet(keySet) - Future.collect { - resultList.map { case (k, v) => mres(k).map { k -> (v, _) } } - }.map { _.toMap } + val mres: Map[K, Future[Option[JoinedV]]] = store.multiGet(keySet) + val resultFutures = resultList.map { case (k, v) => mres(k).map { k -> (v, _) } }.toIndexedSeq + Future.collect(resultFutures).map(_.toMap) } } diff --git a/summingbird-online/src/main/scala/com/twitter/summingbird/online/MultiTriggerCache.scala b/summingbird-online/src/main/scala/com/twitter/summingbird/online/MultiTriggerCache.scala index a00555369..894fd7a9d 100644 --- a/summingbird-online/src/main/scala/com/twitter/summingbird/online/MultiTriggerCache.scala +++ b/summingbird-online/src/main/scala/com/twitter/summingbird/online/MultiTriggerCache.scala @@ -35,9 +35,13 @@ import org.slf4j.{LoggerFactory, Logger} */ object MultiTriggerCache { - def builder[Key, Value](cacheSize: CacheSize, valueCombinerCacheSize: ValueCombinerCacheSize, flushFrequency: FlushFrequency, softMemoryFlush: SoftMemoryFlushPercent, poolSize: AsyncPoolSize) = - {(sg: Semigroup[Value]) => - new MultiTriggerCache[Key, Value](cacheSize, valueCombinerCacheSize, flushFrequency, softMemoryFlush, poolSize)(sg) } + def builder[Key, Value](cacheSize: CacheSize, valueCombinerCacheSize: ValueCombinerCacheSize, + flushFrequency: FlushFrequency, softMemoryFlush: SoftMemoryFlushPercent, + poolSize: AsyncPoolSize): CacheBuilder[Key, Value] = + new CacheBuilder[Key, Value] { + def apply(sg: Semigroup[Value]) = + new MultiTriggerCache[Key, Value](cacheSize, valueCombinerCacheSize, flushFrequency, softMemoryFlush, poolSize)(sg) + } } case class MultiTriggerCache[Key, Value](cacheSizeOpt: CacheSize, valueCombinerCacheSize: ValueCombinerCacheSize, flushFrequency: FlushFrequency, softMemoryFlush: SoftMemoryFlushPercent, poolSize: AsyncPoolSize) diff --git a/summingbird-online/src/main/scala/com/twitter/summingbird/online/Queue.scala b/summingbird-online/src/main/scala/com/twitter/summingbird/online/Queue.scala index 05e1dbcdb..23366aa77 100644 --- a/summingbird-online/src/main/scala/com/twitter/summingbird/online/Queue.scala +++ b/summingbird-online/src/main/scala/com/twitter/summingbird/online/Queue.scala @@ -130,9 +130,17 @@ abstract class Queue[T] { case Some(it) => foldLeft(fn(init, it))(fn) } + /** Take all the items currently in the queue */ def toSeq: Seq[T] = trimTo(0) + + /** Take items currently in the queue up to the max sequence size we want to return + Due to threading/muliple readers/writers this will not be an exact size + */ + + def take(maxSeqSize: Int): Seq[T] = trimTo(math.max(size - maxSeqSize, 0)) + /** * Take enough elements to get .size == maxLength */ diff --git a/summingbird-online/src/main/scala/com/twitter/summingbird/online/SummingQueueCache.scala b/summingbird-online/src/main/scala/com/twitter/summingbird/online/SummingQueueCache.scala index 6c853efe4..4e8bea888 100644 --- a/summingbird-online/src/main/scala/com/twitter/summingbird/online/SummingQueueCache.scala +++ b/summingbird-online/src/main/scala/com/twitter/summingbird/online/SummingQueueCache.scala @@ -32,8 +32,10 @@ import org.slf4j.{LoggerFactory, Logger} */ object SummingQueueCache { def builder[Key, Value](cacheSize: CacheSize, flushFrequency: FlushFrequency) = - {(sg: Semigroup[Value]) => - new SummingQueueCache[Key, Value](cacheSize, flushFrequency)(sg) } + new CacheBuilder[Key, Value] { + def apply(sg: Semigroup[Value]) = + new SummingQueueCache[Key, Value](cacheSize, flushFrequency)(sg) + } } case class SummingQueueCache[Key, Value](cacheSizeOpt: CacheSize, flushFrequency: FlushFrequency) @@ -58,5 +60,5 @@ case class SummingQueueCache[Key, Value](cacheSizeOpt: CacheSize, flushFrequency } def insert(vals: TraversableOnce[(Key, Value)]): Future[Map[Key, Value]] = - Future.value(Monoid.sum(vals.map(Map(_)).map(squeue.put(_).getOrElse(Map.empty)))) + Future.value(squeue.put(Monoid.sum(vals.map(Map(_)))).getOrElse(Map.empty)) } diff --git a/summingbird-online/src/main/scala/com/twitter/summingbird/online/executor/AsyncBase.scala b/summingbird-online/src/main/scala/com/twitter/summingbird/online/executor/AsyncBase.scala index a71ba2540..5568d59d4 100644 --- a/summingbird-online/src/main/scala/com/twitter/summingbird/online/executor/AsyncBase.scala +++ b/summingbird-online/src/main/scala/com/twitter/summingbird/online/executor/AsyncBase.scala @@ -16,16 +16,14 @@ limitations under the License. package com.twitter.summingbird.online.executor -import com.twitter.summingbird.batch.Timestamp import com.twitter.summingbird.online.Queue -import com.twitter.summingbird.online.option.{MaxWaitingFutures, MaxFutureWaitTime} +import com.twitter.summingbird.online.option.{MaxWaitingFutures, MaxFutureWaitTime, MaxEmitPerExecute} import com.twitter.util.{Await, Duration, Future} import scala.util.{Try, Success, Failure} import java.util.concurrent.TimeoutException import org.slf4j.{LoggerFactory, Logger} - -abstract class AsyncBase[I,O,S,D](maxWaitingFutures: MaxWaitingFutures, maxWaitingTime: MaxFutureWaitTime) extends Serializable with OperationContainer[I,O,S,D] { +abstract class AsyncBase[I,O,S,D](maxWaitingFutures: MaxWaitingFutures, maxWaitingTime: MaxFutureWaitTime, maxEmitPerExec: MaxEmitPerExecute) extends Serializable with OperationContainer[I,O,S,D] { @transient protected lazy val logger: Logger = LoggerFactory.getLogger(getClass) @@ -33,27 +31,27 @@ abstract class AsyncBase[I,O,S,D](maxWaitingFutures: MaxWaitingFutures, maxWaiti * cases that need to complete operations after or before doing a FlatMapOperation or * doing a store merge */ - def apply(state: S, in: (Timestamp, I)): Future[Iterable[(List[S], Future[TraversableOnce[(Timestamp, O)]])]] - def tick: Future[Iterable[(List[S], Future[TraversableOnce[(Timestamp, O)]])]] = Future.value(Nil) + def apply(state: S, in: I): Future[TraversableOnce[(List[S], Future[TraversableOnce[O]])]] + def tick: Future[TraversableOnce[(List[S], Future[TraversableOnce[O]])]] = Future.value(Nil) private lazy val outstandingFutures = Queue.linkedNonBlocking[Future[Unit]] - private lazy val responses = Queue.linkedNonBlocking[(List[S], Try[TraversableOnce[(Timestamp, O)]])] + private lazy val responses = Queue.linkedNonBlocking[(List[S], Try[TraversableOnce[O]])] override def executeTick = finishExecute(tick.onFailure{ thr => responses.put(((List(), Failure(thr)))) }) - override def execute(state: S, data: (Timestamp, I)) = + override def execute(state: S, data: I) = finishExecute(apply(state, data).onFailure { thr => responses.put(((List(state), Failure(thr)))) }) - private def finishExecute(fIn: Future[Iterable[(List[S], Future[TraversableOnce[(Timestamp, O)]])]]) = { + private def finishExecute(fIn: Future[TraversableOnce[(List[S], Future[TraversableOnce[O]])]]) = { addOutstandingFuture(handleSuccess(fIn).unit) // always empty the responses emptyQueue } - private def handleSuccess(fut: Future[Iterable[(List[S], Future[TraversableOnce[(Timestamp, O)]])]]) = - fut.onSuccess { iter: Iterable[(List[S], Future[TraversableOnce[(Timestamp, O)]])] => + private def handleSuccess(fut: Future[TraversableOnce[(List[S], Future[TraversableOnce[O]])]]) = + fut.onSuccess { iter: TraversableOnce[(List[S], Future[TraversableOnce[O]])] => // Collect the result onto our responses val iterSize = iter.foldLeft(0) { case (iterSize, (tups, res)) => @@ -88,7 +86,7 @@ abstract class AsyncBase[I,O,S,D](maxWaitingFutures: MaxWaitingFutures, maxWaiti private def forceExtraFutures { outstandingFutures.dequeueAll(_.isDefined) - val toForce = outstandingFutures.trimTo(maxWaitingFutures.get) + val toForce = outstandingFutures.trimTo(maxWaitingFutures.get).toIndexedSeq if(!toForce.isEmpty) { try { Await.ready(Future.collect(toForce), maxWaitingTime.get) @@ -104,6 +102,6 @@ abstract class AsyncBase[I,O,S,D](maxWaitingFutures: MaxWaitingFutures, maxWaiti // don't let too many futures build up forceExtraFutures // Take all results that have been placed for writing to the network - responses.toSeq + responses.take(maxEmitPerExec.get) } } diff --git a/summingbird-online/src/main/scala/com/twitter/summingbird/online/executor/FinalFlatMap.scala b/summingbird-online/src/main/scala/com/twitter/summingbird/online/executor/FinalFlatMap.scala index c0a39688d..d7d5191cb 100644 --- a/summingbird-online/src/main/scala/com/twitter/summingbird/online/executor/FinalFlatMap.scala +++ b/summingbird-online/src/main/scala/com/twitter/summingbird/online/executor/FinalFlatMap.scala @@ -21,13 +21,13 @@ import com.twitter.bijection.Injection import com.twitter.util.Future import com.twitter.summingbird.online.Externalizer -import com.twitter.summingbird.batch.{ Batcher, BatchID, Timestamp} -import com.twitter.summingbird.online.{FlatMapOperation, AsyncCache} +import com.twitter.summingbird.online.{FlatMapOperation, AsyncCache, CacheBuilder} import com.twitter.summingbird.option.CacheSize import com.twitter.summingbird.online.option.{ MaxWaitingFutures, MaxFutureWaitTime, + MaxEmitPerExecute, FlushFrequency } @@ -39,59 +39,78 @@ import com.twitter.summingbird.online.option.{ * @author Ian O Connell */ -class FinalFlatMap[Event, Key, Value, S, D]( +// This is not a user settable variable. +// Its supplied by the planning system usually to ensure its large enough to cover the space +// used by the summers times some delta. +private[summingbird] case class KeyValueShards(get: Int) { + def summerIdFor[K](k: K): Int = k.hashCode % get +} + +class FinalFlatMap[Event, Key, Value: Semigroup, S <: InputState[_], D]( @transient flatMapOp: FlatMapOperation[Event, (Key, Value)], - cacheBuilder: (Semigroup[(List[InputState[S]], Timestamp, Value)]) => AsyncCache[(Key, BatchID), (List[InputState[S]], Timestamp, Value)], + cacheBuilder: CacheBuilder[Int, (List[S], Map[Key, Value])], maxWaitingFutures: MaxWaitingFutures, maxWaitingTime: MaxFutureWaitTime, - pDecoder: Injection[(Timestamp, Event), D], - pEncoder: Injection[(Timestamp, ((Key, BatchID), Value)), D] + maxEmitPerExec: MaxEmitPerExecute, + summerShards: KeyValueShards, + pDecoder: Injection[Event, D], + pEncoder: Injection[(Int, Map[Key, Value]), D] ) - (implicit monoid: Semigroup[Value], batcher: Batcher) - extends AsyncBase[Event, ((Key, BatchID), Value), InputState[S], D](maxWaitingFutures, - maxWaitingTime) { + extends AsyncBase[Event, (Int, Map[Key, Value]), S, D](maxWaitingFutures, + maxWaitingTime, + maxEmitPerExec) { + + type InS = S + type OutputElement = (Int, Map[Key, Value]) + val encoder = pEncoder val decoder = pDecoder val lockedOp = Externalizer(flatMapOp) - lazy val sCache: AsyncCache[(Key, BatchID), (List[InputState[S]], Timestamp, Value)] = cacheBuilder(implicitly[Semigroup[(List[InputState[S]], Timestamp, Value)]]) + lazy val sCache = cacheBuilder(implicitly[Semigroup[(List[S], Map[Key, Value])]]) - - private def formatResult(outData: Map[(Key, BatchID), (List[InputState[S]], Timestamp, Value)]) - : Iterable[(List[InputState[S]], Future[TraversableOnce[(Timestamp, ((Key, BatchID), Value))]])] = { - outData.toList.map{ case ((key, batchID), (tupList, ts, value)) => - (tupList, Future.value(List((ts, ((key, batchID), value))))) + private def formatResult(outData: Map[Int, (List[S], Map[Key, Value])]) + : TraversableOnce[(List[S], Future[TraversableOnce[OutputElement]])] = { + outData.iterator.map { case (outerKey, (tupList, valList)) => + if(valList.isEmpty) { + (tupList, Future.value(Nil)) + } else { + (tupList, Future.value(List((outerKey, valList)))) + } } } - override def tick: Future[Iterable[(List[InputState[S]], Future[TraversableOnce[(Timestamp, ((Key, BatchID), Value))]])]] = { + override def tick: Future[TraversableOnce[(List[S], Future[TraversableOnce[OutputElement]])]] = { sCache.tick.map(formatResult(_)) } - def cache(state: InputState[S], - time: Timestamp, - items: TraversableOnce[(Key, Value)]): Future[Iterable[(List[InputState[S]], Future[TraversableOnce[(Timestamp, ((Key, BatchID), Value))]])]] = { - - val batchID = batcher.batchOf(time) - val itemL = items.toList - if(itemL.size > 0) { - state.fanOut(itemL.size - 1) // Since input state starts at a 1 - - sCache.insert(itemL.map{case (k, v) => (k, batchID) -> (List(state), time, v)}).map(formatResult(_)) - } - else { // Here we handle mapping to nothing, option map et. al - Future.value( - List( - (List(state), Future.value(Nil)) - ) - ) + def cache(state: S, + items: TraversableOnce[(Key, Value)]): Future[TraversableOnce[(List[S], Future[TraversableOnce[OutputElement]])]] = { + try { + val itemL = items.toList + if(itemL.size > 0) { + state.fanOut(itemL.size) + sCache.insert(itemL.map{case (k, v) => + summerShards.summerIdFor(k) -> (List(state), Map(k -> v)) + }).map(formatResult(_)) } + else { // Here we handle mapping to nothing, option map et. al + Future.value( + List( + (List(state), Future.value(Nil)) + ) + ) + } + } + catch { + case t: Throwable => Future.exception(t) + } } - override def apply(state: InputState[S], - timeIn: (Timestamp, Event)) = - lockedOp.get.apply(timeIn._2).map { cache(state, timeIn._1, _) }.flatten + override def apply(state: S, + tup: Event) = + lockedOp.get.apply(tup).map { cache(state, _) }.flatten override def cleanup { lockedOp.get.close diff --git a/summingbird-online/src/main/scala/com/twitter/summingbird/online/executor/InputState.scala b/summingbird-online/src/main/scala/com/twitter/summingbird/online/executor/InputState.scala index 53ca2c2b2..53727b10e 100644 --- a/summingbird-online/src/main/scala/com/twitter/summingbird/online/executor/InputState.scala +++ b/summingbird-online/src/main/scala/com/twitter/summingbird/online/executor/InputState.scala @@ -74,12 +74,17 @@ case class InputState[T](state: T) { val stateTracking = new AtomicStateTransformer(State(1, false)) + + // Fanout is how many tuples one tuple became + // so we increment by 1 less than the amount given def fanOut(by: Int) = { require(by >= 0, "Invalid fanout: %d, by should be >= 0".format(by)) - val newS = stateTracking.update(_.incrBy(by)) + require(stateTracking.get.counter == 1, "You can only call fanOut once, and must do it before acking the tuple.") + val incrementAmount = by - 1 + val newS = stateTracking.update(_.incrBy(incrementAmount)) // If we incremented on something that was 0 or negative // And not in a failed state, then this is an error - if((newS.counter - by <= 0) && !newS.failed) { + if((newS.counter - incrementAmount <= 0) && !newS.failed) { throw new Exception("Invalid call on an inputstate, we had already decremented to 0 and not failed.") } this diff --git a/summingbird-online/src/main/scala/com/twitter/summingbird/online/executor/IntermediateFlatMap.scala b/summingbird-online/src/main/scala/com/twitter/summingbird/online/executor/IntermediateFlatMap.scala index 287b62197..61e33cb65 100644 --- a/summingbird-online/src/main/scala/com/twitter/summingbird/online/executor/IntermediateFlatMap.scala +++ b/summingbird-online/src/main/scala/com/twitter/summingbird/online/executor/IntermediateFlatMap.scala @@ -19,12 +19,12 @@ package com.twitter.summingbird.online.executor import com.twitter.util.Future import com.twitter.bijection.Injection -import com.twitter.summingbird.batch.Timestamp import com.twitter.summingbird.online.Externalizer import com.twitter.summingbird.online.FlatMapOperation import com.twitter.summingbird.online.option.{ MaxWaitingFutures, - MaxFutureWaitTime + MaxFutureWaitTime, + MaxEmitPerExecute } @@ -32,9 +32,10 @@ class IntermediateFlatMap[T,U,S,D]( @transient flatMapOp: FlatMapOperation[T, U], maxWaitingFutures: MaxWaitingFutures, maxWaitingTime: MaxFutureWaitTime, - pDecoder: Injection[(Timestamp, T), D], - pEncoder: Injection[(Timestamp, U), D] - ) extends AsyncBase[T,U,S,D](maxWaitingFutures, maxWaitingTime) { + maxEmitPerExec: MaxEmitPerExecute, + pDecoder: Injection[T, D], + pEncoder: Injection[U, D] + ) extends AsyncBase[T, U, S, D](maxWaitingFutures, maxWaitingTime, maxEmitPerExec) { val encoder = pEncoder val decoder = pDecoder @@ -43,9 +44,9 @@ class IntermediateFlatMap[T,U,S,D]( override def apply(state: S, - timeT: (Timestamp, T)): Future[Iterable[(List[S], Future[TraversableOnce[(Timestamp, U)]])]] = - lockedOp.get.apply(timeT._2).map { res => - List((List(state), Future.value(res.map((timeT._1, _))))) + tup: T): Future[Iterable[(List[S], Future[TraversableOnce[U]])]] = + lockedOp.get.apply(tup).map { res => + List((List(state), Future.value(res))) } override def cleanup { lockedOp.get.close } diff --git a/summingbird-online/src/main/scala/com/twitter/summingbird/online/executor/OperationContainer.scala b/summingbird-online/src/main/scala/com/twitter/summingbird/online/executor/OperationContainer.scala index f88e0b3a2..ecc144abc 100644 --- a/summingbird-online/src/main/scala/com/twitter/summingbird/online/executor/OperationContainer.scala +++ b/summingbird-online/src/main/scala/com/twitter/summingbird/online/executor/OperationContainer.scala @@ -18,15 +18,14 @@ package com.twitter.summingbird.online.executor import scala.util.Try import com.twitter.bijection.Injection -import com.twitter.summingbird.batch.Timestamp trait OperationContainer[Input, Output, State, WireFmt] { - def decoder: Injection[(Timestamp, Input), WireFmt] - def encoder: Injection[(Timestamp, Output), WireFmt] - def executeTick: TraversableOnce[(List[State], Try[TraversableOnce[(Timestamp, Output)]])] + def decoder: Injection[Input, WireFmt] + def encoder: Injection[Output, WireFmt] + def executeTick: TraversableOnce[(List[State], Try[TraversableOnce[Output]])] def execute(state: State, - data: (Timestamp, Input)): - TraversableOnce[(List[State], Try[TraversableOnce[(Timestamp, Output)]])] + data: Input): + TraversableOnce[(List[State], Try[TraversableOnce[Output]])] def init {} def cleanup {} def notifyFailure(inputs: List[State], e: Throwable) {} diff --git a/summingbird-online/src/main/scala/com/twitter/summingbird/online/executor/Summer.scala b/summingbird-online/src/main/scala/com/twitter/summingbird/online/executor/Summer.scala index b9d1172b7..2f339e2bb 100644 --- a/summingbird-online/src/main/scala/com/twitter/summingbird/online/executor/Summer.scala +++ b/summingbird-online/src/main/scala/com/twitter/summingbird/online/executor/Summer.scala @@ -18,16 +18,13 @@ package com.twitter.summingbird.online.executor import com.twitter.util.{Await, Future} import com.twitter.algebird.{Semigroup, SummingQueue} -import com.twitter.storehaus.algebra.MergeableStore +import com.twitter.storehaus.algebra.Mergeable import com.twitter.bijection.Injection -import com.twitter.summingbird.online.{Externalizer, AsyncCache} -import com.twitter.summingbird.batch.{BatchID, Timestamp} +import com.twitter.summingbird.online.{FlatMapOperation, Externalizer, AsyncCache, CacheBuilder} import com.twitter.summingbird.online.option._ import com.twitter.summingbird.option.CacheSize - - /** * The SummerBolt takes two related options: CacheSize and MaxWaitingFutures. * CacheSize sets the number of key-value pairs that the SinkBolt will accept @@ -50,32 +47,34 @@ import com.twitter.summingbird.option.CacheSize * @author Ashu Singhal */ -class Summer[Key, Value: Semigroup, S, D]( - @transient storeSupplier: () => MergeableStore[(Key,BatchID), Value], +class Summer[Key, Value: Semigroup, Event, S, D]( + @transient storeSupplier: () => Mergeable[Key, Value], + @transient flatMapOp: FlatMapOperation[(Key, (Option[Value], Value)), Event], @transient successHandler: OnlineSuccessHandler, @transient exceptionHandler: OnlineExceptionHandler, - cacheBuilder: (Semigroup[(List[S], Timestamp, Value)]) => AsyncCache[(Key, BatchID), (List[S], Timestamp, Value)], + cacheBuilder: CacheBuilder[Key, (List[InputState[S]], Value)], maxWaitingFutures: MaxWaitingFutures, maxWaitingTime: MaxFutureWaitTime, + maxEmitPerExec: MaxEmitPerExecute, includeSuccessHandler: IncludeSuccessHandler, - pDecoder: Injection[(Timestamp, ((Key, BatchID), Value)), D], - pEncoder: Injection[(Timestamp, (Key, (Option[Value], Value))), D]) extends - AsyncBase[((Key, BatchID), Value), (Key, (Option[Value], Value)), S, D]( + pDecoder: Injection[(Int, Map[Key, Value]), D], + pEncoder: Injection[Event, D]) extends + AsyncBase[(Int, Map[Key, Value]), Event, InputState[S], D]( maxWaitingFutures, - maxWaitingTime) { + maxWaitingTime, + maxEmitPerExec) { + val lockedOp = Externalizer(flatMapOp) val encoder = pEncoder val decoder = pDecoder val storeBox = Externalizer(storeSupplier) lazy val store = storeBox.get.apply - // See MaxWaitingFutures for a todo around removing this. - lazy val sCache: AsyncCache[(Key, BatchID), (List[S], Timestamp, Value)] = cacheBuilder(implicitly[Semigroup[(List[S], Timestamp, Value)]]) + lazy val sCache: AsyncCache[Key, (List[InputState[S]], Value)] = cacheBuilder(implicitly[Semigroup[(List[InputState[S]], Value)]]) val exceptionHandlerBox = Externalizer(exceptionHandler.handlerFn.lift) val successHandlerBox = Externalizer(successHandler) - var successHandlerOpt: Option[OnlineSuccessHandler] = null override def init { @@ -83,29 +82,39 @@ class Summer[Key, Value: Semigroup, S, D]( successHandlerOpt = if (includeSuccessHandler.get) Some(successHandlerBox.get) else None } - override def notifyFailure(inputs: List[S], error: Throwable): Unit = { + override def notifyFailure(inputs: List[InputState[S]], error: Throwable): Unit = { super.notifyFailure(inputs, error) exceptionHandlerBox.get.apply(error) } - private def handleResult(kvs: Map[(Key, BatchID), (List[S], Timestamp, Value)]) - : Iterable[(List[S], Future[TraversableOnce[(Timestamp, (Key, (Option[Value], Value)))]])] = { - store.multiMerge(kvs.mapValues(_._3)).map{ case (innerKb, beforeF) => - val (tups, stamp, delta) = kvs(innerKb) - val (k, _) = innerKb - (tups, beforeF.map(before => List((stamp, (k, (before, delta))))) - .onSuccess { _ => successHandlerOpt.get.handlerFn.apply() } ) - } - .toList // force, but order does not matter, so we could optimize this - } + private def handleResult(kvs: Map[Key, (List[InputState[S]], Value)]): TraversableOnce[(List[InputState[S]], Future[TraversableOnce[Event]])] = + store.multiMerge(kvs.mapValues(_._2)).iterator.map { case (k, beforeF) => + val (tups, delta) = kvs(k) + (tups, beforeF.flatMap { before => + lockedOp.get.apply((k, (before, delta))) + }.onSuccess { _ => successHandlerOpt.get.handlerFn.apply() } ) + }.toList + override def tick = sCache.tick.map(handleResult(_)) - override def apply(state: S, - tsIn: (Timestamp, ((Key, BatchID), Value))) = { - val (ts, (kb, v)) = tsIn - sCache.insert(List(kb -> (List(state), ts, v))).map(handleResult(_)) + override def apply(state: InputState[S], + tupList: (Int, Map[Key, Value])) = { + try { + val (_, innerTuples) = tupList + assert(innerTuples.size > 0, "Maps coming in must not be empty") + state.fanOut(innerTuples.size) + val cacheEntries = innerTuples.map { case (k, v) => + (k, (List(state), v)) + } + + sCache.insert(cacheEntries).map(handleResult(_)) + } + catch { + case t: Throwable => Future.exception(t) + } } - override def cleanup { Await.result(store.close) } + + override def cleanup = Await.result(store.close) } diff --git a/summingbird-online/src/main/scala/com/twitter/summingbird/online/option/AllOpts.scala b/summingbird-online/src/main/scala/com/twitter/summingbird/online/option/AllOpts.scala index 359dc6463..dcc1a5f09 100644 --- a/summingbird-online/src/main/scala/com/twitter/summingbird/online/option/AllOpts.scala +++ b/summingbird-online/src/main/scala/com/twitter/summingbird/online/option/AllOpts.scala @@ -57,6 +57,12 @@ case class UseAsyncCache(get: Boolean) */ case class AsyncPoolSize(get: Int) +/* + MaxEmitPerExecute controls the number of elements that can at once be emitted to the underlying platform. + Must be careful this is >> than your fan out or more tuples could be generated than are emitted. +*/ +case class MaxEmitPerExecute(get: Int) + /* SoftMemoryFlushPercent is the percentage of memory used in the JVM at which a flush will be triggered of the cache. */ @@ -67,4 +73,4 @@ case class SoftMemoryFlushPercent(get: Float) { /* ValueCombinerCacheSize is used in cache's that support it as a trigger to crush down a high locality of values without emitting */ -case class ValueCombinerCacheSize(get: Int) \ No newline at end of file +case class ValueCombinerCacheSize(get: Int) diff --git a/summingbird-online/src/main/scala/com/twitter/summingbird/planner/StripNamedNodes.scala b/summingbird-online/src/main/scala/com/twitter/summingbird/planner/StripNamedNodes.scala index a69e8a9a2..a005f6c5a 100644 --- a/summingbird-online/src/main/scala/com/twitter/summingbird/planner/StripNamedNodes.scala +++ b/summingbird-online/src/main/scala/com/twitter/summingbird/planner/StripNamedNodes.scala @@ -30,7 +30,7 @@ object StripNamedNode { def castToKeyed[P <: Platform[P]](node: Producer[P, Any]): KeyedProducer[P, Any, Any] = node.asInstanceOf[KeyedProducer[P, Any, Any]] def processLevel[P <: Platform[P]](optLast: Option[Producer[P, Any]], - l: List[ProducerF[P]], + l: TraversableOnce[ProducerF[P]], m: Map[Producer[P, Any], Producer[P, Any]], op: PartialFunction[Producer[P, Any], Option[Producer[P, Any]]]): (Option[Producer[P, Any]], Map[Producer[P, Any], Producer[P, Any]]) = { l.foldLeft((optLast, m)){ case ((nOptLast, nm), pp) => @@ -126,16 +126,19 @@ object StripNamedNode { } } - def toFunctional[P <: Platform[P]](tail: Producer[P, Any]) = { - val depthInfo = graph.dagDepth(Producer.entireGraphOf(tail))(Producer.parentsOf(_)) + def toFunctional[P <: Platform[P]](tail: Producer[P, Any]) = + graph + .dagDepth(Producer.entireGraphOf(tail))(Producer.parentsOf(_)) + .toSeq + .groupBy(_._2) + .mapValues(_.map(_._1)) + .mapValues(_.map(functionize(_))) + .toSeq - val reversedGraph = depthInfo.toList.groupBy(_._2).mapValues(_.map(_._1)) - reversedGraph.mapValues(_.map(functionize(_))) - } def mutateGraph[P <: Platform[P]](tail: Producer[P, Any], op: PartialFunction[Producer[P, Any], Option[Producer[P, Any]]]) = { val newT: Option[Producer[P, Any]] = None - val x = toFunctional(tail).toList.sortBy(_._1) + val x = toFunctional(tail).sortBy(_._1) x.map(_._2).foldLeft((newT, Map[Producer[P, Any], Producer[P, Any]]())) { case ((optLast, curMap), v) => processLevel(optLast, v, curMap, op) } diff --git a/summingbird-online/src/test/scala/com/twitter/summingbird/online/PlannerSpec.scala b/summingbird-online/src/test/scala/com/twitter/summingbird/online/PlannerSpec.scala index 6ef9959e5..8cc72fff5 100644 --- a/summingbird-online/src/test/scala/com/twitter/summingbird/online/PlannerSpec.scala +++ b/summingbird-online/src/test/scala/com/twitter/summingbird/online/PlannerSpec.scala @@ -20,7 +20,6 @@ import com.twitter.algebird.{MapAlgebra, Semigroup} import com.twitter.storehaus.{ ReadableStore, JMapStore } import com.twitter.storehaus.algebra.MergeableStore import com.twitter.summingbird._ -import com.twitter.summingbird.batch.{BatchID, Batcher} import com.twitter.summingbird.memory._ import com.twitter.summingbird.planner._ import com.twitter.util.Future diff --git a/summingbird-scalding-test/src/main/scala/com/twitter/summingbird/scalding/ErrorThrowState.scala b/summingbird-scalding-test/src/main/scala/com/twitter/summingbird/scalding/ErrorThrowState.scala new file mode 100644 index 000000000..0de196f90 --- /dev/null +++ b/summingbird-scalding-test/src/main/scala/com/twitter/summingbird/scalding/ErrorThrowState.scala @@ -0,0 +1,37 @@ +/* + Copyright 2013 Twitter, Inc. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + */ + +package com.twitter.summingbird.scalding + +import com.twitter.algebird.monad._ +import com.twitter.summingbird.batch._ + +class ErrorThrowState[T](init: T) extends WaitingState[T] { self => + def begin = new PrepareState[T] { + def requested = self.init + def fail(err: Throwable) = { + throw(err) + self + } + def willAccept(intr: T) = Right(new RunningState[T] { + def succeed = self + def fail(err: Throwable) = { + throw(err) + self + } + }) + } +} diff --git a/summingbird-scalding-test/src/main/scala/com/twitter/summingbird/scalding/LoopState.scala b/summingbird-scalding-test/src/main/scala/com/twitter/summingbird/scalding/LoopState.scala new file mode 100644 index 000000000..36699faa2 --- /dev/null +++ b/summingbird-scalding-test/src/main/scala/com/twitter/summingbird/scalding/LoopState.scala @@ -0,0 +1,40 @@ +/* + Copyright 2013 Twitter, Inc. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + */ + +package com.twitter.summingbird.scalding + +import com.twitter.algebird.monad._ +import com.twitter.summingbird.batch._ + + + +// This is not really usable, just a mock that does the same state over and over +class LoopState[T](init: T) extends WaitingState[T] { self => + def begin = new PrepareState[T] { + def requested = self.init + def fail(err: Throwable) = { + println(err) + self + } + def willAccept(intr: T) = Right(new RunningState[T] { + def succeed = self + def fail(err: Throwable) = { + println(err) + self + } + }) + } +} diff --git a/summingbird-scalding-test/src/main/scala/com/twitter/summingbird/scalding/MockMappable.scala b/summingbird-scalding-test/src/main/scala/com/twitter/summingbird/scalding/MockMappable.scala new file mode 100644 index 000000000..fc708ca68 --- /dev/null +++ b/summingbird-scalding-test/src/main/scala/com/twitter/summingbird/scalding/MockMappable.scala @@ -0,0 +1,40 @@ +/* + Copyright 2013 Twitter, Inc. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + */ + +package com.twitter.summingbird.scalding + +import com.twitter.scalding.{ Source => ScaldingSource, Test => TestMode, _ } + +import cascading.tuple.{Tuple, Fields, TupleEntry} +import cascading.tap.Tap +import cascading.scheme.NullScheme +import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.mapred.RecordReader +import org.apache.hadoop.mapred.OutputCollector + +class MockMappable[T](val id: String)(implicit tconv: TupleConverter[T]) + extends ScaldingSource with Mappable[T] { + def converter[U >: T] = TupleConverter.asSuperConverter(tconv) + override def toString = id + override def equals(that: Any) = that match { + case m: MockMappable[_] => m.id == id + case _ => false + } + override def hashCode = id.hashCode + + override def createTap(readOrWrite : AccessMode)(implicit mode : Mode) : Tap[_,_,_] = + TestTapFactory(this, new NullScheme[JobConf, RecordReader[_,_], OutputCollector[_,_], T, T](Fields.ALL, Fields.ALL)).createTap(readOrWrite) +} \ No newline at end of file diff --git a/summingbird-scalding-test/src/main/scala/com/twitter/summingbird/scalding/TestService.scala b/summingbird-scalding-test/src/main/scala/com/twitter/summingbird/scalding/TestService.scala new file mode 100644 index 000000000..b6896159c --- /dev/null +++ b/summingbird-scalding-test/src/main/scala/com/twitter/summingbird/scalding/TestService.scala @@ -0,0 +1,97 @@ +/* + Copyright 2013 Twitter, Inc. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + */ + +package com.twitter.summingbird.scalding + +import com.twitter.algebird.monad._ +import com.twitter.summingbird.batch._ + +import com.twitter.scalding.{ Source => ScaldingSource, Test => TestMode, _ } +import com.twitter.summingbird.scalding.batch.{BatchedService => BBatchedService} +import scala.collection.mutable.Buffer +import cascading.tuple.Tuple +import cascading.flow.FlowDef + +class TestService[K, V](service: String, + inBatcher: Batcher, + minBatch: BatchID, + streams: Map[BatchID, Iterable[(Timestamp, (K, Option[V]))]]) +(implicit ord: Ordering[K], + tset: TupleSetter[(Timestamp, (K, Option[V]))], + tset2: TupleSetter[(Timestamp, (K, V))], + tconv: TupleConverter[(Timestamp, (K, Option[V]))], + tconv2: TupleConverter[(Timestamp, (K, V))]) + extends BBatchedService[K, V] { + + val batcher = inBatcher + val ordering = ord + val reducers = None + // Needed to init the Test mode: + val sourceToBuffer: Map[ScaldingSource, Buffer[Tuple]] = + (lasts.map { case (b, it) => lastMappable(b) -> toBuffer(it) } ++ + streams.map { case (b, it) => streamMappable(b) -> toBuffer(it) }).toMap + + /** The lasts are computed from the streams */ + lazy val lasts: Map[BatchID, Iterable[(Timestamp, (K, V))]] = { + (streams + .toList + .sortBy(_._1) + .foldLeft(Map.empty[BatchID, Map[K, (Timestamp, V)]]) { + case (map, (batch: BatchID, writes: Iterable[(Timestamp, (K, Option[V]))])) => + val thisBatch = writes.foldLeft(map.get(batch).getOrElse(Map.empty[K, (Timestamp, V)])) { + case (innerMap, (time, (k, v))) => + v match { + case None => innerMap - k + case Some(v) => innerMap + (k -> (time -> v)) + } + } + map + (batch -> thisBatch) + } + .mapValues { innerMap => + innerMap.toSeq.map { case (k, (time, v)) => (time, (k, v)) } + }) + (minBatch -> Iterable.empty) + } + + def lastMappable(b: BatchID): Mappable[(Timestamp, (K, V))] = + new MockMappable[(Timestamp, (K, V))](service + "/last/" + b.toString) + + def streamMappable(b: BatchID): Mappable[(Timestamp, (K, Option[V]))] = + new MockMappable[(Timestamp, (K, Option[V]))](service + "/stream/" + b.toString) + + def toBuffer[T](it: Iterable[T])(implicit ts: TupleSetter[T]): Buffer[Tuple] = + it.map { ts(_) }.toBuffer + + override def readStream(batchID: BatchID, mode: Mode): Option[FlowToPipe[(K, Option[V])]] = { + streams.get(batchID).map { iter => + val mappable = streamMappable(batchID) + Reader { (fd: (FlowDef, Mode)) => TypedPipe.from(mappable)(fd._1, fd._2) } + } + } + override def readLast(exclusiveUB: BatchID, mode: Mode) = { + val candidates = lasts.filter { _._1 < exclusiveUB } + if(candidates.isEmpty) { + Left(List("No batches < :" + exclusiveUB.toString)) + } + else { + val (batch, _) = candidates.maxBy { _._1 } + val mappable = lastMappable(batch) + val rdr = Reader { (fd: (FlowDef, Mode)) => + TypedPipe.from(mappable)(fd._1, fd._2).values + } + Right((batch, rdr)) + } + } +} diff --git a/summingbird-scalding-test/src/main/scala/com/twitter/summingbird/scalding/TestSink.scala b/summingbird-scalding-test/src/main/scala/com/twitter/summingbird/scalding/TestSink.scala new file mode 100644 index 000000000..30087e8ad --- /dev/null +++ b/summingbird-scalding-test/src/main/scala/com/twitter/summingbird/scalding/TestSink.scala @@ -0,0 +1,47 @@ +/* + Copyright 2013 Twitter, Inc. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + */ + +package com.twitter.summingbird.scalding + +import com.twitter.algebird.monad._ +import com.twitter.summingbird.batch._ + +import com.twitter.scalding.{ Source => ScaldingSource, Test => TestMode, _ } +import com.twitter.scalding.typed.TypedSink + +/** This is a test sink that assumes single threaded testing with + * cascading local mode + */ +class TestSink[T] extends Sink[T] { + private var data: Vector[(Timestamp, T)] = Vector.empty + + def write(incoming: PipeFactory[T]): PipeFactory[T] = + // three functors deep: + incoming.map { state => + state.map { reader => + reader.map { timeItem => + data = data :+ timeItem + timeItem + } + } + } + + def reset: Vector[(Timestamp, T)] = { + val oldData = data + data = Vector.empty + oldData + } +} diff --git a/summingbird-scalding-test/src/main/scala/com/twitter/summingbird/scalding/TestSource.scala b/summingbird-scalding-test/src/main/scala/com/twitter/summingbird/scalding/TestSource.scala new file mode 100644 index 000000000..33d450d25 --- /dev/null +++ b/summingbird-scalding-test/src/main/scala/com/twitter/summingbird/scalding/TestSource.scala @@ -0,0 +1,38 @@ + + + /* + Copyright 2013 Twitter, Inc. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + */ + +package com.twitter.summingbird.scalding + +import com.twitter.algebird.monad._ +import com.twitter.summingbird.batch._ + +import com.twitter.scalding.{ Source => ScaldingSource, Test => TestMode, _ } +import com.twitter.summingbird.{Producer, TimeExtractor } +import scala.collection.mutable.Buffer +import cascading.tuple.Tuple +import cascading.flow.FlowDef + +object TestSource { + def apply[T](iter: Iterable[T]) + (implicit mf: Manifest[T], te: TimeExtractor[T], tc: TupleConverter[T], tset: TupleSetter[T]): + (Map[ScaldingSource, Buffer[Tuple]], Producer[Scalding, T]) = { + val src = IterableSource(iter) + val prod = Scalding.sourceFromMappable { _ => src } + (Map(src -> iter.map { tset(_) }.toBuffer), prod) + } +} diff --git a/summingbird-scalding-test/src/main/scala/com/twitter/summingbird/scalding/TestStore.scala b/summingbird-scalding-test/src/main/scala/com/twitter/summingbird/scalding/TestStore.scala new file mode 100644 index 000000000..91701de68 --- /dev/null +++ b/summingbird-scalding-test/src/main/scala/com/twitter/summingbird/scalding/TestStore.scala @@ -0,0 +1,96 @@ +/* + Copyright 2013 Twitter, Inc. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + */ + +package com.twitter.summingbird.scalding + +import com.twitter.algebird.monad._ +import com.twitter.summingbird.batch._ +import com.twitter.summingbird.batch.state.HDFSState + +import com.twitter.scalding.{ Source => ScaldingSource, Test => TestMode, _ } +import com.twitter.scalding.typed.TypedSink + +import scala.collection.mutable.Buffer + +import cascading.scheme.local.{TextDelimited => CLTextDelimited} +import cascading.tuple.{Tuple, TupleEntry} +import cascading.flow.FlowDef + + +object TestStore { + def apply[K, V](store: String, + inBatcher: Batcher, + initStore: Iterable[(K, V)], + lastTime: Long, + pruning: PrunedSpace[(K, V)] = PrunedSpace.neverPruned, + boundedKeySpace: TimeBoundedKeySpace[K] = TimeBoundedKeySpace.neverFrozen) + (implicit ord: Ordering[K], tset: TupleSetter[(K, V)], tconv: TupleConverter[(K, V)]) = { + val startBatch = inBatcher.batchOf(Timestamp(0)).prev + val endBatch = inBatcher.batchOf(Timestamp(lastTime)).next + new TestStore[K, V](store, inBatcher, startBatch, initStore, endBatch, pruning, boundedKeySpace) + } +} + +class TestStore[K, V](store: String, + inBatcher: Batcher, + initBatch: BatchID, + initStore: Iterable[(K, V)], + lastBatch: BatchID, + override val pruning: PrunedSpace[(K, V)], + override val boundedKeySpace: TimeBoundedKeySpace[K]) +(implicit ord: Ordering[K], tset: TupleSetter[(K, V)], tconv: TupleConverter[(K, V)]) + extends batch.BatchedStore[K, V] { + + var writtenBatches = Set[BatchID](initBatch) + val batches: Map[BatchID, Mappable[(K, V)]] = + BatchID.range(initBatch, lastBatch).map { b => (b, mockFor(b)) }.toMap + + // Needed to init the Test mode: + val sourceToBuffer: Map[ScaldingSource, Buffer[Tuple]] = + BatchID.range(initBatch, lastBatch).map { b => + if (initBatch == b) (batches(b), initStore.map { tset(_) }.toBuffer) + else (batches(b), Buffer.empty[Tuple]) + }.toMap + + // Call this after you compute to check the results of the + def lastToIterable: Iterable[(K, V)] = + sourceToBuffer(batches(writtenBatches.max)).toIterable.map { tup => tconv(new TupleEntry(tup)) } + + val batcher = inBatcher + val ordering = ord + + def mockFor(b: BatchID): Mappable[(K, V)] = + new MockMappable(store + b.toString) + + override def readLast(exclusiveUB: BatchID, mode: Mode) = { + val candidates = writtenBatches.filter { _ < exclusiveUB } + if(candidates.isEmpty) { + Left(List("No batches < :" + exclusiveUB.toString)) + } + else { + val batch = candidates.max + val mappable = batches(batch) + val rdr = Reader { (fd: (FlowDef, Mode)) => TypedPipe.from(mappable)(fd._1, fd._2) } + Right((batch, rdr)) + } + } + /** Instances may choose to write out the last or just compute it from the stream */ + override def writeLast(batchID: BatchID, lastVals: TypedPipe[(K, V)])(implicit flowDef: FlowDef, mode: Mode): Unit = { + val out = batches(batchID) + lastVals.write(TypedSink[(K, V)](out)) + writtenBatches = writtenBatches + batchID + } +} diff --git a/summingbird-scalding-test/src/main/scala/com/twitter/summingbird/scalding/TestUtil.scala b/summingbird-scalding-test/src/main/scala/com/twitter/summingbird/scalding/TestUtil.scala new file mode 100644 index 000000000..cfd9e2730 --- /dev/null +++ b/summingbird-scalding-test/src/main/scala/com/twitter/summingbird/scalding/TestUtil.scala @@ -0,0 +1,89 @@ +/* + Copyright 2013 Twitter, Inc. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + */ + +package com.twitter.summingbird.scalding + +import com.twitter.algebird.{MapAlgebra, Monoid, Group, Interval, Last} +import com.twitter.algebird.monad._ +import com.twitter.summingbird.batch._ +import com.twitter.summingbird.TimeExtractor + +import com.twitter.scalding.{ Source => ScaldingSource, Test => TestMode, _ } + +import org.scalacheck._ +import org.scalacheck.Prop._ +import org.scalacheck.Properties + +object TestUtil { + def simpleTimeExtractor[T <: (Long, _)]: TimeExtractor[T] = TimeExtractor( _._1 ) + + def compareMaps[K,V:Group](original: Iterable[Any], inMemory: Map[K, V], testStore: TestStore[K, V], name: String = ""): Boolean = { + val produced = testStore.lastToIterable.toMap + val diffMap = Group.minus(inMemory, produced) + val wrong = Monoid.isNonZero(diffMap) + if(wrong) { + if(!name.isEmpty) println("%s is wrong".format(name)) + println("input: " + original) + println("input size: " + original.size) + println("input batches: " + testStore.batcher.batchOf(Timestamp(original.size))) + println("producer extra keys: " + (produced.keySet -- inMemory.keySet)) + println("producer missing keys: " + (inMemory.keySet -- produced.keySet)) + println("written batches: " + testStore.writtenBatches) + println("earliest unwritten time: " + testStore.batcher.earliestTimeOf(testStore.writtenBatches.max.next)) + println("Difference: " + diffMap) + } + !wrong + } + + def batchedCover(batcher: Batcher, minTime: Long, maxTime: Long): Interval[Timestamp] = + batcher.cover( + Interval.leftClosedRightOpen(Timestamp(minTime), Timestamp(maxTime+1L)) + ).mapNonDecreasing(b => batcher.earliestTimeOf(b.next)) + + val simpleBatcher = new Batcher { + def batchOf(d: Timestamp) = + if (d == Timestamp.Max) BatchID(2) + else if (d.milliSinceEpoch >= 0L) BatchID(1) + else BatchID(0) + + def earliestTimeOf(batch: BatchID) = batch.id match { + case 0L => Timestamp.Min + case 1L => Timestamp(0) + case 2L => Timestamp.Max + case 3L => Timestamp.Max + } + // this is just for testing, it covers everything with batch 1 + override def cover(interval: Interval[Timestamp]): Interval[BatchID] = + Interval.leftClosedRightOpen(BatchID(1), BatchID(2)) + } + + def randomBatcher(items: Iterable[(Long, Any)]): Batcher = { + if(items.isEmpty) simpleBatcher + else randomBatcher(items.iterator.map(_._1).min, items.iterator.map(_._1).max) + } + + def randomBatcher(mintimeInc: Long, maxtimeInc: Long): Batcher = { //simpleBatcher + // we can have between 1 and (maxtime - mintime + 1) batches. + val delta = (maxtimeInc - mintimeInc) + val MaxBatches = 5L min delta + val batches = 1L + Gen.choose(0L, MaxBatches).sample.get + if(batches == 1L) simpleBatcher + else { + val timePerBatch = (delta + 1L)/batches + new MillisecondBatcher(timePerBatch) + } + } +} diff --git a/summingbird-scalding/src/test/scala/com/twitter/summingbird/scalding/LookupJoinTest.scala b/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/LookupJoinTest.scala similarity index 100% rename from summingbird-scalding/src/test/scala/com/twitter/summingbird/scalding/LookupJoinTest.scala rename to summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/LookupJoinTest.scala diff --git a/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/ScaldingLaws.scala b/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/ScaldingLaws.scala new file mode 100644 index 000000000..331071d17 --- /dev/null +++ b/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/ScaldingLaws.scala @@ -0,0 +1,374 @@ +/* + Copyright 2013 Twitter, Inc. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + */ + +package com.twitter.summingbird.scalding + +import com.twitter.algebird.{MapAlgebra, Monoid, Group, Interval, Last} +import com.twitter.algebird.monad._ +import com.twitter.summingbird.{Producer, TimeExtractor, TestGraphs} +import com.twitter.summingbird.batch._ +import com.twitter.summingbird.batch.state.HDFSState + +import java.util.TimeZone +import java.io.File + +import com.twitter.scalding.{ Source => ScaldingSource, Test => TestMode, _ } +import com.twitter.scalding.typed.TypedSink + +import org.scalacheck._ +import org.scalacheck.Prop._ +import org.scalacheck.Properties + +import org.apache.hadoop.conf.Configuration + +import scala.collection.JavaConverters._ +import scala.collection.mutable.{ArrayBuffer, Buffer, HashMap => MutableHashMap, Map => MutableMap, SynchronizedBuffer, SynchronizedMap} + +import cascading.scheme.local.{TextDelimited => CLTextDelimited} +import cascading.tuple.{Tuple, Fields, TupleEntry} +import cascading.flow.FlowDef +import cascading.tap.Tap +import cascading.scheme.NullScheme +import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.mapred.RecordReader +import org.apache.hadoop.mapred.OutputCollector + + +import org.specs2.mutable._ + +/** + * Tests for Summingbird's Scalding planner. + */ + +object ScaldingLaws extends Specification { + import MapAlgebra.sparseEquiv + + implicit def timeExtractor[T <: (Long, _)] = TestUtil.simpleTimeExtractor[T] + + def sample[T: Arbitrary]: T = Arbitrary.arbitrary[T].sample.get + + "The ScaldingPlatform" should { + //Set up the job: + "match scala for single step jobs" in { + val original = sample[List[Int]] + val fn = sample[(Int) => List[(Int, Int)]] + val initStore = sample[Map[Int, Int]] + val inMemory = TestGraphs.singleStepInScala(original)(fn) + // Add a time: + val inWithTime = original.zipWithIndex.map { case (item, time) => (time.toLong, item) } + val batcher = TestUtil.randomBatcher(inWithTime) + val testStore = TestStore[Int,Int]("test", batcher, initStore, inWithTime.size) + val (buffer, source) = TestSource(inWithTime) + + val summer = TestGraphs.singleStepJob[Scalding,(Long,Int),Int,Int](source, testStore)(t => + fn(t._2)) + + val scald = Scalding("scalaCheckJob") + val intr = TestUtil.batchedCover(batcher, 0L, original.size.toLong) + val ws = new LoopState(intr) + val mode: Mode = TestMode(t => (testStore.sourceToBuffer ++ buffer).get(t)) + + scald.run(ws, mode, scald.plan(summer)) + // Now check that the inMemory == + + TestUtil.compareMaps(original, Monoid.plus(initStore, inMemory), testStore) must be_==(true) + } + + + "match scala single step pruned jobs" in { + val original = sample[List[Int]] + val fn = sample[(Int) => List[(Int, Int)]] + val initStore = sample[Map[Int, Int]] + val prunedList = sample[Set[Int]] + val inMemory = { + val computedMap = TestGraphs.singleStepInScala(original)(fn) + val totalMap = Monoid.plus(initStore, computedMap) + totalMap.filter(kv => !prunedList.contains(kv._1)).toMap + } + + val pruner = new PrunedSpace[(Int, Int)] { + def prune(item: (Int, Int), writeTime: Timestamp) = { + prunedList.contains(item._1) + } + } + // Add a time: + val inWithTime = original.zipWithIndex.map { case (item, time) => (time.toLong, item) } + val batcher = TestUtil.randomBatcher(inWithTime) + val testStore = TestStore[Int,Int]("test", batcher, initStore, inWithTime.size, pruner) + val (buffer, source) = TestSource(inWithTime) + + val summer = TestGraphs.singleStepJob[Scalding,(Long,Int),Int,Int](source, testStore)(t => + fn(t._2)) + + val scald = Scalding("scalaCheckJob") + val intr = TestUtil.batchedCover(batcher, 0L, original.size.toLong) + val ws = new LoopState(intr) + val mode: Mode = TestMode(t => (testStore.sourceToBuffer ++ buffer).get(t)) + + scald.run(ws, mode, scald.plan(summer)) + // Now check that the inMemory == + + TestUtil.compareMaps(original, inMemory, testStore) must be_==(true) + } + + "match scala for flatMapKeys jobs" in { + val original = sample[List[Int]] + val initStore = sample[Map[Int,Int]] + val fnA = sample[(Int) => List[(Int, Int)]] + val fnB = sample[Int => List[Int]] + val inMemory = TestGraphs.singleStepMapKeysInScala(original)(fnA, fnB) + // Add a time: + val inWithTime = original.zipWithIndex.map { case (item, time) => (time.toLong, item) } + val batcher = TestUtil.randomBatcher(inWithTime) + val testStore = TestStore[Int,Int]("test", batcher, initStore, inWithTime.size) + + val (buffer, source) = TestSource(inWithTime) + + val summer = TestGraphs.singleStepMapKeysJob[Scalding,(Long,Int),Int,Int, Int](source, testStore)(t => + fnA(t._2), fnB) + + val intr = TestUtil.batchedCover(batcher, 0L, original.size.toLong) + val scald = Scalding("scalaCheckJob") + val ws = new LoopState(intr) + val mode: Mode = TestMode(t => (testStore.sourceToBuffer ++ buffer).get(t)) + + scald.run(ws, mode, scald.plan(summer)) + // Now check that the inMemory == + + TestUtil.compareMaps(original, Monoid.plus(initStore, inMemory), testStore) must beTrue + } + + "match scala for multiple summer jobs" in { + val original = sample[List[Int]] + val initStoreA = sample[Map[Int,Int]] + val initStoreB = sample[Map[Int,Int]] + val fnA = sample[(Int) => List[(Int)]] + val fnB = sample[(Int) => List[(Int, Int)]] + val fnC = sample[(Int) => List[(Int, Int)]] + val (inMemoryA, inMemoryB) = TestGraphs.multipleSummerJobInScala(original)(fnA, fnB, fnC) + + // Add a time: + val inWithTime = original.zipWithIndex.map { case (item, time) => (time.toLong, item) } + val batcher = TestUtil.randomBatcher(inWithTime) + val testStoreA = TestStore[Int,Int]("testA", batcher, initStoreA, inWithTime.size) + val testStoreB = TestStore[Int,Int]("testB", batcher, initStoreB, inWithTime.size) + val (buffer, source) = TestSource(inWithTime) + + val tail = TestGraphs.multipleSummerJob[Scalding, (Long, Int), Int, Int, Int, Int, Int](source, testStoreA, testStoreB)({t => fnA(t._2)}, fnB, fnC) + + val scald = Scalding("scalaCheckMultipleSumJob") + val intr = TestUtil.batchedCover(batcher, 0L, original.size.toLong) + val ws = new LoopState(intr) + val mode: Mode = TestMode(t => (testStoreA.sourceToBuffer ++ testStoreB.sourceToBuffer ++ buffer).get(t)) + + scald.run(ws, mode, scald.plan(tail)) + // Now check that the inMemory == + + TestUtil.compareMaps(original, Monoid.plus(initStoreA, inMemoryA), testStoreA) must beTrue + TestUtil.compareMaps(original, Monoid.plus(initStoreB, inMemoryB), testStoreB) must beTrue + } + + + "match scala for leftJoin jobs" in { + val original = sample[List[Int]] + val prejoinMap = sample[(Int) => List[(Int, Int)]] + val service = sample[(Int,Int) => Option[Int]] + val postJoin = sample[((Int, (Int, Option[Int]))) => List[(Int, Int)]] + // We need to keep track of time correctly to use the service + var fakeTime = -1 + val timeIncIt = new Iterator[Int] { + val inner = original.iterator + def hasNext = inner.hasNext + def next = { + fakeTime += 1 + inner.next + } + } + val srvWithTime = { (key: Int) => service(fakeTime, key) } + val inMemory = TestGraphs.leftJoinInScala(timeIncIt)(srvWithTime)(prejoinMap)(postJoin) + + // Add a time: + val allKeys = original.flatMap(prejoinMap).map { _._1 } + val allTimes = (0 until original.size) + val stream = for { time <- allTimes; key <- allKeys; v = service(time, key) } yield (time.toLong, (key, v)) + + val inWithTime = original.zipWithIndex.map { case (item, time) => (time.toLong, item) } + val batcher = TestUtil.randomBatcher(inWithTime) + val initStore = sample[Map[Int, Int]] + val testStore = TestStore[Int,Int]("test", batcher, initStore, inWithTime.size) + + /** + * Create the batched service + */ + val batchedService = stream.map{case (time, v) => (Timestamp(time), v)}.groupBy { case (ts, _) => batcher.batchOf(ts) } + val testService = new TestService[Int, Int]("srv", batcher, batcher.batchOf(Timestamp(0)).prev, batchedService) + + val (buffer, source) = TestSource(inWithTime) + + val summer = + TestGraphs.leftJoinJob[Scalding,(Long, Int),Int,Int,Int,Int](source, testService, testStore) { tup => prejoinMap(tup._2) }(postJoin) + + val intr = TestUtil.batchedCover(batcher, 0L, original.size.toLong) + val scald = Scalding("scalaCheckleftJoinJob") + val ws = new LoopState(intr) + val mode: Mode = TestMode(s => (testStore.sourceToBuffer ++ buffer ++ testService.sourceToBuffer).get(s)) + + scald.run(ws, mode, summer) + // Now check that the inMemory == + + TestUtil.compareMaps(original, Monoid.plus(initStore, inMemory), testStore) must beTrue + } + + "match scala for diamond jobs with write" in { + val original = sample[List[Int]] + val fn1 = sample[(Int) => List[(Int, Int)]] + val fn2 = sample[(Int) => List[(Int, Int)]] + val inMemory = TestGraphs.diamondJobInScala(original)(fn1)(fn2) + // Add a time: + val inWithTime = original.zipWithIndex.map { case (item, time) => (time.toLong, item) } + val batcher = TestUtil.randomBatcher(inWithTime) + val initStore = sample[Map[Int, Int]] + val testStore = TestStore[Int,Int]("test", batcher, initStore, inWithTime.size) + val testSink = new TestSink[(Long,Int)] + val (buffer, source) = TestSource(inWithTime) + + val summer = TestGraphs + .diamondJob[Scalding,(Long, Int),Int,Int](source, + testSink, + testStore)(t => fn1(t._2))(t => fn2(t._2)) + + val scald = Scalding("scalding-diamond-Job") + val intr = TestUtil.batchedCover(batcher, 0L, original.size.toLong) + val ws = new LoopState(intr) + val mode: Mode = TestMode(s => (testStore.sourceToBuffer ++ buffer).get(s)) + + scald.run(ws, mode, summer) + // Now check that the inMemory == + + val sinkOut = testSink.reset + TestUtil.compareMaps(original, Monoid.plus(initStore, inMemory), testStore) must beTrue + val wrongSink = sinkOut.map { _._2 }.toList != inWithTime + wrongSink must be_==(false) + if(wrongSink) { + println("input: " + inWithTime) + println("SinkExtra: " + (sinkOut.map(_._2).toSet -- inWithTime.toSet)) + println("SinkMissing: " + (inWithTime.toSet -- sinkOut.map(_._2).toSet)) + } + } + + "Correctly aggregate multiple sumByKeys" in { + val original = sample[List[(Int,Int)]] + val keyExpand = sample[(Int) => List[Int]] + val (inMemoryA, inMemoryB) = TestGraphs.twoSumByKeyInScala(original, keyExpand) + // Add a time: + val inWithTime = original.zipWithIndex.map { case (item, time) => (time.toLong, item) } + val batcher = TestUtil.randomBatcher(inWithTime) + val initStore = sample[Map[Int, Int]] + val testStoreA = TestStore[Int,Int]("testA", batcher, initStore, inWithTime.size) + val testStoreB = TestStore[Int,Int]("testB", batcher, initStore, inWithTime.size) + val (buffer, source) = TestSource(inWithTime) + + val summer = TestGraphs + .twoSumByKey[Scalding,Int,Int,Int](source.map(_._2), testStoreA, keyExpand, testStoreB) + + val scald = Scalding("scalding-diamond-Job") + val intr = TestUtil.batchedCover(batcher, 0L, original.size.toLong) + val ws = new LoopState(intr) + val mode: Mode = TestMode((testStoreA.sourceToBuffer ++ testStoreB.sourceToBuffer ++ buffer).get(_)) + + scald.run(ws, mode, summer) + // Now check that the inMemory == + + TestUtil.compareMaps(original, Monoid.plus(initStore, inMemoryA), testStoreA, "A") must beTrue + TestUtil.compareMaps(original, Monoid.plus(initStore, inMemoryB), testStoreB, "B") must beTrue + } + + "Correctly propagates keys outside merge interval" in { + val frozenInitialData = sample[Set[(Int, Int)]] + val frozenKeys = frozenInitialData.map(_._1) + val liquidInitialData = sample[Set[(Int, Int)]] -- frozenInitialData + + val initialData = (frozenInitialData ++ liquidInitialData).toMap + + // Delta set should not intersect with the frozen keys + + val deltaSet = sample[List[(Int, Int)]].filterNot(kv => frozenKeys.contains(kv._1)) + + val summedDataInMemory = MapAlgebra.sumByKey(deltaSet) + val mergedDataInMemory = Monoid.plus(initialData, summedDataInMemory) + + val dataWithTime = deltaSet.zipWithIndex.map { case (item, time) => (time.toLong, item) } + val batcher = new MillisecondBatcher(1L) + + + def testFrozen(key: Int, range: Interval[Timestamp]): Boolean = { + frozenKeys.contains(key) + } + + val store = TestStore[Int, Int]("test", batcher, initialData, dataWithTime.size, boundedKeySpace = TimeBoundedKeySpace(testFrozen)) + + val (buffer, source) = TestSource(dataWithTime) + val summer = source.map(_._2).sumByKey(store) + + val scald = Scalding("scalding-keyFixTime-Job") + val mode: Mode = TestMode((store.sourceToBuffer ++ buffer).get(_)) + + scald.run(new LoopState(TestUtil.batchedCover(batcher, 0L, dataWithTime.size.toLong)), mode, summer) + + TestUtil.compareMaps(deltaSet, mergedDataInMemory, store, "store") must beTrue + } + + + "Should throw exception if frozen keys and delta's intersect" in { + val frozenInitialData = sample[Set[(Int, Int)]] + val frozenKeys = frozenInitialData.map(_._1) + val liquidInitialData = sample[Set[(Int, Int)]] -- frozenInitialData + + val initialData = (frozenInitialData ++ liquidInitialData).toMap + + // Delta set should not intersect with the frozen keys + + val deltaSet = sample[List[(Int, Int)]] ++ frozenInitialData + + val summedDataInMemory = MapAlgebra.sumByKey(deltaSet) + val mergedDataInMemory = Monoid.plus(initialData, summedDataInMemory) + + val dataWithTime = deltaSet.zipWithIndex.map { case (item, time) => (time.toLong, item) } + val batcher = new MillisecondBatcher(1L) + + + def testFrozen(key: Int, range: Interval[Timestamp]): Boolean = { + frozenKeys.contains(key) + } + + val store = TestStore[Int, Int]("test", batcher, initialData, dataWithTime.size, boundedKeySpace = TimeBoundedKeySpace(testFrozen)) + + val (buffer, source) = TestSource(dataWithTime) + val summer = source.map(_._2).sumByKey(store) + + val scald = Scalding("scalding-keyFixTime-Job") + val mode: Mode = TestMode((store.sourceToBuffer ++ buffer).get(_)) + + try { + scald.run(new ErrorThrowState(TestUtil.batchedCover(batcher, 0L, dataWithTime.size.toLong)), mode, summer) + false // Should never reach here + } + catch { + case _: Throwable => true + } + } + } +} diff --git a/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/ScaldingSerializationSpec.scala b/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/ScaldingSerializationSpec.scala new file mode 100644 index 000000000..4a24e8c04 --- /dev/null +++ b/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/ScaldingSerializationSpec.scala @@ -0,0 +1,75 @@ +/* + Copyright 2013 Twitter, Inc. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + */ + +package com.twitter.summingbird.scalding + +import com.twitter.algebird.{MapAlgebra, Monoid, Group, Interval, Last} +import com.twitter.algebird.monad._ +import com.twitter.summingbird.{Producer, TimeExtractor, TestGraphs} +import com.twitter.summingbird.batch._ +import com.twitter.summingbird.batch.state.HDFSState + +import java.util.TimeZone +import java.io.File + +import com.twitter.scalding.{ Source => ScaldingSource, Test => TestMode, _ } +import com.twitter.scalding.typed.TypedSink + +import org.scalacheck._ +import org.scalacheck.Prop._ +import org.scalacheck.Properties + +import org.apache.hadoop.conf.Configuration + +import scala.collection.JavaConverters._ +import scala.collection.mutable.{ArrayBuffer, Buffer, HashMap => MutableHashMap, Map => MutableMap, SynchronizedBuffer, SynchronizedMap} + +import cascading.scheme.local.{TextDelimited => CLTextDelimited} +import cascading.tuple.{Tuple, Fields, TupleEntry} +import cascading.flow.FlowDef +import cascading.tap.Tap +import cascading.scheme.NullScheme +import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.mapred.RecordReader +import org.apache.hadoop.mapred.OutputCollector + + +import org.specs2.mutable._ + +class ScaldingSerializationSpecs extends Specification { + implicit def tupleExtractor[T <: (Long, _)]: TimeExtractor[T] = TimeExtractor( _._1 ) + + "ScaldingPlatform" should { + "serialize Hadoop Jobs for single step jobs" in { + // Add a time: + val inWithTime = List(1, 2, 3).zipWithIndex.map { case (item, time) => (time.toLong, item) } + val batcher = TestUtil.randomBatcher(inWithTime) + val testStore = TestStore[Int,Int]("test", batcher, Iterable.empty, inWithTime.size) + val (buffer, source) = TestSource(inWithTime) + + val summer = TestGraphs.singleStepJob[Scalding,(Long, Int),Int,Int](source, testStore) { + tup => List((1 -> tup._2)) + } + + val mode = HadoopTest(new Configuration, {case x: ScaldingSource => buffer.get(x)}) + val intr = Interval.leftClosedRightOpen(Timestamp(0L), Timestamp(inWithTime.size.toLong)) + val scald = Scalding("scalaCheckJob") + + (try { scald.toFlow(intr, mode, scald.plan(summer)); true } + catch { case t: Throwable => println(toTry(t)); false }) must beTrue + } + } +} diff --git a/summingbird-scalding/src/test/scala/com/twitter/summingbird/scalding/TimePathedSourceTest.scala b/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/TimePathedSourceTest.scala similarity index 87% rename from summingbird-scalding/src/test/scala/com/twitter/summingbird/scalding/TimePathedSourceTest.scala rename to summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/TimePathedSourceTest.scala index 399fe4d3a..57c9d835c 100644 --- a/summingbird-scalding/src/test/scala/com/twitter/summingbird/scalding/TimePathedSourceTest.scala +++ b/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/TimePathedSourceTest.scala @@ -17,6 +17,7 @@ package com.twitter.summingbird.scalding import com.twitter.summingbird.scalding._ +import source.{TimePathedSource => BTimePathedSource} import com.twitter.summingbird._ import org.scalacheck._ import Gen._ @@ -28,7 +29,7 @@ import com.twitter.scalding.{DateRange, RichDate} case class TestData(requestedRange: DateRange, availableRange: DateRange, embiggen: Long) -object TimePathSourceLaws extends Properties("Time path source") { +object TimePathSourceLaws extends Properties("Time path source") { implicit def arbRateRange: Arbitrary[DateRange] = Arbitrary(for { startTs <- Gen.choose(-137878042589500L, 137878042589500L) @@ -36,20 +37,20 @@ object TimePathSourceLaws extends Properties("Time path source") { endTsDelta <- Gen.choose(10L, 137878042589500L) endDate <- RichDate(startTs + endTsDelta) } yield DateRange(startDate, endDate)) - + implicit def arbData = Arbitrary(for { reqRange <- arbitrary[DateRange] availableRange <- arbitrary[DateRange] embiggenVal <- Gen.choose(1L, 100000L) embiggen <- Gen.oneOf(embiggenVal, 0L) } yield TestData(reqRange, availableRange, embiggen)) - + def genEmbiggen(embiggen: Long): (DateRange => DateRange) = { ((dr: DateRange) => DateRange(RichDate(dr.start.timestamp - embiggen), RichDate(dr.end.timestamp + embiggen))) } def genVertractor(availableRange: DateRange): (DateRange => Option[DateRange]) = { - {(dr: DateRange) => + {(dr: DateRange) => val botTs = max(dr.start.timestamp, availableRange.start.timestamp) val topTs = min(dr.end.timestamp, availableRange.end.timestamp) if (botTs > topTs) None else Some(DateRange(RichDate(botTs), RichDate(topTs))) @@ -61,10 +62,10 @@ object TimePathSourceLaws extends Properties("Time path source") { } def rangeLength(dr: DateRange): Long = dr.end.timestamp - dr.start.timestamp + 1 - + property("if the reqRange + embiggen is inside the avail range, return should == requested") = forAll { (data: TestData) => - val retData = TimePathedSource.minify(genEmbiggen(data.embiggen), genVertractor(data.availableRange))(data.requestedRange) + val retData = BTimePathedSource.minify(genEmbiggen(data.embiggen), genVertractor(data.availableRange))(data.requestedRange) if(rangeWithEmbgginContained(data.requestedRange, data.embiggen, data.availableRange)) { retData == Some(data.requestedRange) @@ -72,7 +73,7 @@ object TimePathSourceLaws extends Properties("Time path source") { } property("If not a complete subset, but overlapping we can imply a few prerequisites") = forAll { (data: TestData) => - val retData = TimePathedSource.minify(genEmbiggen(data.embiggen), genVertractor(data.availableRange))(data.requestedRange) + val retData = BTimePathedSource.minify(genEmbiggen(data.embiggen), genVertractor(data.availableRange))(data.requestedRange) retData match { case None => true case Some(range) => @@ -85,9 +86,9 @@ object TimePathSourceLaws extends Properties("Time path source") { property("If the return is none, then the ranges should be disjoint or one is None") = forAll { (data: TestData) => - val retData = TimePathedSource.minify(genEmbiggen(data.embiggen), genVertractor(data.availableRange))(data.requestedRange) + val retData = BTimePathedSource.minify(genEmbiggen(data.embiggen), genVertractor(data.availableRange))(data.requestedRange) retData match { - case None => + case None => ( rangeLength(data.requestedRange) == 0 || rangeLength(data.availableRange) == 0 @@ -95,7 +96,7 @@ object TimePathSourceLaws extends Properties("Time path source") { || data.requestedRange.isAfter(data.availableRange.end) // Disjoint ) case Some(_) => true // Not in this test - } + } } } diff --git a/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/VersionBatchLaws.scala b/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/VersionBatchLaws.scala new file mode 100644 index 000000000..876668edf --- /dev/null +++ b/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/VersionBatchLaws.scala @@ -0,0 +1,77 @@ +/* + Copyright 2013 Twitter, Inc. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + */ + +package com.twitter.summingbird.scalding + +import com.twitter.algebird.{MapAlgebra, Monoid, Group, Interval, Last} +import com.twitter.algebird.monad._ +import com.twitter.summingbird.{Producer, TimeExtractor, TestGraphs} +import com.twitter.summingbird.batch._ +import com.twitter.summingbird.batch.state.HDFSState + +import java.util.TimeZone +import java.io.File + +import com.twitter.scalding.{ Source => ScaldingSource, Test => TestMode, _ } +import com.twitter.scalding.typed.TypedSink + +import org.scalacheck._ +import org.scalacheck.Prop._ +import org.scalacheck.Properties + +import org.apache.hadoop.conf.Configuration + +import scala.collection.JavaConverters._ +import scala.collection.mutable.{ArrayBuffer, Buffer, HashMap => MutableHashMap, Map => MutableMap, SynchronizedBuffer, SynchronizedMap} + +import cascading.scheme.local.{TextDelimited => CLTextDelimited} +import cascading.tuple.{Tuple, Fields, TupleEntry} +import cascading.flow.FlowDef +import cascading.tap.Tap +import cascading.scheme.NullScheme +import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.mapred.RecordReader +import org.apache.hadoop.mapred.OutputCollector + + +import org.specs2.mutable._ + +object VersionBatchLaws extends Properties("VersionBatchLaws") { + property("version -> BatchID -> version") = forAll { (l: Long) => + val vbs = new store.VersionedBatchStore[Int, Int, Array[Byte], Array[Byte]](null, + 0, Batcher.ofHours(1))(null)(null) + val b = vbs.versionToBatchID(l) + vbs.batchIDToVersion(b) <= l + } + property("BatchID -> version -> BatchID") = forAll { (bint: Int) => + val b = BatchID(bint) + val vbs = new store.VersionedBatchStore[Int, Int, Array[Byte], Array[Byte]](null, + 0, Batcher.ofHours(1))(null)(null) + val v = vbs.batchIDToVersion(b) + vbs.versionToBatchID(v) == b + } + property("version is an upperbound on time") = forAll { (lBig: Long) => + val l = lBig/1000L + val batcher = Batcher.ofHours(1) + val vbs = new store.VersionedBatchStore[Int, Int, Array[Byte], Array[Byte]](null, + 0, batcher)(null)(null) + val b = vbs.versionToBatchID(l) + (batcher.earliestTimeOf(b.next).milliSinceEpoch <= l) && + (batcher.earliestTimeOf(b).milliSinceEpoch < l) + (batcher.earliestTimeOf(b.next.next).milliSinceEpoch > l) + } +} + diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/Executor.scala b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/Executor.scala index 0dc8ba188..6eb1bdff2 100644 --- a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/Executor.scala +++ b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/Executor.scala @@ -18,7 +18,8 @@ package com.twitter.summingbird.scalding import com.twitter.summingbird._ import com.twitter.scalding.{RichDate, DateParser, Hdfs, Args} -import com.twitter.summingbird.batch.Timestamp +import com.twitter.summingbird.batch.{Timestamp, WaitingState} +import com.twitter.summingbird.batch.option.{FlatMapShards, Reducers} import com.twitter.summingbird.chill.ChillExecutionConfig import com.twitter.algebird.Interval @@ -26,7 +27,6 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.util.GenericOptionsParser import java.util.TimeZone -import option.{FlatMapShards, Reducers} import org.slf4j.LoggerFactory /** @@ -35,7 +35,11 @@ import org.slf4j.LoggerFactory trait ScaldingExecutionConfig extends ChillExecutionConfig[Scalding] { - def getWaitingState(hadoopConfig: Configuration, startDate: Option[Timestamp], batches: Int): WaitingState[Interval[Timestamp]] + def getWaitingState( + hadoopConfig: Configuration, + startDate: Option[Timestamp], + batches: Int + ): WaitingState[Interval[Timestamp]] } object Executor { diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/PipeFactoryOps.scala b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/PipeFactoryOps.scala new file mode 100644 index 000000000..bc30eac6f --- /dev/null +++ b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/PipeFactoryOps.scala @@ -0,0 +1,37 @@ +/* + Copyright 2013 Twitter, Inc. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + */ + +package com.twitter.summingbird.scalding + +import com.twitter.summingbird.batch.Timestamp +import com.twitter.scalding.TypedPipe + +class PipeFactoryOps[+T](pipeFactory: PipeFactory[T]) { + + def flatMapElements[U](fn: (T => TraversableOnce[U])): PipeFactory[U] = + mapPipe(_.flatMap { case (time, tup) => + fn(tup).map((time, _)) + }) + + def mapElements[U](fn: (T => U)): PipeFactory[U] = + flatMapElements({tup => List(fn(tup))}) + + def mapPipe[U](fn: (TypedPipe[(Timestamp, T)] => TypedPipe[(Timestamp, U)])): PipeFactory[U] = { + pipeFactory.map { flowProducer => + flowProducer.map(fn(_)) + } + } +} diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/ScaldingPlatform.scala b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/ScaldingPlatform.scala index 70cf44051..6202f446d 100644 --- a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/ScaldingPlatform.scala +++ b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/ScaldingPlatform.scala @@ -17,14 +17,24 @@ package com.twitter.summingbird.scalding import com.twitter.algebird.{ Monoid, Semigroup, Monad } -import com.twitter.algebird.{ Universe, Empty, Interval, Intersection, InclusiveLower, ExclusiveLower, ExclusiveUpper, InclusiveUpper } +import com.twitter.algebird.{ + Universe, + Empty, + Interval, + Intersection, + InclusiveLower, + ExclusiveLower, + ExclusiveUpper, + InclusiveUpper +} import com.twitter.algebird.monad.{ StateWithError, Reader } import com.twitter.bijection.Conversion.asMethod -import com.twitter.bijection.Injection +import com.twitter.bijection.{AbstractInjection, Injection} import com.twitter.scalding.{ Tool => STool, Source => SSource, TimePathedSource => STPS, _} import com.twitter.summingbird._ -import com.twitter.summingbird.scalding.option.{ FlatMapShards, Reducers } +import com.twitter.summingbird.batch.option.{ FlatMapShards, Reducers } import com.twitter.summingbird.batch._ +import com.twitter.summingbird.scalding.source.{TimePathedSource => BTimePathedSource} import com.twitter.chill.IKryoRegistrar import com.twitter.summingbird.chill._ import com.twitter.summingbird.option._ @@ -50,25 +60,26 @@ object Scalding { new Scalding(jobName, options, identity, List()) } - implicit val dateRangeInjection: Injection[DateRange, Interval[Time]] = Injection.build { - (dr: DateRange) => { - val DateRange(l, u) = dr - Interval.leftClosedRightOpen(l.timestamp, u.timestamp + 1L) - } - } { - case Intersection(lb, ub) => - val low = lb match { - case InclusiveLower(l) => l - case ExclusiveLower(l) => l+1L + implicit val dateRangeInjection: Injection[DateRange, Interval[Timestamp]] = + new AbstractInjection[DateRange, Interval[Timestamp]] { + override def apply(dr: DateRange) = { + val DateRange(l, u) = dr + Intersection(InclusiveLower(Timestamp(l.timestamp)), ExclusiveUpper(Timestamp(u.timestamp + 1L))) } - val high = ub match { - case InclusiveUpper(u) => u - case ExclusiveUpper(u) => u-1L + override def invert(in: Interval[Timestamp]) = in match { + case Intersection(lb, ub) => + val low = lb match { + case InclusiveLower(l) => l + case ExclusiveLower(l) => l.next + } + val high = ub match { + case InclusiveUpper(u) => u + case ExclusiveUpper(u) => u.prev + } + Success(DateRange(low.toRichDate, high.toRichDate)) + case _ => Failure(new RuntimeException("Unbounded interval!")) } - Success(DateRange(RichDate(low), RichDate(high))) - case _ => Failure(new RuntimeException("Unbounded interval!")) - } - + } def emptyFlowProducer[T]: FlowProducer[TypedPipe[T]] = Reader({ implicit fdm: (FlowDef, Mode) => TypedPipe.empty }) @@ -78,9 +89,9 @@ object Scalding { s: Summer[Scalding, _, _]): Commutativity = { val commutativity = getOrElse(options, names, s, { - logger.warn("Store: {} has no commutativity setting. Assuming {}", - names, MonoidIsCommutative.default) - MonoidIsCommutative.default + val default = MonoidIsCommutative.default + logger.warn("Store: %s has no commutativity setting. Assuming %s".format(names, default)) + default }).commutativity commutativity match { @@ -94,7 +105,7 @@ object Scalding { } def intersect(dr1: DateRange, dr2: DateRange): Option[DateRange] = - (dr1.as[Interval[Time]] && (dr2.as[Interval[Time]])).as[Option[DateRange]] + (dr1.as[Interval[Timestamp]] && (dr2.as[Interval[Timestamp]])).as[Option[DateRange]] /** Given a constructor function, computes the maximum available range * of time or gives an error. @@ -107,7 +118,7 @@ object Scalding { try { val available = (mode, factory(desired)) match { case (hdfs: Hdfs, ts: STPS) => - TimePathedSource.satisfiableHdfs(hdfs, desired, factory.asInstanceOf[DateRange => STPS]) + BTimePathedSource.satisfiableHdfs(hdfs, desired, factory.asInstanceOf[DateRange => STPS]) case _ => bisectingMinify(mode, desired)(factory) } available.flatMap { intersect(desired, _) } @@ -150,20 +161,42 @@ object Scalding { */ def pipeFactory[T](factory: (DateRange) => Mappable[T]) (implicit timeOf: TimeExtractor[T]): PipeFactory[T] = - StateWithError[(Interval[Time], Mode), List[FailureReason], FlowToPipe[T]]{ - (timeMode: (Interval[Time], Mode)) => { + optionMappedPipeFactory(factory)(t => Some(t)) + + /** + * Like pipeFactory, but allows the output of the factory to be mapped. + * + * Useful when using TextLine, for example, where the lines need to be + * parsed before you can extract the timestamps. + */ + def mappedPipeFactory[T,U](factory: (DateRange) => Mappable[T])(fn: T => U) + (implicit timeOf: TimeExtractor[U]): PipeFactory[U] = + optionMappedPipeFactory(factory)(t => Some(fn(t))) + + /** + * Like pipeFactory, but allows the output of the factory to be mapped to an optional value. + * + * Useful when using TextLine, for example, where the lines need to be + * parsed before you can extract the timestamps. + */ + def optionMappedPipeFactory[T,U](factory: (DateRange) => Mappable[T])(fn: T => Option[U]) + (implicit timeOf: TimeExtractor[U]): PipeFactory[U] = + StateWithError[(Interval[Timestamp], Mode), List[FailureReason], FlowToPipe[U]]{ + (timeMode: (Interval[Timestamp], Mode)) => { val (timeSpan, mode) = timeMode toDateRange(timeSpan).right.flatMap { dr => minify(mode, dr)(factory) .right.map { newDr => - val newIntr = newDr.as[Interval[Time]] + val newIntr = newDr.as[Interval[Timestamp]] val mappable = factory(newDr) ((newIntr, mode), Reader { (fdM: (FlowDef, Mode)) => TypedPipe.from(mappable)(fdM._1, fdM._2) .flatMap { t => - val time = timeOf(t) - if(newIntr(time)) Some((time, t)) else None + fn(t).flatMap { mapped => + val time = Timestamp(timeOf(mapped)) + if(newIntr(time)) Some((time, mapped)) else None + } } }) } @@ -173,8 +206,8 @@ object Scalding { def pipeFactoryExact[T](factory: (DateRange) => Mappable[T]) (implicit timeOf: TimeExtractor[T]): PipeFactory[T] = - StateWithError[(Interval[Time], Mode), List[FailureReason], FlowToPipe[T]]{ - (timeMode: (Interval[Time], Mode)) => { + StateWithError[(Interval[Timestamp], Mode), List[FailureReason], FlowToPipe[T]]{ + (timeMode: (Interval[Timestamp], Mode)) => { val (timeSpan, mode) = timeMode toDateRange(timeSpan).right.map { dr => @@ -183,7 +216,7 @@ object Scalding { mappable.validateTaps(fdM._2) //This can throw, but that is what this caller wants TypedPipe.from(mappable)(fdM._1, fdM._2) .flatMap { t => - val time = timeOf(t) + val time = Timestamp(timeOf(t)) if(timeSpan(time)) Some((time, t)) else None } }) @@ -195,12 +228,12 @@ object Scalding { factory: (DateRange) => Mappable[T]): Producer[Scalding, T] = Producer.source[Scalding, T](pipeFactory(factory)) - def toDateRange(timeSpan: Interval[Time]): Try[DateRange] = + def toDateRange(timeSpan: Interval[Timestamp]): Try[DateRange] = timeSpan.as[Option[DateRange]] .map { Right(_) } .getOrElse(Left(List("only finite time ranges are supported by scalding: " + timeSpan.toString))) - def limitTimes[T](range: Interval[Time], in: FlowToPipe[T]): FlowToPipe[T] = + def limitTimes[T](range: Interval[Timestamp], in: FlowToPipe[T]): FlowToPipe[T] = in.map { pipe => pipe.filter { case (time, _) => range(time) } } def merge[T](left: FlowToPipe[T], right: FlowToPipe[T]): FlowToPipe[T] = @@ -239,7 +272,7 @@ object Scalding { maybePair match { case None => - logger.debug("Producer ({}): Using default setting {}", producer.getClass.getName, default) + logger.debug("Producer (%s): Using default setting %s".format(producer.getClass.getName, default)) default case Some((id, opt)) => logger.info("Producer ({}) Using {} found via NamedProducer \"{}\"", Array[AnyRef](producer.getClass.getName, opt, id)) @@ -291,7 +324,7 @@ object Scalding { val srcPf = if (shards <= 1) src else - src.map(_.map(_.shard(shards))) + src.mapPipe(_.shard(shards)) (srcPf, built) } @@ -373,7 +406,7 @@ object Scalding { val fmpSharded = if (shards < 1) fmp else - fmp.map(_.map(_.shard(shards))) + fmp.mapPipe(_.shard(shards)) (fmpSharded.map { flowP => flowP.map { typedPipe => @@ -437,8 +470,8 @@ object Scalding { */ def toPipe[T](dr: DateRange, prod: Producer[Scalding, T], - opts: Map[String, Options] = Map.empty)(implicit fd: FlowDef, mode: Mode): Try[(DateRange, TypedPipe[(Long, T)])] = { - val ts = dr.as[Interval[Time]] + opts: Map[String, Options] = Map.empty)(implicit fd: FlowDef, mode: Mode): Try[(DateRange, TypedPipe[(Timestamp, T)])] = { + val ts = dr.as[Interval[Timestamp]] val pf = planProducer(opts, prod) toPipe(ts, fd, mode, pf).right.map { case (ts, pipe) => (ts.as[Option[DateRange]].get, pipe) @@ -450,23 +483,23 @@ object Scalding { */ def toPipeExact[T](dr: DateRange, prod: Producer[Scalding, T], - opts: Map[String, Options] = Map.empty)(implicit fd: FlowDef, mode: Mode): Try[TypedPipe[(Long, T)]] = { - val ts = dr.as[Interval[Time]] + opts: Map[String, Options] = Map.empty)(implicit fd: FlowDef, mode: Mode): Try[TypedPipe[(Timestamp, T)]] = { + val ts = dr.as[Interval[Timestamp]] val pf = planProducer(opts, prod) toPipeExact(ts, fd, mode, pf) } - def toPipe[T](timeSpan: Interval[Time], + def toPipe[T](timeSpan: Interval[Timestamp], flowDef: FlowDef, mode: Mode, - pf: PipeFactory[T]): Try[(Interval[Time], TimedPipe[T])] = { + pf: PipeFactory[T]): Try[(Interval[Timestamp], TimedPipe[T])] = { logger.info("Planning on interval: {}", timeSpan.as[Option[DateRange]]) pf((timeSpan, mode)) .right .map { case (((ts, m), flowDefMutator)) => (ts, flowDefMutator((flowDef, m))) } } - def toPipeExact[T](timeSpan: Interval[Time], + def toPipeExact[T](timeSpan: Interval[Timestamp], flowDef: FlowDef, mode: Mode, pf: PipeFactory[T]): Try[TimedPipe[T]] = { @@ -508,9 +541,9 @@ class Scalding( extends Platform[Scalding] { type Source[T] = PipeFactory[T] - type Store[K, V] = ScaldingStore[K, V] - type Sink[T] = ScaldingSink[T] - type Service[K, V] = ScaldingService[K, V] + type Store[K, V] = scalding.Store[K, V] + type Sink[T] = scalding.Sink[T] + type Service[K, V] = scalding.Service[K, V] type Plan[T] = PipeFactory[T] def plan[T](prod: TailProducer[Scalding, T]): PipeFactory[T] = @@ -558,10 +591,24 @@ class Scalding( } private def setIoSerializations(c: Configuration): Unit = - c.set("io.serializations", ioSerializations.map { _.getName }.mkString(",")) + c.set("io.serializations", ioSerializations.map { _.getName }.mkString(",")) + + private val HADOOP_DEFAULTS = Map( + ("mapred.output.compression.type", "BLOCK"), + ("io.compression.codec.lzo.compression.level", "3"), + ("mapred.output.compress", "true"), + ("mapred.compress.map.output", "true"), + ("mapreduce.output.fileoutputformat.compress", "true"), + ("mapreduce.output.fileoutputformat.compress.codec", "com.hadoop.compression.lzo.LzoCodec") + ) + + private def setHadoopConfigDefaults(c: Configuration): Unit = + HADOOP_DEFAULTS.foreach { case (k, v) => + c.set(k, v) + } // This is a side-effect-free computation that is called by run - def toFlow(timeSpan: Interval[Time], mode: Mode, pf: PipeFactory[_]): Try[(Interval[Time], Flow[_])] = { + def toFlow(timeSpan: Interval[Timestamp], mode: Mode, pf: PipeFactory[_]): Try[(Interval[Timestamp], Flow[_])] = { val flowDef = new FlowDef flowDef.setName(jobName) Scalding.toPipe(timeSpan, flowDef, mode, pf) @@ -587,20 +634,20 @@ class Scalding( mode match { case Hdfs(_, conf) => + // Set these before the user settings, so that the user + // can change them if needed + setHadoopConfigDefaults(conf) updateConfig(conf) setIoSerializations(conf) case _ => } - - val prepareState = state.begin - val timeSpan = prepareState.requested.mapNonDecreasing(_.milliSinceEpoch) - toFlow(timeSpan, mode, pf) match { + toFlow(prepareState.requested, mode, pf) match { case Left(errs) => prepareState.fail(FlowPlanException(errs)) case Right((ts,flow)) => - prepareState.willAccept(ts.mapNonDecreasing(Timestamp(_))) match { + prepareState.willAccept(ts) match { case Right(runningState) => try { options.get(jobName).foreach { jopt => diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/Service.scala b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/Service.scala new file mode 100644 index 000000000..33a6823df --- /dev/null +++ b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/Service.scala @@ -0,0 +1,22 @@ +/* + Copyright 2013 Twitter, Inc. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + */ + +package com.twitter.summingbird.scalding + +trait Service[K, +V] extends java.io.Serializable { + // A static, or write-once service can potentially optimize this without writing the (K, V) stream out + def lookup[W](getKeys: PipeFactory[(K, W)]): PipeFactory[(K, (W, Option[V]))] +} diff --git a/summingbird-storm/src/main/scala/com/twitter/summingbird/util/package.scala b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/Sink.scala similarity index 68% rename from summingbird-storm/src/main/scala/com/twitter/summingbird/util/package.scala rename to summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/Sink.scala index a11b1d30c..b7737dc06 100644 --- a/summingbird-storm/src/main/scala/com/twitter/summingbird/util/package.scala +++ b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/Sink.scala @@ -14,12 +14,8 @@ limitations under the License. */ -package com.twitter.summingbird +package com.twitter.summingbird.scalding -package object util { - @deprecated("Use com.twitter.summingbird.option.CacheSize", "0.1.0") - type CacheSize = option.CacheSize - - @deprecated("Use com.twitter.summingbird.option.CacheSize", "0.1.0") - val CacheSize = option.CacheSize +trait Sink[T] { + def write(incoming: PipeFactory[T]): PipeFactory[T] } diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/Store.scala b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/Store.scala new file mode 100644 index 000000000..18938c136 --- /dev/null +++ b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/Store.scala @@ -0,0 +1,61 @@ +/* + Copyright 2013 Twitter, Inc. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + */ + +package com.twitter.summingbird.scalding + +import com.twitter.algebird.Semigroup +import com.twitter.algebird.monad.{StateWithError, Reader} +import com.twitter.scalding.{Dsl, TypedPipe, MapsideReduce, TupleSetter, TupleConverter} +import com.twitter.summingbird._ +import com.twitter.summingbird.option._ +import cascading.flow.FlowDef + + +import org.slf4j.LoggerFactory + +object Store extends java.io.Serializable { + // This could be moved to scalding, but the API needs more design work + // This DOES NOT trigger a grouping + def mapsideReduce[K,V](pipe: TypedPipe[(K, V)])(implicit sg: Semigroup[V]): TypedPipe[(K, V)] = { + import Dsl._ + val fields = ('key, 'value) + val gpipe = pipe.toPipe(fields)(TupleSetter.tup2Setter[(K,V)]) + val msr = new MapsideReduce(sg, fields._1, fields._2, None)( + TupleConverter.singleConverter[V], TupleSetter.singleSetter[V]) + TypedPipe.from(gpipe.eachTo(fields -> fields) { _ => msr }, fields)(TupleConverter.of[(K, V)]) + } +} + +trait Store[K, V] extends java.io.Serializable { + /** + * Accepts deltas along with their timestamps, returns triples of + * (time, K, V(aggregated up to the time)). + * + * Same return as lookup on a ScaldingService. + */ + def merge(delta: PipeFactory[(K, V)], + sg: Semigroup[V], + commutativity: Commutativity, + reducers: Int): PipeFactory[(K, (Option[V], V))] + + /** This is an optional method, by default it a pass-through. + * it may be called by ScaldingPlatform before a key transformation + * that leads only to this store. + */ + def partialMerge[K1](delta: PipeFactory[(K1, V)], + sg: Semigroup[V], + commutativity: Commutativity): PipeFactory[(K1, V)] = delta +} diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/BatchedOperation.scala b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedOperations.scala similarity index 68% rename from summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/BatchedOperation.scala rename to summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedOperations.scala index 2fb7e71a0..a4b480668 100644 --- a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/BatchedOperation.scala +++ b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedOperations.scala @@ -14,12 +14,13 @@ limitations under the License. */ -package com.twitter.summingbird.scalding +package com.twitter.summingbird.scalding.batch import com.twitter.summingbird.batch.{ BatchID, Batcher } import com.twitter.algebird.{ Universe, Empty, Interval, Intersection, InclusiveLower, ExclusiveUpper, InclusiveUpper } import com.twitter.bijection.{Injection, Bijection, Conversion} import com.twitter.summingbird.batch.Timestamp +import com.twitter.summingbird.scalding._ import com.twitter.scalding.Mode import Conversion.asMethod @@ -27,29 +28,24 @@ import Conversion.asMethod /** Services and Stores are very similar, but not exact. * This shares the logic for them. */ -class BatchedOperations(batcher: Batcher) { +private class BatchedOperations(batcher: Batcher) { - implicit val timeToBatchInterval = Bijection.build { bint: Interval[Time] => - bint.mapNonDecreasing { Timestamp(_) } } { bint: Interval[Timestamp] => - bint.mapNonDecreasing { _.milliSinceEpoch } - } - - def coverIt[T](timeSpan: Interval[Time]): Iterable[BatchID] = { + def coverIt[T](timeSpan: Interval[Timestamp]): Iterable[BatchID] = { val batchInterval = batcher.cover(timeSpan.as[Interval[Timestamp]]) BatchID.toIterable(batchInterval) } - def batchToTime(bint: Interval[BatchID]): Interval[Time] = - bint.mapNonDecreasing { batcher.earliestTimeOf(_).milliSinceEpoch } + def batchToTimestamp(bint: Interval[BatchID]): Interval[Timestamp] = + bint.mapNonDecreasing { batcher.earliestTimeOf(_) } - def intersect(batches: Interval[BatchID], ts: Interval[Time]): Interval[Time] = - batchToTime(batches) && ts + def intersect(batches: Interval[BatchID], ts: Interval[Timestamp]): Interval[Timestamp] = + batchToTimestamp(batches) && ts - def intersect(batches: Iterable[BatchID], ts: Interval[Time]): Option[Interval[Time]] = + def intersect(batches: Iterable[BatchID], ts: Interval[Timestamp]): Option[Interval[Timestamp]] = BatchID.toInterval(batches).map { intersect(_, ts) } def readBatched[T](inBatches: Interval[BatchID], mode: Mode, in: PipeFactory[T]): Try[(Interval[BatchID], FlowToPipe[T])] = { - val inTimes = batchToTime(inBatches) + val inTimes = batchToTimestamp(inBatches) // Read the delta stream for the needed times in((inTimes, mode)) .right diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/ScaldingService.scala b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedService.scala similarity index 81% rename from summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/ScaldingService.scala rename to summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedService.scala index 196b361d1..b1d78b508 100644 --- a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/ScaldingService.scala +++ b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedService.scala @@ -14,25 +14,17 @@ limitations under the License. */ -package com.twitter.summingbird.scalding +package com.twitter.summingbird.scalding.batch import com.twitter.algebird.monad.{StateWithError, Reader} import com.twitter.algebird.{Interval, Semigroup} import com.twitter.scalding.{Mode, TypedPipe} import com.twitter.summingbird.batch.{ BatchID, Batcher, Timestamp } +import com.twitter.summingbird.scalding._ +import com.twitter.summingbird.scalding import cascading.flow.FlowDef -trait ScaldingService[K, +V] extends java.io.Serializable { - // A static, or write-once service can potentially optimize this without writing the (K, V) stream out - def lookup[W](getKeys: PipeFactory[(K, W)]): PipeFactory[(K, (W, Option[V]))] -} - -class EmptyService[K, V] extends ScaldingService[K, V] { - def lookup[W](getKeys: PipeFactory[(K, W)]): PipeFactory[(K, (W, Option[V]))] = - getKeys.map { _.map { _.map { case (t, (k, v)) => (t, (k, (v, None: Option[V]))) } } } -} - -trait BatchedService[K, V] extends ScaldingService[K, V] { +trait BatchedService[K, V] extends Service[K, V] { // The batcher that describes this service def batcher: Batcher def ordering: Ordering[K] @@ -54,8 +46,8 @@ trait BatchedService[K, V] extends ScaldingService[K, V] { * You are guaranteed that all the service data needed * to do the join is present. */ - def lookup[W](incoming: TypedPipe[(Time, (K, W))], - servStream: TypedPipe[(Time, (K, Option[V]))]): TypedPipe[(Time, (K, (W, Option[V])))] = { + def lookup[W](incoming: TypedPipe[(Timestamp, (K, W))], + servStream: TypedPipe[(Timestamp, (K, Option[V]))]): TypedPipe[(Timestamp, (K, (W, Option[V])))] = { def flatOpt[T](o: Option[Option[T]]): Option[T] = o.flatMap(identity) @@ -64,13 +56,13 @@ trait BatchedService[K, V] extends ScaldingService[K, V] { .map { case (t, (k, (w, optoptv))) => (t, (k, (w, flatOpt(optoptv)))) } } - protected def batchedLookup[W](covers: Interval[Time], + protected def batchedLookup[W](covers: Interval[Timestamp], getKeys: FlowToPipe[(K, W)], last: (BatchID, FlowProducer[TypedPipe[(K, V)]]), streams: Iterable[(BatchID, FlowToPipe[(K, Option[V])])]): FlowToPipe[(K, (W, Option[V]))] = Reader[FlowInput, KeyValuePipe[K, (W, Option[V])]] { (flowMode: (FlowDef, Mode)) => val left = getKeys(flowMode) - val earliestInLast = batcher.earliestTimeOf(last._1).milliSinceEpoch + val earliestInLast = batcher.earliestTimeOf(last._1) val liftedLast: KeyValuePipe[K, Option[V]] = last._2(flowMode) .map { case (k, w) => (earliestInLast, (k, Some(w))) } // TODO (https://github.com/twitter/summingbird/issues/91): we @@ -128,8 +120,8 @@ object BatchedService extends java.io.Serializable { * a BatchedService * Assumes the batcher is the same for both */ - def fromStoreAndSink[K,V](store: BatchedScaldingStore[K, V], - sink: BatchedScaldingSink[(K, Option[V])], + def fromStoreAndSink[K,V](store: BatchedStore[K, V], + sink: BatchedSink[(K, Option[V])], reducerOption: Option[Int] = None): BatchedService[K, V] = new BatchedService[K, V] { override def ordering = store.ordering override def batcher = { @@ -147,8 +139,8 @@ object BatchedService extends java.io.Serializable { * a BatchedService * Assumes the batcher is the same for both */ - def fromStoreAndDeltaSink[K,V:Semigroup](store: BatchedScaldingStore[K, V], - sink: BatchedScaldingSink[(K, V)], - reducerOption: Option[Int] = None): BatchedDeltaService[K, V] = - new BatchedDeltaService[K, V](store, sink, reducerOption) + def fromStoreAndDeltaSink[K,V:Semigroup](store: BatchedStore[K, V], + sink: BatchedSink[(K, V)], + reducerOption: Option[Int] = None): scalding.service.BatchedDeltaService[K, V] = + new scalding.service.BatchedDeltaService[K, V](store, sink, reducerOption) } diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/ScaldingSink.scala b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedSink.scala similarity index 88% rename from summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/ScaldingSink.scala rename to summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedSink.scala index 6b4cafb90..e904f177b 100644 --- a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/ScaldingSink.scala +++ b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedSink.scala @@ -14,20 +14,16 @@ limitations under the License. */ -package com.twitter.summingbird.scalding +package com.twitter.summingbird.scalding.batch import com.twitter.algebird.monad.{StateWithError, Reader} -import com.twitter.algebird.{ Universe, Empty, Interval, Intersection, InclusiveLower, ExclusiveUpper, InclusiveUpper } +import com.twitter.algebird.{ Interval, Intersection, InclusiveLower, ExclusiveUpper, InclusiveUpper } import com.twitter.summingbird.batch.{ BatchID, Batcher, Timestamp } -import com.twitter.scalding.{Mode, TypedPipe} -import com.twitter.scalding.typed.Grouped +import com.twitter.summingbird.scalding._ +import com.twitter.scalding.Mode import cascading.flow.FlowDef -trait ScaldingSink[T] { - def write(incoming: PipeFactory[T]): PipeFactory[T] -} - -trait BatchedScaldingSink[T] extends ScaldingSink[T] { +trait BatchedSink[T] extends Sink[T] { def batcher: Batcher /** If this full stream for this batch is already materialized, return it @@ -52,7 +48,7 @@ trait BatchedScaldingSink[T] extends ScaldingSink[T] { // We need to write each of these. iter.foreach { batch => - val range = batcher.toInterval(batch).mapNonDecreasing { _.milliSinceEpoch } + val range = batcher.toInterval(batch) writeStream(batch, inPipe.filter { case (time, _) => range(time) })(flowMode._1, flowMode._2) @@ -89,7 +85,7 @@ trait BatchedScaldingSink[T] extends ScaldingSink[T] { .takeWhile { _._2.isDefined } .collect { case (batch, Some(flow)) => (batch, flow) } - def mergeExistingAndBuilt(optBuilt: Option[(Interval[BatchID], FlowToPipe[T])]): Try[((Interval[Time], Mode), FlowToPipe[T])] = { + def mergeExistingAndBuilt(optBuilt: Option[(Interval[BatchID], FlowToPipe[T])]): Try[((Interval[Timestamp], Mode), FlowToPipe[T])] = { val (aBatches, aFlows) = existing.unzip val flows = aFlows ++ (optBuilt.map { _._2 }) val batches = aBatches ++ (optBuilt.map { pair => BatchID.toIterable(pair._1) }.getOrElse(Iterable.empty)) diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/ScaldingStore.scala b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedStore.scala similarity index 72% rename from summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/ScaldingStore.scala rename to summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedStore.scala index 99c0f97b5..cf957698c 100644 --- a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/ScaldingStore.scala +++ b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedStore.scala @@ -14,8 +14,7 @@ limitations under the License. */ -package com.twitter.summingbird.scalding - +package com.twitter.summingbird.scalding.batch import com.twitter.algebird.bijection.BijectedSemigroup import com.twitter.algebird.{Monoid, Semigroup} @@ -24,48 +23,16 @@ import com.twitter.algebird.monad.{StateWithError, Reader} import com.twitter.bijection.{ Bijection, ImplicitBijection } import com.twitter.scalding.{Dsl, Mode, TypedPipe, IterableSource, MapsideReduce, TupleSetter, TupleConverter} import com.twitter.scalding.typed.Grouped +import com.twitter.summingbird.scalding._ +import com.twitter.summingbird.scalding import com.twitter.summingbird._ import com.twitter.summingbird.option._ -import com.twitter.summingbird.batch.{ BatchID, Batcher, Timestamp} +import com.twitter.summingbird.batch._ import cascading.flow.FlowDef import org.slf4j.LoggerFactory -object ScaldingStore extends java.io.Serializable { - // This could be moved to scalding, but the API needs more design work - // This DOES NOT trigger a grouping - def mapsideReduce[K,V](pipe: TypedPipe[(K, V)])(implicit sg: Semigroup[V]): TypedPipe[(K, V)] = { - import Dsl._ - val fields = ('key, 'value) - val gpipe = pipe.toPipe(fields)(TupleSetter.tup2Setter[(K,V)]) - val msr = new MapsideReduce(sg, fields._1, fields._2, None)( - TupleConverter.singleConverter[V], TupleSetter.singleSetter[V]) - TypedPipe.from(gpipe.eachTo(fields -> fields) { _ => msr }, fields)(TupleConverter.of[(K, V)]) - } -} - -trait ScaldingStore[K, V] extends java.io.Serializable { - /** - * Accepts deltas along with their timestamps, returns triples of - * (time, K, V(aggregated up to the time)). - * - * Same return as lookup on a ScaldingService. - */ - def merge(delta: PipeFactory[(K, V)], - sg: Semigroup[V], - commutativity: Commutativity, - reducers: Int): PipeFactory[(K, (Option[V], V))] - - /** This is an optional method, by default it a pass-through. - * it may be called by ScaldingPlatform before a key transformation - * that leads only to this store. - */ - def partialMerge[K1](delta: PipeFactory[(K1, V)], - sg: Semigroup[V], - commutativity: Commutativity): PipeFactory[(K1, V)] = delta -} - -trait BatchedScaldingStore[K, V] extends ScaldingStore[K, V] { self => +trait BatchedStore[K, V] extends scalding.Store[K, V] { self => /** The batcher for this store */ def batcher: Batcher @@ -78,12 +45,26 @@ trait BatchedScaldingStore[K, V] extends ScaldingStore[K, V] { self => */ def select(b: List[BatchID]): List[BatchID] = b + + /** + * Override this to set up store pruning, by default, no (key,value) pairs + * are pruned. This is a house keeping function to permanently remove entries + * matching a criteria. + */ + def pruning: PrunedSpace[(K, V)] = PrunedSpace.neverPruned + + /** + * Override this to control when keys are frozen. This allows + * us to avoid sorting and shuffling keys that are not updated. + */ + def boundedKeySpace: TimeBoundedKeySpace[K] = TimeBoundedKeySpace.neverFrozen + /** * For (firstNonZero - 1) we read empty. For all before we error on read. For all later, we proxy * On write, we throw if batchID is less than firstNonZero */ - def withInitialBatch(firstNonZero: BatchID): BatchedScaldingStore[K, V] = - new InitialBatchedStore(firstNonZero, self) + def withInitialBatch(firstNonZero: BatchID): BatchedStore[K, V] = + new scalding.store.InitialBatchedStore(firstNonZero, self) /** Get the most recent last batch and the ID (strictly less than the input ID) * The "Last" is the stream with only the newest value for each key, within the batch @@ -95,7 +76,7 @@ trait BatchedScaldingStore[K, V] extends ScaldingStore[K, V] { self => /** Record a computed batch of code */ def writeLast(batchID: BatchID, lastVals: TypedPipe[(K, V)])(implicit flowDef: FlowDef, mode: Mode): Unit - @transient private val logger = LoggerFactory.getLogger(classOf[BatchedScaldingStore[_,_]]) + @transient private val logger = LoggerFactory.getLogger(classOf[BatchedStore[_,_]]) /** The writeLast method as a FlowProducer */ private def writeFlow(batches: List[BatchID], lastVals: TypedPipe[(BatchID, (K, V))]): FlowProducer[Unit] = { @@ -104,25 +85,26 @@ trait BatchedScaldingStore[K, V] extends ScaldingStore[K, V] { self => // make sure we checkpoint to disk to avoid double computation: val checked = if(batches.size > 1) lastVals.forceToDisk else lastVals batches.foreach { batchID => - val thisBatch = checked.filter { case (b, _) => b == batchID } + val thisBatch = checked.filter { case (b, kv) => + (b == batchID) && !pruning.prune(kv, batcher.latestTimeOf(b)) + } writeLast(batchID, thisBatch.values)(flow, mode) } } } - protected def sumByBatches[K1,V:Semigroup](ins: TypedPipe[(Long, (K1, V))], + protected def sumByBatches[K1,V:Semigroup](ins: TypedPipe[(Timestamp, (K1, V))], capturedBatcher: Batcher, commutativity: Commutativity): TypedPipe[((K1, BatchID), (Timestamp, V))] = { implicit val timeValueSemigroup: Semigroup[(Timestamp, V)] = IteratorSums.optimizedPairSemigroup[Timestamp, V](1000) val inits = ins.map { case (t, (k, v)) => - val ts = Timestamp(t) - val batch = capturedBatcher.batchOf(ts) - ((k, batch), (ts, v)) + val batch = capturedBatcher.batchOf(t) + ((k, batch), (t, v)) } (commutativity match { - case Commutative => ScaldingStore.mapsideReduce(inits) + case Commutative => Store.mapsideReduce(inits) case NonCommutative => inits }) } @@ -140,7 +122,7 @@ trait BatchedScaldingStore[K, V] extends ScaldingStore[K, V] { self => case Commutative => delta.map { flow => flow.map { typedP => sumByBatches(typedP, capturedBatcher, Commutative) - .map { case ((k, _), (ts, v)) => (ts.milliSinceEpoch, (k, v)) } + .map { case ((k, _), (ts, v)) => (ts, (k, v)) } } } case NonCommutative => delta @@ -164,14 +146,14 @@ trait BatchedScaldingStore[K, V] extends ScaldingStore[K, V] { self => import IteratorSums._ // get the groupedSum, partials function - logger.info("Previous written batch: {}, computing: {}", inBatch, batches) + logger.info("Previous written batch: {}, computing: {}", inBatch.asInstanceOf[Any], batches) def prepareOld(old: TypedPipe[(K, V)]): TypedPipe[(K, (BatchID, (Timestamp, V)))] = old.map { case (k, v) => (k, (inBatch, (Timestamp.Min, v))) } val capturedBatcher = batcher //avoid a closure on the whole store - def prepareDeltas(ins: TypedPipe[(Long, (K, V))]): TypedPipe[(K, (BatchID, (Timestamp, V)))] = + def prepareDeltas(ins: TypedPipe[(Timestamp, (K, V))]): TypedPipe[(K, (BatchID, (Timestamp, V)))] = sumByBatches(ins, capturedBatcher, commutativity) .map { case ((k, batch), (ts, v)) => (k, (batch, (ts, v))) } @@ -218,25 +200,46 @@ trait BatchedScaldingStore[K, V] extends ScaldingStore[K, V] { self => // This builds the format we send to consumer nodes def toOutputFormat(res: TypedPipe[(K, (BatchID, (Option[Option[(Timestamp, V)]], Option[(Timestamp, V)])))]): - TypedPipe[(Long, (K, (Option[V], V)))] = + TypedPipe[(Timestamp, (K, (Option[V], V)))] = res.flatMap { case (k, (batchid, (optopt, opt))) => opt.map { case (ts, v) => val prev = flatOpt(optopt).map(_._2) - (ts.milliSinceEpoch, (k, (prev, v))) + (ts, (k, (prev, v))) } } + // Avoid closures where easy + val thisTimeInterval = capturedBatcher.toTimestamp(batchIntr) + val capturedKeyCheck = boundedKeySpace + + def getFrozenKeys(p: TypedPipe[(K,V)]): TypedPipe[(BatchID, (K, V))] = + p.filter { case (k, _) => capturedKeyCheck.isFrozen(k, thisTimeInterval) } + .flatMap { kv => filteredBatches.map { (_, kv) } } + + def getLiquidKeys(p: TypedPipe[(K,V)]): TypedPipe[(K, V)] = + p.filter { case (k, _) => !capturedKeyCheck.isFrozen(k, thisTimeInterval) } + + def assertDeltasAreLiquid(p: TypedPipe[(Timestamp, (K, V))]): TypedPipe[(Timestamp, (K, V))] = + p.map { tkv => + assert(!capturedKeyCheck.isFrozen(tkv._2._1, thisTimeInterval), "Frozen key in deltas: " + tkv) + tkv + } + // Now in the flow-producer monad; do it: for { pipeInput <- input - pipeDeltas <- deltas + frozen = getFrozenKeys(pipeInput) + liquid = getLiquidKeys(pipeInput) + ds <- deltas + liquidDeltas = assertDeltasAreLiquid(ds) // fork below so scalding can make sure not to do the operation twice - merged = mergeAll(prepareOld(pipeInput) ++ prepareDeltas(pipeDeltas)).fork - lastOut = toLastFormat(merged) + merged = mergeAll(prepareOld(liquid) ++ prepareDeltas(liquidDeltas)).fork + lastOut = toLastFormat(merged) ++ frozen _ <- writeFlow(filteredBatches, lastOut) } yield toOutputFormat(merged) } + /** instances of this trait MAY NOT change the logic here. This always follows the rule * that we look for existing data (avoiding reading deltas in that case), then we fall * back to the last checkpointed output by calling readLast. In that case, we compute the @@ -283,27 +286,3 @@ trait BatchedScaldingStore[K, V] extends ScaldingStore[K, V] { self => } }) } - - -/** - * For (firstNonZero - 1) we read empty. For all before we error on read. For all later, we proxy - * On write, we throw if batchID is less than firstNonZero - */ -class InitialBatchedStore[K,V](val firstNonZero: BatchID, val proxy: BatchedScaldingStore[K, V]) - extends BatchedScaldingStore[K, V] { - - def batcher = proxy.batcher - def ordering = proxy.ordering - // This one is dangerous and marked override because it has a default - override def select(b: List[BatchID]) = proxy.select(b) - def writeLast(batchID: BatchID, lastVals: TypedPipe[(K, V)])(implicit flowDef: FlowDef, mode: Mode) = - if (batchID >= firstNonZero) proxy.writeLast(batchID, lastVals) - else sys.error("Earliest batch set at :" + firstNonZero + " but tried to write: " + batchID) - - // Here is where we switch: - def readLast(exclusiveUB: BatchID, mode: Mode): Try[(BatchID, FlowProducer[TypedPipe[(K, V)]])] = { - if (exclusiveUB > firstNonZero) proxy.readLast(exclusiveUB, mode) - else if (exclusiveUB == firstNonZero) Right((firstNonZero.prev, Scalding.emptyFlowProducer[(K,V)])) - else Left(List("Earliest batch set at :" + firstNonZero + " but tried to read: " + exclusiveUB)) - } -} diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/package.scala b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/package.scala index ae8ff0070..ec35f1c79 100644 --- a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/package.scala +++ b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/package.scala @@ -23,17 +23,18 @@ import cascading.flow.FlowDef import com.twitter.algebird.monad.{Reader, StateWithError} import com.twitter.algebird.Interval +import com.twitter.summingbird +import com.twitter.summingbird.batch.Timestamp +import org.apache.hadoop.io.Writable package object scalding { - /** We represent time as Long Millis */ - type Time = Long /** How we represent the streams in scalding */ - type TimedPipe[+T] = TypedPipe[(Time, T)] + type TimedPipe[+T] = TypedPipe[(Timestamp, T)] type KeyValuePipe[+K, +V] = TimedPipe[(K, V)] /** The Platform recursively passes this input around to describe a * step forward: requested input time span, and scalding Mode */ - type FactoryInput = (Interval[Time], Mode) + type FactoryInput = (Interval[Timestamp], Mode) /** When it is time to run build the final flow, * this is what scalding needs. It is modified in the Reader[FlowInput, T] */ @@ -58,6 +59,7 @@ package object scalding { // Helps interop with scalding: implicit def modeFromTuple(implicit fm: (FlowDef, Mode)): Mode = fm._2 implicit def flowDefFromTuple(implicit fm: (FlowDef, Mode)): FlowDef = fm._1 + implicit def toPipeFactoryOps[T](pipeF: PipeFactory[T]) = new PipeFactoryOps(pipeF) def toTry(e: Throwable): Try[Nothing] = { val writer = new java.io.StringWriter @@ -66,4 +68,7 @@ package object scalding { printWriter.flush Left(List(writer.toString)) } + + val ScaldingConfig = summingbird.batch.BatchConfig } + diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/BatchedDeltaService.scala b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/service/BatchedDeltaService.scala similarity index 76% rename from summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/BatchedDeltaService.scala rename to summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/service/BatchedDeltaService.scala index 9693a5cb9..1388d17b7 100644 --- a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/BatchedDeltaService.scala +++ b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/service/BatchedDeltaService.scala @@ -14,21 +14,22 @@ limitations under the License. */ -package com.twitter.summingbird.scalding +package com.twitter.summingbird.scalding.service import com.twitter.algebird.monad.{StateWithError, Reader} import com.twitter.algebird.Semigroup import com.twitter.scalding.{Mode, TypedPipe} -import com.twitter.summingbird.batch.{ BatchID, Batcher } +import com.twitter.summingbird.batch.{ BatchID, Batcher, Timestamp } import cascading.flow.FlowDef +import com.twitter.summingbird.scalding._ /** * Use this when you have written JUST BEFORE the store. * This is what you get from an IntermediateWrite in Builder API. */ -class BatchedDeltaService[K, V:Semigroup](val store: BatchedScaldingStore[K, V], - val deltas: BatchedScaldingSink[(K, V)], - override val reducers: Option[Int] = None) extends BatchedService[K, V] { +class BatchedDeltaService[K, V:Semigroup](val store: batch.BatchedStore[K, V], + val deltas: batch.BatchedSink[(K, V)], + override val reducers: Option[Int] = None) extends batch.BatchedService[K, V] { assert(store.batcher == deltas.batcher, "Batchers do not match") @@ -47,8 +48,8 @@ class BatchedDeltaService[K, V:Semigroup](val store: BatchedScaldingStore[K, V], * You are guaranteed that all the service data needed * to do the join is present. */ - override def lookup[W](incoming: TypedPipe[(Time, (K, W))], - servStream: TypedPipe[(Time, (K, Option[V]))]): TypedPipe[(Time, (K, (W, Option[V])))] = { + override def lookup[W](incoming: TypedPipe[(Timestamp, (K, W))], + servStream: TypedPipe[(Timestamp, (K, Option[V]))]): TypedPipe[(Timestamp, (K, (W, Option[V])))] = { def flatOpt[T](o: Option[Option[T]]): Option[T] = o.flatMap(identity) diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/BatchedWindowService.scala b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/service/BatchedWindowService.scala similarity index 65% rename from summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/BatchedWindowService.scala rename to summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/service/BatchedWindowService.scala index 0ec68737c..8a45cb245 100644 --- a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/BatchedWindowService.scala +++ b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/service/BatchedWindowService.scala @@ -14,9 +14,10 @@ limitations under the License. */ -package com.twitter.summingbird.scalding +package com.twitter.summingbird.scalding.service -import com.twitter.summingbird.batch.{ BatchID, Batcher, Timestamp } +import com.twitter.summingbird.batch.{BatchID, Batcher, Timestamp, Milliseconds} +import com.twitter.summingbird.scalding._ import com.twitter.scalding.{Mode, TypedPipe, AbsoluteDuration} import com.twitter.algebird.monad.Reader import cascading.flow.FlowDef @@ -32,22 +33,22 @@ import cascading.flow.FlowDef * ordering * reducers */ -trait BatchedWindowService[K, V] extends BatchedService[K, V] { +trait BatchedWindowService[K, V] extends batch.BatchedService[K, V] { /** * A request must come in LESS than this window since the last * key written to the service */ - def windowSize: Time + def windowSize: Milliseconds /** * The batched window never reads an aggregated last. Instead we just output * an empty pipe that is outside the window. */ def readLast(exclusiveUB: BatchID, mode: Mode): Try[(BatchID, FlowProducer[TypedPipe[(K, V)]])] = { - val earliestInput = batcher.earliestTimeOf(exclusiveUB).milliSinceEpoch + val earliestInput = batcher.earliestTimeOf(exclusiveUB) val earliestNeededKey = earliestInput - windowSize // We may need values from this batch: - val earliestNeededBatch = batcher.batchOf(Timestamp(earliestNeededKey)) + val earliestNeededBatch = batcher.batchOf(earliestNeededKey) // But all of these values are definitly too old: val firstZeroBatch = earliestNeededBatch.prev Right((firstZeroBatch, Scalding.emptyFlowProducer)) @@ -57,30 +58,17 @@ trait BatchedWindowService[K, V] extends BatchedService[K, V] { * You are guaranteed that all the service data needed * to do the join is present */ - override def lookup[W](incoming: TypedPipe[(Time, (K, W))], - servStream: TypedPipe[(Time, (K, Option[V]))]): TypedPipe[(Time, (K, (W, Option[V])))] = { + override def lookup[W](incoming: TypedPipe[(Timestamp, (K, W))], + servStream: TypedPipe[(Timestamp, (K, Option[V]))]): TypedPipe[(Timestamp, (K, (W, Option[V])))] = { def flatOpt[T](o: Option[Option[T]]): Option[T] = o.flatMap(identity) implicit val ord = ordering val win = windowSize // call this once so scala makes a smarter closure - LookupJoin.withWindow(incoming, servStream, reducers) { (l: Time, r: Time) => (l-r) < win } + LookupJoin.withWindow(incoming, servStream, reducers) { (l: Timestamp, r: Timestamp) => (l-r) < win } .map { case (t, (k, (w, optoptv))) => (t, (k, (w, flatOpt(optoptv)))) } } } -/** More familiar interface to scalding users that creates - * the Reader from two other methods - */ -trait SimpleWindowedService[K, V] extends BatchedWindowService[K, V] { - def streamIsAvailable(b: BatchID, m: Mode): Boolean - def read(b: BatchID)(implicit f: FlowDef, m: Mode): TypedPipe[(Time, (K, Option[V]))] - final def readStream(batchID: BatchID, mode: Mode): Option[FlowToPipe[(K, Option[V])]] = { - if(!streamIsAvailable(batchID, mode)) { - None - } - else Some(Reader({ implicit fdm: (FlowDef, Mode) => read(batchID) })) - } -} diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/service/EmptyService.scala b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/service/EmptyService.scala new file mode 100644 index 000000000..480eb3cd9 --- /dev/null +++ b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/service/EmptyService.scala @@ -0,0 +1,25 @@ +/* + Copyright 2013 Twitter, Inc. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + */ + +package com.twitter.summingbird.scalding.service + +import com.twitter.summingbird.scalding._ + +class EmptyService[K, V] extends Service[K, V] { + def lookup[W](getKeys: PipeFactory[(K, W)]): PipeFactory[(K, (W, Option[V]))] = + getKeys.map { _.map { _.map { case (t, (k, v)) => (t, (k, (v, None: Option[V]))) } } } +} + diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/SimpleService.scala b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/service/SimpleService.scala similarity index 84% rename from summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/SimpleService.scala rename to summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/service/SimpleService.scala index e7371fdd5..9968ae601 100644 --- a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/SimpleService.scala +++ b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/service/SimpleService.scala @@ -14,12 +14,13 @@ limitations under the License. */ -package com.twitter.summingbird.scalding +package com.twitter.summingbird.scalding.service import com.twitter.algebird.monad.{Reader, StateWithError} import com.twitter.algebird.Interval import com.twitter.bijection.Conversion.asMethod - +import com.twitter.summingbird.scalding._ +import com.twitter.summingbird.batch.Timestamp import com.twitter.scalding.{Source => SSource, _} import cascading.flow.FlowDef @@ -29,7 +30,7 @@ import cascading.flow.FlowDef * content IDs, cryptographic hashes, etc... */ -trait SimpleService[K, V] extends ScaldingService[K, V] { +trait SimpleService[K, V] extends Service[K, V] { import Scalding.dateRangeInjection @@ -37,7 +38,7 @@ trait SimpleService[K, V] extends ScaldingService[K, V] { def satisfiable(requested: DateRange, mode: Mode): Try[DateRange] def serve[W](covering: DateRange, - input: TypedPipe[(Long, (K, W))])(implicit flowDef: FlowDef, mode: Mode): TypedPipe[(Long, (K, (W, Option[V])))] + input: TypedPipe[(Timestamp, (K, W))])(implicit flowDef: FlowDef, mode: Mode): TypedPipe[(Timestamp, (K, (W, Option[V])))] final def lookup[W](getKeys: PipeFactory[(K, W)]): PipeFactory[(K, (W, Option[V]))] = StateWithError({ intMode: FactoryInput => @@ -45,7 +46,7 @@ trait SimpleService[K, V] extends ScaldingService[K, V] { Scalding.toDateRange(timeSpan).right .flatMap(satisfiable(_, mode)).right .flatMap { dr => - val ts = dr.as[Interval[Time]] + val ts = dr.as[Interval[Timestamp]] getKeys((ts, mode)).right .map { case ((avail, m), getFlow) => val rdr = Reader({ implicit fdM: (FlowDef, Mode) => diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/service/SimpleWindowService.scala b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/service/SimpleWindowService.scala new file mode 100644 index 000000000..a3da489e7 --- /dev/null +++ b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/service/SimpleWindowService.scala @@ -0,0 +1,38 @@ +/* + Copyright 2013 Twitter, Inc. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + */ + +package com.twitter.summingbird.scalding.service + +import com.twitter.summingbird.batch.{ BatchID, Batcher, Timestamp } +import com.twitter.summingbird.scalding._ +import com.twitter.scalding.{Mode, TypedPipe, AbsoluteDuration} +import com.twitter.algebird.monad.Reader +import cascading.flow.FlowDef + +/** More familiar interface to scalding users that creates + * the Reader from two other methods + */ +trait SimpleWindowedService[K, V] extends BatchedWindowService[K, V] { + def streamIsAvailable(b: BatchID, m: Mode): Boolean + def read(b: BatchID)(implicit f: FlowDef, m: Mode): TypedPipe[(Timestamp, (K, Option[V]))] + + final def readStream(batchID: BatchID, mode: Mode): Option[FlowToPipe[(K, Option[V])]] = { + if(!streamIsAvailable(batchID, mode)) { + None + } + else Some(Reader({ implicit fdm: (FlowDef, Mode) => read(batchID) })) + } +} \ No newline at end of file diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/UniqueKeyedService.scala b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/service/UniqueKeyedService.scala similarity index 87% rename from summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/UniqueKeyedService.scala rename to summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/service/UniqueKeyedService.scala index 82887b663..8fbda5065 100644 --- a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/UniqueKeyedService.scala +++ b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/service/UniqueKeyedService.scala @@ -14,14 +14,15 @@ limitations under the License. */ -package com.twitter.summingbird.scalding +package com.twitter.summingbird.scalding.service import com.twitter.algebird.monad.{Reader, StateWithError} import com.twitter.algebird.Interval import com.twitter.bijection.Conversion.asMethod - -import com.twitter.scalding.{Source => SSource, _ } -import com.twitter.scalding.typed.{ TypedPipe => _, _ } +import com.twitter.summingbird.scalding._ +import com.twitter.summingbird.batch.Timestamp +import com.twitter.scalding.{Grouped => _, Source => SSource, _ } +import com.twitter.scalding.typed.{TypedPipe => _, _ } import cascading.flow.FlowDef /** A UniqueKeyedService covers the case where Keys are globally @@ -37,8 +38,8 @@ trait UniqueKeyedService[K, V] extends SimpleService[K, V] { def reducers: Option[Int] /** You can override this to use hashJoin for instance */ - def doJoin[W](in: TypedPipe[(Time, (K, W))], - serv: TypedPipe[(K, V)])(implicit flowDef: FlowDef, mode: Mode): TypedPipe[(Time, (K, (W, Option[V])))] = { + def doJoin[W](in: TypedPipe[(Timestamp, (K, W))], + serv: TypedPipe[(K, V)])(implicit flowDef: FlowDef, mode: Mode): TypedPipe[(Timestamp, (K, (W, Option[V])))] = { implicit val ord: Ordering[K] = ordering def withReducers[U, T](grouped: Grouped[U, T]) = reducers.map { grouped.withReducers(_) }.getOrElse(grouped) @@ -50,7 +51,7 @@ trait UniqueKeyedService[K, V] extends SimpleService[K, V] { } final override def serve[W](covering: DateRange, - input: TypedPipe[(Long, (K, W))])(implicit flowDef: FlowDef, mode: Mode) = + input: TypedPipe[(Timestamp, (K, W))])(implicit flowDef: FlowDef, mode: Mode) = doJoin(input, readDateRange(covering)) } diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/TimePathedSource.scala b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/source/TimePathedSource.scala similarity index 98% rename from summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/TimePathedSource.scala rename to summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/source/TimePathedSource.scala index 16ae79304..b9b65ea55 100644 --- a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/TimePathedSource.scala +++ b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/source/TimePathedSource.scala @@ -14,7 +14,7 @@ limitations under the License. */ -package com.twitter.summingbird.scalding +package com.twitter.summingbird.scalding.source import com.twitter.scalding.{TimePathedSource => STPS, _} import org.apache.hadoop.fs.Path diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/DirectoryBatchedStore.scala b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/store/DirectoryBatchedStore.scala similarity index 84% rename from summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/DirectoryBatchedStore.scala rename to summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/store/DirectoryBatchedStore.scala index 716723072..db3b54fe3 100644 --- a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/DirectoryBatchedStore.scala +++ b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/store/DirectoryBatchedStore.scala @@ -14,7 +14,7 @@ limitations under the License. */ -package com.twitter.summingbird.scalding +package com.twitter.summingbird.scalding.store import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} @@ -27,9 +27,10 @@ import com.twitter.algebird.monad.{StateWithError, Reader} import com.twitter.bijection.{ Bijection, ImplicitBijection } import com.twitter.scalding.{Dsl, Mode, Hdfs, TypedPipe, IterableSource, WritableSequenceFile, MapsideReduce, TupleSetter, TupleConverter} import com.twitter.scalding.typed. TypedSink -import com.twitter.summingbird._ + import com.twitter.summingbird.option._ -import com.twitter.summingbird.batch.{ BatchID, Batcher} +import com.twitter.summingbird.batch.{ BatchID, Batcher, Timestamp} +import com.twitter.summingbird.scalding._ /** * DirectoryBatched Scalding Store, which only contains (K, V) data pairs in the data. @@ -40,27 +41,27 @@ import com.twitter.summingbird.batch.{ BatchID, Batcher} class DirectoryBatchedStore[K <: Writable, V <: Writable](val rootPath: String) (implicit inBatcher: Batcher, ord: Ordering[K], tset: TupleSetter[(K, V)], tconv: TupleConverter[(K, V)]) - extends BatchedScaldingStore[K, V] { - import Dsl._ + extends batch.BatchedStore[K, V] { + import Dsl._ val batcher = inBatcher val ordering = ord protected def getFileStatus(p: String, conf: Configuration) = { val path = new Path(p) - val (isGood, lastModifyTime): (Boolean, Long) = + val (isGood, lastModifyTime): (Boolean, Timestamp) = Option(path.getFileSystem(conf).globStatus(path)) .map { statuses: Array[FileStatus] => // Must have a file that is called "_SUCCESS" val isGood = statuses.exists { fs: FileStatus => fs.getPath.getName == "_SUCCESS" } - val lastModifyTime = statuses.map{_.getModificationTime}.max - (isGood, lastModifyTime) + val lastModifyTime = Timestamp(statuses.map{_.getModificationTime}.max) + (isGood, lastModifyTime) } - .getOrElse((false, 0)) - - (isGood, lastModifyTime, path.getName) + .getOrElse((false, Timestamp(0))) + + (isGood, lastModifyTime, path.getName) } /* @@ -79,20 +80,20 @@ class DirectoryBatchedStore[K <: Writable, V <: Writable](val rootPath: String) .toList } - val lastBatchStatus = + val lastBatchStatus = hdfsPaths.map(getFileStatus(_, conf)) - .filter{input => input._1 && BatchID(input._2) < exclusiveUB} + .filter{input => input._1 && (BatchID(input._2) < exclusiveUB)} .reduceOption{(a, b) => if (a._2 > b._2) a else b} .getOrElse((false, 0, "0")) - - if (lastBatchStatus._1) BatchID(lastBatchStatus._3) + + if (lastBatchStatus._1) BatchID(lastBatchStatus._3) else throw new Exception( "No good data <= " + exclusiveUB + " is available at : " + rootPath) } case _ => { throw new Exception( "DirectoryBatchedStore must work in Hdfs. Mode: " + mode.toString + " found.") - } + } } } @@ -100,13 +101,13 @@ class DirectoryBatchedStore[K <: Writable, V <: Writable](val rootPath: String) val outSource = WritableSequenceFile(rootPath + "/" + batchID.toString, 'key -> 'val) lastVals.write(TypedSink[(K, V)](outSource)) } - + override def readLast(exclusiveUB: BatchID, mode: Mode) = { val lastID = getLastBatchID(exclusiveUB, mode) - + val src = WritableSequenceFile(rootPath + "/" + lastID.toString, 'key -> 'val) val rdr = Reader { (fd: (FlowDef, Mode)) => TypedPipe.from(src.read(fd._1, fd._2), Fields.ALL)} Right((lastID, rdr)) - + } } diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/store/InitialBatchedStore.scala b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/store/InitialBatchedStore.scala new file mode 100644 index 000000000..add94a89e --- /dev/null +++ b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/store/InitialBatchedStore.scala @@ -0,0 +1,47 @@ +/* + Copyright 2013 Twitter, Inc. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + */ + +package com.twitter.summingbird.scalding.store + +import com.twitter.scalding.{Mode, TypedPipe} +import com.twitter.summingbird._ +import com.twitter.summingbird.scalding.batch.BatchedStore +import com.twitter.summingbird.scalding.{Try, FlowProducer, Scalding} +import com.twitter.summingbird.batch.BatchID +import cascading.flow.FlowDef +import com.twitter.summingbird.scalding._ + +/** + * For (firstNonZero - 1) we read empty. For all before we error on read. For all later, we proxy + * On write, we throw if batchID is less than firstNonZero + */ +class InitialBatchedStore[K,V](val firstNonZero: BatchID, override val proxy: BatchedStore[K, V]) + extends ProxyBatchedStore[K, V] { + + override def writeLast(batchID: BatchID, lastVals: TypedPipe[(K, V)])(implicit flowDef: FlowDef, mode: Mode) = + if (batchID >= firstNonZero) proxy.writeLast(batchID, lastVals) + else sys.error("Earliest batch set at :" + firstNonZero + " but tried to write: " + batchID) + + // Here is where we switch: + override def readLast(exclusiveUB: BatchID, mode: Mode): Try[(BatchID, FlowProducer[TypedPipe[(K, V)]])] = { + if (exclusiveUB > firstNonZero) proxy.readLast(exclusiveUB, mode) + else if (exclusiveUB == firstNonZero) Right((firstNonZero.prev, Scalding.emptyFlowProducer[(K,V)])) + else Left(List("Earliest batch set at :" + firstNonZero + " but tried to read: " + exclusiveUB)) + } + + override def toString = + "InitialBatchedStore(firstNonZero=%s, proxyingFor=%s)".format(firstNonZero.toString, proxy.toString) +} diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/store/ProxyBatchedStore.scala b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/store/ProxyBatchedStore.scala new file mode 100644 index 000000000..11879f74a --- /dev/null +++ b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/store/ProxyBatchedStore.scala @@ -0,0 +1,38 @@ +/* +Copyright 2013 Twitter, Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + +http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package com.twitter.summingbird.scalding.store + +import cascading.flow.FlowDef +import com.twitter.summingbird.batch.BatchID +import com.twitter.summingbird.scalding._ +import com.twitter.scalding.{Mode, TypedPipe} + +/** Use this class to easily change, for instance, the pruning + * for an existing store. + */ + abstract class ProxyBatchedStore[K, V] extends batch.BatchedStore[K, V] { + def proxy: batch.BatchedStore[K, V] + override def batcher = proxy.batcher + override def ordering = proxy.ordering + override def select(b: List[BatchID]) = proxy.select(b) + override def pruning = proxy.pruning + override def readLast(exclusiveUB: BatchID, mode: Mode) = proxy.readLast(exclusiveUB, mode) + override def writeLast(batchID: BatchID, lastVals: TypedPipe[(K, V)])(implicit flowDef: FlowDef, mode: Mode): Unit = + proxy.writeLast(batchID, lastVals)(flowDef, mode) + + override def toString = "ProxyBatchedStore(proxyingFor=%s)".format(proxy.toString) + } diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/VersionedBatchStore.scala b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/store/VersionedBatchStore.scala similarity index 96% rename from summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/VersionedBatchStore.scala rename to summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/store/VersionedBatchStore.scala index 6258b9774..4d94f7421 100644 --- a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/VersionedBatchStore.scala +++ b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/store/VersionedBatchStore.scala @@ -14,15 +14,18 @@ See the License for the specific language governing permissions and limitations under the License. */ -package com.twitter.summingbird.scalding +package com.twitter.summingbird.scalding.store -import com.twitter.summingbird.scalding.store.HDFSMetadata +import com.twitter.summingbird.batch.store.HDFSMetadata import cascading.flow.FlowDef import com.twitter.bijection.Injection import cascading.flow.FlowDef import com.twitter.scalding.{Dsl, Mode, TDsl, TypedPipe, Hdfs => HdfsMode, TupleSetter} import com.twitter.scalding.commons.source.VersionedKeyValSource +import com.twitter.summingbird.scalding.batch.BatchedStore +import com.twitter.summingbird.scalding.{Try, FlowProducer, Scalding} import com.twitter.algebird.monad.Reader +import com.twitter.summingbird.scalding._ import com.twitter.summingbird.batch.{BatchID, Batcher, Timestamp } import scala.util.{ Try => ScalaTry } @@ -50,7 +53,7 @@ object VersionedBatchStore { * Allows subclasses to share the means of reading version numbers but * plug in methods to actually read or write the data. */ -abstract class VersionedBatchStoreBase[K, V](val rootPath: String) extends BatchedScaldingStore[K, V] { +abstract class VersionedBatchStoreBase[K, V](val rootPath: String) extends BatchedStore[K, V] { /** * Returns a snapshot of the store's (K, V) pairs aggregated up to * (but not including!) the time covered by the supplied batchID. diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/store/VersionedStore.scala b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/store/VersionedStore.scala index a0277552b..0b7849a9b 100644 --- a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/store/VersionedStore.scala +++ b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/store/VersionedStore.scala @@ -20,9 +20,9 @@ import cascading.flow.FlowDef import com.twitter.bijection.Injection import com.twitter.scalding.{Dsl, Mode, TDsl, TypedPipe} import com.twitter.scalding.commons.source.VersionedKeyValSource -import com.twitter.summingbird.batch.{ Batcher, BatchID } -import com.twitter.summingbird.scalding.VersionedBatchStore +import com.twitter.summingbird.batch.{ Batcher, BatchID, PrunedSpace, Timestamp} import scala.util.control.Exception.allCatch +import com.twitter.summingbird.scalding._ /** * Scalding implementation of the batch read and write components @@ -49,13 +49,17 @@ object VersionedStore { * See summingbird-client's ClientStore for more information on the * merge between offline and online data. */ - def apply[K, V](rootPath: String, versionsToKeep: Int = VersionedKeyValSource.defaultVersionsToKeep) - (implicit injection: Injection[(K, (BatchID, V)), (Array[Byte], Array[Byte])], + def apply[K, V]( + rootPath: String, + versionsToKeep: Int = VersionedKeyValSource.defaultVersionsToKeep, + prunedSpace: PrunedSpace[(K,V)] = PrunedSpace.neverPruned)( + implicit injection: Injection[(K, (BatchID, V)), (Array[Byte], Array[Byte])], batcher: Batcher, ord: Ordering[K]): VersionedBatchStore[K, V, K, (BatchID, V)] = new VersionedBatchStore[K, V, K, (BatchID, V)]( rootPath, versionsToKeep, batcher )({ case (batchID, (k, v)) => (k, (batchID.next, v)) })({ case (k, (_, v)) => (k, v) }) { override def select(b: List[BatchID]) = List(b.last) + override def pruning = prunedSpace } } diff --git a/summingbird-scalding/src/test/scala/com/twitter/summingbird/scalding/ScaldingLaws.scala b/summingbird-scalding/src/test/scala/com/twitter/summingbird/scalding/ScaldingLaws.scala deleted file mode 100644 index bb71c1abb..000000000 --- a/summingbird-scalding/src/test/scala/com/twitter/summingbird/scalding/ScaldingLaws.scala +++ /dev/null @@ -1,561 +0,0 @@ -/* - Copyright 2013 Twitter, Inc. - - Licensed under the Apache License, Version 2.0 (the "License"); - you may not use this file except in compliance with the License. - You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. - */ - -package com.twitter.summingbird.scalding - -import com.twitter.algebird.{MapAlgebra, Monoid, Group, Interval, Last} -import com.twitter.algebird.monad._ -import com.twitter.summingbird._ -import com.twitter.summingbird.batch._ -import com.twitter.summingbird.scalding.state.HDFSState - -import java.util.TimeZone -import java.io.File - -import com.twitter.scalding.{ Source => ScaldingSource, Test => TestMode, _ } -import com.twitter.scalding.typed.TypedSink - -import org.scalacheck._ -import org.scalacheck.Prop._ -import org.scalacheck.Properties - -import org.apache.hadoop.conf.Configuration - -import scala.collection.JavaConverters._ -import scala.collection.mutable.{ArrayBuffer, Buffer, HashMap => MutableHashMap, Map => MutableMap, SynchronizedBuffer, SynchronizedMap} - -import cascading.scheme.local.{TextDelimited => CLTextDelimited} -import cascading.tuple.{Tuple, Fields, TupleEntry} -import cascading.flow.FlowDef -import cascading.tap.Tap -import cascading.scheme.NullScheme -import org.apache.hadoop.mapred.JobConf -import org.apache.hadoop.mapred.RecordReader -import org.apache.hadoop.mapred.OutputCollector - - -import org.specs2.mutable._ - -/** - * Tests for Summingbird's Scalding planner. - */ - -class MockMappable[T](val id: String)(implicit tconv: TupleConverter[T]) - extends ScaldingSource with Mappable[T] { - def converter[U >: T] = TupleConverter.asSuperConverter(tconv) - override def toString = id - override def equals(that: Any) = that match { - case m: MockMappable[_] => m.id == id - case _ => false - } - override def hashCode = id.hashCode - - override def createTap(readOrWrite : AccessMode)(implicit mode : Mode) : Tap[_,_,_] = - TestTapFactory(this, new NullScheme[JobConf, RecordReader[_,_], OutputCollector[_,_], T, T](Fields.ALL, Fields.ALL)).createTap(readOrWrite) -} - -object TestStore { - def apply[K, V](store: String, inBatcher: Batcher, initStore: Iterable[(K, V)], lastTime: Long) - (implicit ord: Ordering[K], tset: TupleSetter[(K, V)], tconv: TupleConverter[(K, V)]) = { - val startBatch = inBatcher.batchOf(Timestamp(0)).prev - val endBatch = inBatcher.batchOf(Timestamp(lastTime)).next - new TestStore[K, V](store, inBatcher, startBatch, initStore, endBatch) - } -} - -class TestStore[K, V](store: String, inBatcher: Batcher, initBatch: BatchID, initStore: Iterable[(K, V)], lastBatch: BatchID) -(implicit ord: Ordering[K], tset: TupleSetter[(K, V)], tconv: TupleConverter[(K, V)]) - extends BatchedScaldingStore[K, V] { - - var writtenBatches = Set[BatchID](initBatch) - val batches: Map[BatchID, Mappable[(K, V)]] = - BatchID.range(initBatch, lastBatch).map { b => (b, mockFor(b)) }.toMap - - // Needed to init the Test mode: - val sourceToBuffer: Map[ScaldingSource, Buffer[Tuple]] = - BatchID.range(initBatch, lastBatch).map { b => - if (initBatch == b) (batches(b), initStore.map { tset(_) }.toBuffer) - else (batches(b), Buffer.empty[Tuple]) - }.toMap - - // Call this after you compute to check the results of the - def lastToIterable: Iterable[(K, V)] = - sourceToBuffer(batches(writtenBatches.max)).toIterable.map { tup => tconv(new TupleEntry(tup)) } - - val batcher = inBatcher - val ordering = ord - - def mockFor(b: BatchID): Mappable[(K, V)] = - new MockMappable(store + b.toString) - - override def readLast(exclusiveUB: BatchID, mode: Mode) = { - val candidates = writtenBatches.filter { _ < exclusiveUB } - if(candidates.isEmpty) { - Left(List("No batches < :" + exclusiveUB.toString)) - } - else { - val batch = candidates.max - val mappable = batches(batch) - val rdr = Reader { (fd: (FlowDef, Mode)) => TypedPipe.from(mappable)(fd._1, fd._2) } - Right((batch, rdr)) - } - } - /** Instances may choose to write out the last or just compute it from the stream */ - override def writeLast(batchID: BatchID, lastVals: TypedPipe[(K, V)])(implicit flowDef: FlowDef, mode: Mode): Unit = { - val out = batches(batchID) - lastVals.write(TypedSink[(K, V)](out)) - writtenBatches = writtenBatches + batchID - } -} - -class TestService[K, V](service: String, - inBatcher: Batcher, - minBatch: BatchID, - streams: Map[BatchID, Iterable[(Time, (K, Option[V]))]]) -(implicit ord: Ordering[K], - tset: TupleSetter[(Time, (K, Option[V]))], - tset2: TupleSetter[(Time, (K, V))], - tconv: TupleConverter[(Time, (K, Option[V]))], - tconv2: TupleConverter[(Time, (K, V))]) - extends BatchedService[K, V] { - - val batcher = inBatcher - val ordering = ord - val reducers = None - // Needed to init the Test mode: - val sourceToBuffer: Map[ScaldingSource, Buffer[Tuple]] = - (lasts.map { case (b, it) => lastMappable(b) -> toBuffer(it) } ++ - streams.map { case (b, it) => streamMappable(b) -> toBuffer(it) }).toMap - - /** The lasts are computed from the streams */ - lazy val lasts: Map[BatchID, Iterable[(Time, (K, V))]] = { - (streams - .toList - .sortBy(_._1) - .foldLeft(Map.empty[BatchID, Map[K, (Time, V)]]) { - case (map, (batch: BatchID, writes: Iterable[(Time, (K, Option[V]))])) => - val thisBatch = writes.foldLeft(map.get(batch).getOrElse(Map.empty[K, (Time, V)])) { - case (innerMap, (time, (k, v))) => - v match { - case None => innerMap - k - case Some(v) => innerMap + (k -> (time -> v)) - } - } - map + (batch -> thisBatch) - } - .mapValues { innerMap => - innerMap.toSeq.map { case (k, (time, v)) => (time, (k, v)) } - }) + (minBatch -> Iterable.empty) - } - - def lastMappable(b: BatchID): Mappable[(Time, (K, V))] = - new MockMappable[(Time, (K, V))](service + "/last/" + b.toString) - - def streamMappable(b: BatchID): Mappable[(Time, (K, Option[V]))] = - new MockMappable[(Time, (K, Option[V]))](service + "/stream/" + b.toString) - - def toBuffer[T](it: Iterable[T])(implicit ts: TupleSetter[T]): Buffer[Tuple] = - it.map { ts(_) }.toBuffer - - override def readStream(batchID: BatchID, mode: Mode): Option[FlowToPipe[(K, Option[V])]] = { - streams.get(batchID).map { iter => - val mappable = streamMappable(batchID) - Reader { (fd: (FlowDef, Mode)) => TypedPipe.from(mappable)(fd._1, fd._2) } - } - } - override def readLast(exclusiveUB: BatchID, mode: Mode) = { - val candidates = lasts.filter { _._1 < exclusiveUB } - if(candidates.isEmpty) { - Left(List("No batches < :" + exclusiveUB.toString)) - } - else { - val (batch, _) = candidates.maxBy { _._1 } - val mappable = lastMappable(batch) - val rdr = Reader { (fd: (FlowDef, Mode)) => - TypedPipe.from(mappable)(fd._1, fd._2).values - } - Right((batch, rdr)) - } - } -} - -/** This is a test sink that assumes single threaded testing with - * cascading local mode - */ -class TestSink[T] extends ScaldingSink[T] { - private var data: Vector[(Long, T)] = Vector.empty - - def write(incoming: PipeFactory[T]): PipeFactory[T] = - // three functors deep: - incoming.map { state => - state.map { reader => - reader.map { timeItem => - data = data :+ timeItem - timeItem - } - } - } - - def reset: Vector[(Long, T)] = { - val oldData = data - data = Vector.empty - oldData - } -} - -// This is not really usable, just a mock that does the same state over and over -class LoopState[T](init: T) extends WaitingState[T] { self => - def begin = new PrepareState[T] { - def requested = self.init - def fail(err: Throwable) = { - println(err) - self - } - def willAccept(intr: T) = Right(new RunningState[T] { - def succeed = self - def fail(err: Throwable) = { - println(err) - self - } - }) - } -} - -object ScaldingLaws extends Specification { - import MapAlgebra.sparseEquiv - - def sample[T: Arbitrary]: T = Arbitrary.arbitrary[T].sample.get - - def testSource[T](iter: Iterable[T]) - (implicit mf: Manifest[T], te: TimeExtractor[T], tc: TupleConverter[T], tset: TupleSetter[T]): - (Map[ScaldingSource, Buffer[Tuple]], Producer[Scalding, T]) = { - val src = IterableSource(iter) - val prod = Scalding.sourceFromMappable { _ => src } - (Map(src -> iter.map { tset(_) }.toBuffer), prod) - } - - implicit def tupleExtractor[T <: (Long, _)]: TimeExtractor[T] = TimeExtractor( _._1 ) - - def compareMaps[K,V:Group](original: Iterable[Any], inMemory: Map[K, V], testStore: TestStore[K, V], name: String = ""): Boolean = { - val produced = testStore.lastToIterable.toMap - val diffMap = Group.minus(inMemory, produced) - val wrong = Monoid.isNonZero(diffMap) - if(wrong) { - if(!name.isEmpty) println("%s is wrong".format(name)) - println("input: " + original) - println("input size: " + original.size) - println("input batches: " + testStore.batcher.batchOf(Timestamp(original.size))) - println("producer extra keys: " + (produced.keySet -- inMemory.keySet)) - println("producer missing keys: " + (inMemory.keySet -- produced.keySet)) - println("written batches: " + testStore.writtenBatches) - println("earliest unwritten time: " + testStore.batcher.earliestTimeOf(testStore.writtenBatches.max.next)) - println("Difference: " + diffMap) - } - !wrong - } - - def batchedCover(batcher: Batcher, minTime: Long, maxTime: Long): Interval[Timestamp] = - batcher.cover( - Interval.leftClosedRightOpen(Timestamp(minTime), Timestamp(maxTime+1L)) - ).mapNonDecreasing(b => batcher.earliestTimeOf(b.next)) - - val simpleBatcher = new Batcher { - def batchOf(d: Timestamp) = - if (d == Timestamp.Max) BatchID(2) - else if (d.milliSinceEpoch >= 0L) BatchID(1) - else BatchID(0) - - def earliestTimeOf(batch: BatchID) = batch.id match { - case 0L => Timestamp.Min - case 1L => Timestamp(0) - case 2L => Timestamp.Max - case 3L => Timestamp.Max - } - // this is just for testing, it covers everything with batch 1 - override def cover(interval: Interval[Timestamp]): Interval[BatchID] = - Interval.leftClosedRightOpen(BatchID(1), BatchID(2)) - } - - def randomBatcher(items: Iterable[(Long, Any)]): Batcher = { - if(items.isEmpty) simpleBatcher - else randomBatcher(items.iterator.map(_._1).min, items.iterator.map(_._1).max) - } - - def randomBatcher(mintimeInc: Long, maxtimeInc: Long): Batcher = { //simpleBatcher - // we can have between 1 and (maxtime - mintime + 1) batches. - val delta = (maxtimeInc - mintimeInc) - val MaxBatches = 5L min delta - val batches = 1L + Gen.choose(0L, MaxBatches).sample.get - if(batches == 1L) simpleBatcher - else { - val timePerBatch = (delta + 1L)/batches - new MillisecondBatcher(timePerBatch) - } - } - - "The ScaldingPlatform" should { - //Set up the job: - "match scala for single step jobs" in { - val original = sample[List[Int]] - val fn = sample[(Int) => List[(Int, Int)]] - val initStore = sample[Map[Int, Int]] - val inMemory = TestGraphs.singleStepInScala(original)(fn) - // Add a time: - val inWithTime = original.zipWithIndex.map { case (item, time) => (time.toLong, item) } - val batcher = randomBatcher(inWithTime) - val testStore = TestStore[Int,Int]("test", batcher, initStore, inWithTime.size) - val (buffer, source) = testSource(inWithTime) - - val summer = TestGraphs.singleStepJob[Scalding,(Long,Int),Int,Int](source, testStore)(t => - fn(t._2)) - - val scald = Scalding("scalaCheckJob") - val intr = batchedCover(batcher, 0L, original.size.toLong) - val ws = new LoopState(intr) - val mode: Mode = TestMode(t => (testStore.sourceToBuffer ++ buffer).get(t)) - - scald.run(ws, mode, scald.plan(summer)) - // Now check that the inMemory == - - compareMaps(original, Monoid.plus(initStore, inMemory), testStore) must be_==(true) - } - - "match scala for flatMapKeys jobs" in { - val original = sample[List[Int]] - val initStore = sample[Map[Int,Int]] - val fnA = sample[(Int) => List[(Int, Int)]] - val fnB = sample[Int => List[Int]] - val inMemory = TestGraphs.singleStepMapKeysInScala(original)(fnA, fnB) - // Add a time: - val inWithTime = original.zipWithIndex.map { case (item, time) => (time.toLong, item) } - val batcher = randomBatcher(inWithTime) - val testStore = TestStore[Int,Int]("test", batcher, initStore, inWithTime.size) - - val (buffer, source) = testSource(inWithTime) - - val summer = TestGraphs.singleStepMapKeysJob[Scalding,(Long,Int),Int,Int, Int](source, testStore)(t => - fnA(t._2), fnB) - - val intr = batchedCover(batcher, 0L, original.size.toLong) - val scald = Scalding("scalaCheckJob") - val ws = new LoopState(intr) - val mode: Mode = TestMode(t => (testStore.sourceToBuffer ++ buffer).get(t)) - - scald.run(ws, mode, scald.plan(summer)) - // Now check that the inMemory == - - compareMaps(original, Monoid.plus(initStore, inMemory), testStore) must beTrue - } - - "match scala for multiple summer jobs" in { - val original = sample[List[Int]] - val initStoreA = sample[Map[Int,Int]] - val initStoreB = sample[Map[Int,Int]] - val fnA = sample[(Int) => List[(Int)]] - val fnB = sample[(Int) => List[(Int, Int)]] - val fnC = sample[(Int) => List[(Int, Int)]] - val (inMemoryA, inMemoryB) = TestGraphs.multipleSummerJobInScala(original)(fnA, fnB, fnC) - - // Add a time: - val inWithTime = original.zipWithIndex.map { case (item, time) => (time.toLong, item) } - val batcher = randomBatcher(inWithTime) - val testStoreA = TestStore[Int,Int]("testA", batcher, initStoreA, inWithTime.size) - val testStoreB = TestStore[Int,Int]("testB", batcher, initStoreB, inWithTime.size) - val (buffer, source) = testSource(inWithTime) - - val tail = TestGraphs.multipleSummerJob[Scalding, (Long, Int), Int, Int, Int, Int, Int](source, testStoreA, testStoreB)({t => fnA(t._2)}, fnB, fnC) - - val scald = Scalding("scalaCheckMultipleSumJob") - val intr = batchedCover(batcher, 0L, original.size.toLong) - val ws = new LoopState(intr) - val mode: Mode = TestMode(t => (testStoreA.sourceToBuffer ++ testStoreB.sourceToBuffer ++ buffer).get(t)) - - scald.run(ws, mode, scald.plan(tail)) - // Now check that the inMemory == - - compareMaps(original, Monoid.plus(initStoreA, inMemoryA), testStoreA) must beTrue - compareMaps(original, Monoid.plus(initStoreB, inMemoryB), testStoreB) must beTrue - } - - - "match scala for leftJoin jobs" in { - val original = sample[List[Int]] - val prejoinMap = sample[(Int) => List[(Int, Int)]] - val service = sample[(Int,Int) => Option[Int]] - val postJoin = sample[((Int, (Int, Option[Int]))) => List[(Int, Int)]] - // We need to keep track of time correctly to use the service - var fakeTime = -1 - val timeIncIt = new Iterator[Int] { - val inner = original.iterator - def hasNext = inner.hasNext - def next = { - fakeTime += 1 - inner.next - } - } - val srvWithTime = { (key: Int) => service(fakeTime, key) } - val inMemory = TestGraphs.leftJoinInScala(timeIncIt)(srvWithTime)(prejoinMap)(postJoin) - - // Add a time: - val allKeys = original.flatMap(prejoinMap).map { _._1 } - val allTimes = (0 until original.size) - val stream = for { time <- allTimes; key <- allKeys; v = service(time, key) } yield (time.toLong, (key, v)) - - val inWithTime = original.zipWithIndex.map { case (item, time) => (time.toLong, item) } - val batcher = randomBatcher(inWithTime) - val initStore = sample[Map[Int, Int]] - val testStore = TestStore[Int,Int]("test", batcher, initStore, inWithTime.size) - - /** - * Create the batched service - */ - val batchedService = stream.groupBy { case (time, _) => batcher.batchOf(Timestamp(time)) } - val testService = new TestService[Int, Int]("srv", batcher, batcher.batchOf(Timestamp(0)).prev, batchedService) - - val (buffer, source) = testSource(inWithTime) - - val summer = - TestGraphs.leftJoinJob[Scalding,(Long, Int),Int,Int,Int,Int](source, testService, testStore) { tup => prejoinMap(tup._2) }(postJoin) - - val intr = batchedCover(batcher, 0L, original.size.toLong) - val scald = Scalding("scalaCheckleftJoinJob") - val ws = new LoopState(intr) - val mode: Mode = TestMode(s => (testStore.sourceToBuffer ++ buffer ++ testService.sourceToBuffer).get(s)) - - scald.run(ws, mode, summer) - // Now check that the inMemory == - - compareMaps(original, Monoid.plus(initStore, inMemory), testStore) must beTrue - } - - "match scala for diamond jobs with write" in { - val original = sample[List[Int]] - val fn1 = sample[(Int) => List[(Int, Int)]] - val fn2 = sample[(Int) => List[(Int, Int)]] - val inMemory = TestGraphs.diamondJobInScala(original)(fn1)(fn2) - // Add a time: - val inWithTime = original.zipWithIndex.map { case (item, time) => (time.toLong, item) } - val batcher = randomBatcher(inWithTime) - val initStore = sample[Map[Int, Int]] - val testStore = TestStore[Int,Int]("test", batcher, initStore, inWithTime.size) - val testSink = new TestSink[(Long,Int)] - val (buffer, source) = testSource(inWithTime) - - val summer = TestGraphs - .diamondJob[Scalding,(Long, Int),Int,Int](source, - testSink, - testStore)(t => fn1(t._2))(t => fn2(t._2)) - - val scald = Scalding("scalding-diamond-Job") - val intr = batchedCover(batcher, 0L, original.size.toLong) - val ws = new LoopState(intr) - val mode: Mode = TestMode(s => (testStore.sourceToBuffer ++ buffer).get(s)) - - scald.run(ws, mode, summer) - // Now check that the inMemory == - - val sinkOut = testSink.reset - compareMaps(original, Monoid.plus(initStore, inMemory), testStore) must beTrue - val wrongSink = sinkOut.map { _._2 }.toList != inWithTime - wrongSink must be_==(false) - if(wrongSink) { - println("input: " + inWithTime) - println("SinkExtra: " + (sinkOut.map(_._2).toSet -- inWithTime.toSet)) - println("SinkMissing: " + (inWithTime.toSet -- sinkOut.map(_._2).toSet)) - } - } - - "Correctly aggregate multiple sumByKeys" in { - val original = sample[List[(Int,Int)]] - val keyExpand = sample[(Int) => List[Int]] - val (inMemoryA, inMemoryB) = TestGraphs.twoSumByKeyInScala(original, keyExpand) - // Add a time: - val inWithTime = original.zipWithIndex.map { case (item, time) => (time.toLong, item) } - val batcher = randomBatcher(inWithTime) - val initStore = sample[Map[Int, Int]] - val testStoreA = TestStore[Int,Int]("testA", batcher, initStore, inWithTime.size) - val testStoreB = TestStore[Int,Int]("testB", batcher, initStore, inWithTime.size) - val (buffer, source) = testSource(inWithTime) - - val summer = TestGraphs - .twoSumByKey[Scalding,Int,Int,Int](source.map(_._2), testStoreA, keyExpand, testStoreB) - - val scald = Scalding("scalding-diamond-Job") - val intr = batchedCover(batcher, 0L, original.size.toLong) - val ws = new LoopState(intr) - val mode: Mode = TestMode((testStoreA.sourceToBuffer ++ testStoreB.sourceToBuffer ++ buffer).get(_)) - - scald.run(ws, mode, summer) - // Now check that the inMemory == - - compareMaps(original, Monoid.plus(initStore, inMemoryA), testStoreA, "A") must beTrue - compareMaps(original, Monoid.plus(initStore, inMemoryB), testStoreB, "B") must beTrue - } - } -} - -object VersionBatchLaws extends Properties("VersionBatchLaws") { - property("version -> BatchID -> version") = forAll { (l: Long) => - val vbs = new VersionedBatchStore[Int, Int, Array[Byte], Array[Byte]](null, - 0, Batcher.ofHours(1))(null)(null) - val b = vbs.versionToBatchID(l) - vbs.batchIDToVersion(b) <= l - } - property("BatchID -> version -> BatchID") = forAll { (bint: Int) => - val b = BatchID(bint) - val vbs = new VersionedBatchStore[Int, Int, Array[Byte], Array[Byte]](null, - 0, Batcher.ofHours(1))(null)(null) - val v = vbs.batchIDToVersion(b) - vbs.versionToBatchID(v) == b - } - property("version is an upperbound on time") = forAll { (lBig: Long) => - val l = lBig/1000L - val batcher = Batcher.ofHours(1) - val vbs = new VersionedBatchStore[Int, Int, Array[Byte], Array[Byte]](null, - 0, batcher)(null)(null) - val b = vbs.versionToBatchID(l) - (batcher.earliestTimeOf(b.next).milliSinceEpoch <= l) && - (batcher.earliestTimeOf(b).milliSinceEpoch < l) - (batcher.earliestTimeOf(b.next.next).milliSinceEpoch > l) - } -} - -class ScaldingSerializationSpecs extends Specification { - - - implicit def tupleExtractor[T <: (Long, _)]: TimeExtractor[T] = TimeExtractor( _._1 ) - - "ScaldingPlatform" should { - "serialize Hadoop Jobs for single step jobs" in { - // Add a time: - val inWithTime = List(1, 2, 3).zipWithIndex.map { case (item, time) => (time.toLong, item) } - val batcher = ScaldingLaws.randomBatcher(inWithTime) - val testStore = TestStore[Int,Int]("test", batcher, Iterable.empty, inWithTime.size) - val (buffer, source) = ScaldingLaws.testSource(inWithTime) - - val summer = TestGraphs.singleStepJob[Scalding,(Long, Int),Int,Int](source, testStore) { - tup => List((1 -> tup._2)) - } - - val mode = HadoopTest(new Configuration, {case x: ScaldingSource => buffer.get(x)}) - val intr = Interval.leftClosedRightOpen(0L, inWithTime.size.toLong) - val scald = Scalding("scalaCheckJob") - - (try { scald.toFlow(intr, mode, scald.plan(summer)); true } - catch { case t: Throwable => println(toTry(t)); false }) must beTrue - } - } -} diff --git a/summingbird-storm-test/src/main/scala/com/twitter/summingbird/storm/StormTestRun.scala b/summingbird-storm-test/src/main/scala/com/twitter/summingbird/storm/StormTestRun.scala new file mode 100644 index 000000000..73174f42a --- /dev/null +++ b/summingbird-storm-test/src/main/scala/com/twitter/summingbird/storm/StormTestRun.scala @@ -0,0 +1,117 @@ +/* + Copyright 2013 Twitter, Inc. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + */ + +package com.twitter.summingbird.storm + +import com.twitter.algebird.Semigroup +import backtype.storm.{ Config => BacktypeStormConfig, LocalCluster, Testing } +import com.twitter.summingbird.online.executor.InflightTuples +import backtype.storm.testing.{ CompleteTopologyParam, MockedSources } +import com.twitter.summingbird.storm.spout.TraversableSpout +import com.twitter.summingbird.online.option._ +import com.twitter.summingbird.option._ +import com.twitter.summingbird._ +import com.twitter.summingbird.planner._ +import com.twitter.tormenta.spout.Spout +import scala.collection.JavaConverters._ +import java.security.Permission +import com.twitter.util.Duration + + + +/** + * This stops Storm's exception handling triggering an exit(1) + */ +private[storm] class MySecurityManager extends SecurityManager { + override def checkExit(status: Int): Unit = { + throw new SecurityException(); + } + override def checkAccess(t: Thread) = {} + override def checkPermission(p: Permission) = {} +} + +/* + * This is a wrapper to run a storm topology. + * We use the SecurityManager code to catch the System.exit storm calls when it + * fails. We wrap it into a normal exception instead so it can report better/retry. + */ + +object StormTestRun { + private def completeTopologyParam(conf: BacktypeStormConfig) = { + val ret = new CompleteTopologyParam() + ret.setMockedSources(new MockedSources) + ret.setStormConf(conf) + ret.setCleanupState(false) + ret + } + + + private def tryRun(plannedTopology: PlannedTopology): Unit = { + //Before running the external Command + val oldSecManager = System.getSecurityManager() + System.setSecurityManager(new MySecurityManager()); + InflightTuples.reset + try { + val cluster = new LocalCluster() + cluster.submitTopology("test topology", plannedTopology.config, plannedTopology.topology) + Thread.sleep(4000) + cluster.killTopology("test topology") + Thread.sleep(1000) + cluster.shutdown + } finally { + System.setSecurityManager(oldSecManager) + } + require(InflightTuples.get == 0, "Inflight tuples is: %d".format(InflightTuples.get)) + } + + def apply(graph: TailProducer[Storm, Any])(implicit storm: Storm) { + val topo = storm.plan(graph) + apply(topo) + } + + def simpleRun[T, K, V: Semigroup](original: List[T], mkJob: (Producer[Storm, T], Storm#Store[K, V]) => TailProducer[Storm, Any]) + : TestStore[K, V] = { + + implicit def extractor[T]: TimeExtractor[T] = TimeExtractor(_ => 0L) + + val (id, store) = TestStore.createStore[K, V]() + + val job = mkJob( + Storm.source(TraversableSpout(original)), + store + ) + + implicit val s = Storm.local(Map( + "DEFAULT" -> Options().set(CacheSize(4)) + .set(FlushFrequency(Duration.fromMilliseconds(1))) + )) + + apply(job) + TestStore[K, V](id).getOrElse(sys.error("Error running test, unable to find store at the end")) + } + + def apply(plannedTopology: PlannedTopology) { + this.synchronized { + try { + tryRun(plannedTopology) + } catch { + case _: Throwable => + Thread.sleep(3000) + tryRun(plannedTopology) + } + } + } +} diff --git a/summingbird-storm-test/src/main/scala/com/twitter/summingbird/storm/TestStore.scala b/summingbird-storm-test/src/main/scala/com/twitter/summingbird/storm/TestStore.scala new file mode 100644 index 000000000..aa6ddb566 --- /dev/null +++ b/summingbird-storm-test/src/main/scala/com/twitter/summingbird/storm/TestStore.scala @@ -0,0 +1,109 @@ +/* + Copyright 2013 Twitter, Inc. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + */ + +package com.twitter.summingbird.storm + +import com.twitter.algebird.Semigroup +import com.twitter.storehaus.algebra.MergeableStore +import com.twitter.summingbird.batch.{BatchID, Batcher} +import com.twitter.util.Future +import java.util.{Collections, HashMap, Map => JMap, UUID} +import java.util.concurrent.atomic.AtomicInteger +import scala.collection.mutable.SynchronizedMap +import java.util.WeakHashMap +import scala.collection.JavaConverters._ + + + +object TestStore { + private val testStores = new WeakHashMap[String, TestStore[_, _]] + + def apply[K, V: Semigroup](storeID: String): Option[TestStore[K, V]] = + (Option(testStores.get(storeID)).map{s => + s.asInstanceOf[TestStore[K, V]]}) + + private def buildStore[K, V: Semigroup](initialData: Map[K, V]) : String = { + val storeID = UUID.randomUUID.toString + val newInitStore = TestStore[K, V](storeID, initialData) + testStores.synchronized { + testStores.put(storeID, newInitStore) + } + storeID + } + + def createBatchedStore[K, V] + (initialData: Map[(K, BatchID), V] = Map.empty[(K, BatchID), V]) + (implicit batcher: Batcher, valueSG: Semigroup[V]): + (String, MergeableStoreSupplier[K, V]) = { + + val storeID = buildStore[(K, BatchID), V](initialData) + val supplier = MergeableStoreSupplier.from( + TestStore.apply[(K, BatchID), V](storeID) + .getOrElse(sys.error("Weak hash map no longer contains store")) + ) + (storeID, supplier) + } + + def createStore[K, V: Semigroup](initialData: Map[K, V] = Map.empty[K, V]): + (String, MergeableStoreSupplier[K, V]) = { + val storeID = buildStore[K, V](initialData) + val supplier = MergeableStoreSupplier.fromOnlineOnly( + TestStore.apply[K, V](storeID) + .getOrElse(sys.error("Weak hash map no longer contains store")) + ) + + (storeID, supplier) + } +} + + +case class TestStore[K, V: Semigroup](storeID: String, initialData: Map[K, V]) extends MergeableStore[K, V] { + private val backingStore: JMap[K, Option[V]] = + Collections.synchronizedMap(new HashMap[K, Option[V]]()) + val updates: AtomicInteger = new AtomicInteger(0) + val reads: AtomicInteger = new AtomicInteger(0) + + def toScala: Map[K, V] = backingStore.asScala.collect{ case (k, Some(v)) => (k, v)}.toMap + + private def getOpt(k: K) = { + reads.incrementAndGet + Option(backingStore.get(k)).flatMap(i => i) + } + + val semigroup = implicitly[Semigroup[V]] + + override def get(k: K) = Future.value(getOpt(k)) + + override def put(pair: (K, Option[V])) = { + val (k, optV) = pair + if (optV.isDefined) + backingStore.put(k, optV) + else + backingStore.remove(k) + updates.incrementAndGet + Future.Unit + } + + override def merge(pair: (K, V)) = { + val (k, v) = pair + val oldV = getOpt(k) + val newV = Semigroup.plus(Some(v), oldV) + updates.incrementAndGet + backingStore.put(k, newV) + Future.value(oldV) + } + +} diff --git a/summingbird-storm/src/test/scala/com/twitter/summingbird/storm/InjectionLaws.scala b/summingbird-storm-test/src/test/scala/com/twitter/summingbird/storm/InjectionLaws.scala similarity index 87% rename from summingbird-storm/src/test/scala/com/twitter/summingbird/storm/InjectionLaws.scala rename to summingbird-storm-test/src/test/scala/com/twitter/summingbird/storm/InjectionLaws.scala index e4827c36a..742c4cb51 100644 --- a/summingbird-storm/src/test/scala/com/twitter/summingbird/storm/InjectionLaws.scala +++ b/summingbird-storm-test/src/test/scala/com/twitter/summingbird/storm/InjectionLaws.scala @@ -27,11 +27,11 @@ object InjectionLaws extends Properties("InjectionTests") { implicit def ts: Arbitrary[Timestamp] = Arbitrary(Arbitrary.arbitrary[Long].map(Timestamp(_))) - property("Single injection works") = forAll { in: (Timestamp, String) => + property("Single injection works") = forAll { in: String => val inj = new SingleItemInjection[String] inj.invert(inj(in)).get == in } - property("KV injection works") = forAll { in: (Timestamp, (String, List[Int])) => + property("KV injection works") = forAll { in: (String, List[Int]) => val inj = new KeyValueInjection[String, List[Int]] inj.invert(inj(in)).get == in } diff --git a/summingbird-storm/src/test/scala/com/twitter/summingbird/storm/StormLaws.scala b/summingbird-storm-test/src/test/scala/com/twitter/summingbird/storm/StormLaws.scala similarity index 63% rename from summingbird-storm/src/test/scala/com/twitter/summingbird/storm/StormLaws.scala rename to summingbird-storm-test/src/test/scala/com/twitter/summingbird/storm/StormLaws.scala index dcccbc809..0fcc43b60 100644 --- a/summingbird-storm/src/test/scala/com/twitter/summingbird/storm/StormLaws.scala +++ b/summingbird-storm-test/src/test/scala/com/twitter/summingbird/storm/StormLaws.scala @@ -49,70 +49,6 @@ import java.security.Permission * Tests for Summingbird's Storm planner. */ -/** - * State required to perform a single Storm test run. - */ -case class TestState[T, K, V]( - store: JMap[(K, BatchID), Option[V]] = Collections.synchronizedMap(new HashMap[(K, BatchID), Option[V]]()), - used: ArrayBuffer[T] = new ArrayBuffer[T] with SynchronizedBuffer[T], - placed: AtomicInteger = new AtomicInteger -) - -object TrueGlobalState { - val data = new MutableHashMap[String, TestState[Int, Int, Int]] - with SynchronizedMap[String, TestState[Int, Int, Int]] -} - - class MySecurityManager extends SecurityManager { - override def checkExit(status: Int): Unit = { - throw new SecurityException(); - } - override def checkAccess(t: Thread) = {} - override def checkPermission(p: Permission) = {} - } - -/* - * This is a wrapper to run a storm topology. - * We use the SecurityManager code to catch the System.exit storm calls when it - * fails. We wrap it into a normal exception instead so it can report better/retry. - */ - -object StormRunner { - private def completeTopologyParam(conf: BacktypeStormConfig) = { - val ret = new CompleteTopologyParam() - ret.setMockedSources(new MockedSources) - ret.setStormConf(conf) - ret.setCleanupState(false) - ret - } - - private def tryRun(plannedTopology: PlannedTopology): Unit = { - //Before running the external Command - val oldSecManager = System.getSecurityManager() - System.setSecurityManager(new MySecurityManager()); - try { - val cluster = new LocalCluster() - Testing.completeTopology(cluster, plannedTopology.topology, completeTopologyParam(plannedTopology.config)) - // Sleep to prevent this race: https://github.com/nathanmarz/storm/pull/667 - Thread.sleep(1000) - cluster.shutdown - } finally { - System.setSecurityManager(oldSecManager) - } - } - - def run(plannedTopology: PlannedTopology) { - this.synchronized { - try { - tryRun(plannedTopology) - } catch { - case _: Throwable => - Thread.sleep(3000) - tryRun(plannedTopology) - } - } - } -} object StormLaws extends Specification { sequential @@ -124,77 +60,19 @@ object StormLaws extends Specification { implicit def extractor[T]: TimeExtractor[T] = TimeExtractor(_ => 0L) implicit val batcher = Batcher.unit - /** - * Global state shared by all tests. - */ - val globalState = TrueGlobalState.data - - /** - * Returns a MergeableStore that routes get, put and merge calls - * through to the backing store in the proper globalState entry. - */ - def testingStore(id: String) = - new MergeableStore[(Int, BatchID), Int] with java.io.Serializable { - val semigroup = implicitly[Semigroup[Int]] - def wrappedStore = globalState(id).store - private def getOpt(k: (Int, BatchID)) = Option(wrappedStore.get(k)).flatMap(i => i) - override def get(k: (Int, BatchID)) = Future.value(getOpt(k)) - override def put(pair: ((Int, BatchID), Option[Int])) = { - val (k, optV) = pair - if (optV.isDefined) - wrappedStore.put(k, optV) - else - wrappedStore.remove(k) - globalState(id).placed.incrementAndGet - Future.Unit - } - override def merge(pair: ((Int, BatchID), Int)) = { - val (k, v) = pair - val oldV = getOpt(k) - val newV = Semigroup.plus(Some(v), oldV) - wrappedStore.put(k, newV) - globalState(id).placed.incrementAndGet - Future.value(oldV) - } - } - val testFn = sample[Int => List[(Int, Int)]] - val storm = Storm.local(Map( - )) + implicit val storm = Storm.local(Map()) def sample[T: Arbitrary]: T = Arbitrary.arbitrary[T].sample.get - def genStore: (String, Storm#Store[Int, Int]) = { - val id = UUID.randomUUID.toString - globalState += (id -> TestState()) - val store = MergeableStoreSupplier(() => testingStore(id), Batcher.unit) - (id, store) - } + def genStore: (String, Storm#Store[Int, Int]) = TestStore.createStore[Int, Int]() def genSink:() => ((Int) => Future[Unit]) = () => {x: Int => append(x) Future.Unit } - /** - * Perform a single run of TestGraphs.singleStepJob using the - * supplied list of integers and the testFn defined above. - */ - def runOnce(original: List[Int])(mkJob: (Producer[Storm, Int], Storm#Store[Int, Int]) => TailProducer[Storm, Any]) - : TestState[Int, Int, Int] = { - - val (id, store) = genStore - - val job = mkJob( - Storm.source(TraversableSpout(original)), - store - ) - val topo = storm.plan(job) - StormRunner.run(topo) - globalState(id) - } - def memoryPlanWithoutSummer(original: List[Int])(mkJob: (Producer[Memory, Int], Memory#Sink[Int]) => TailProducer[Memory, Int]) : List[Int] = { val memory = new Memory @@ -225,13 +103,10 @@ object StormLaws extends Specification { Storm.sink[Int]({ (x: Int) => append(x); Future.Unit }) ) - val topo = storm.plan(job) - StormRunner.run(topo) + StormTestRun(job) StormLaws.outputList.toList } - - val nextFn = { pair: ((Int, (Int, Option[Int]))) => val (k, (v, joinedV)) = pair List((k -> joinedV.getOrElse(10))) @@ -245,15 +120,13 @@ object StormLaws extends Specification { "StormPlatform matches Scala for single step jobs" in { val original = sample[List[Int]] val returnedState = - runOnce(original)( + StormTestRun.simpleRun[Int, Int, Int](original, TestGraphs.singleStepJob[Storm, Int, Int, Int](_,_)(testFn) ) - Equiv[Map[Int, Int]].equiv( TestGraphs.singleStepInScala(original)(testFn), - returnedState.store.asScala.toMap - .collect { case ((k, batchID), Some(v)) => (k, v) } + returnedState.toScala ) must beTrue } @@ -261,15 +134,13 @@ object StormLaws extends Specification { val original = sample[List[Int]] val fn = {(x: Int) => List[(Int, Int)]()} val returnedState = - runOnce(original)( + StormTestRun.simpleRun[Int, Int, Int](original, TestGraphs.singleStepJob[Storm, Int, Int, Int](_,_)(fn) ) - Equiv[Map[Int, Int]].equiv( TestGraphs.singleStepInScala(original)(fn), - returnedState.store.asScala.toMap - .collect { case ((k, batchID), Some(v)) => (k, v) } + returnedState.toScala ) must beTrue } @@ -279,15 +150,14 @@ object StormLaws extends Specification { val fnB = sample[Int => List[(Int,Int)]] val returnedState = - runOnce(original)( + StormTestRun.simpleRun[Int, Int, Int](original, TestGraphs.twinStepOptionMapFlatMapJob[Storm, Int, Int, Int, Int](_,_)(fnA, fnB) ) Equiv[Map[Int, Int]].equiv( TestGraphs.twinStepOptionMapFlatMapScala(original)(fnA, fnB), - returnedState.store.asScala.toMap - .collect { case ((k, batchID), Some(v)) => (k, v) } + returnedState.toScala ) must beTrue } @@ -297,13 +167,12 @@ object StormLaws extends Specification { val fnB = sample[Int => List[(Int,Int)]] val returnedState = - runOnce(original)( + StormTestRun.simpleRun[Int, Int, Int](original, TestGraphs.twinStepOptionMapFlatMapJob[Storm, Int, Int, Int, Int](_,_)(fnA, fnB) ) Equiv[Map[Int, Int]].equiv( TestGraphs.twinStepOptionMapFlatMapScala(original)(fnA, fnB), - returnedState.store.asScala.toMap - .collect { case ((k, batchID), Some(v)) => (k, v) } + returnedState.toScala ) must beTrue } @@ -314,14 +183,13 @@ object StormLaws extends Specification { expander(x).flatMap{case (k, v) => List((k, v), (k, v), (k, v), (k, v), (k, v))} } val returnedState = - runOnce(original)( + StormTestRun.simpleRun[Int, Int, Int](original, TestGraphs.singleStepJob[Storm, Int, Int, Int](_,_)(expansionFunc) ) Equiv[Map[Int, Int]].equiv( TestGraphs.singleStepInScala(original)(expansionFunc), - returnedState.store.asScala.toMap - .collect { case ((k, batchID), Some(v)) => (k, v) } + returnedState.toScala ) must beTrue } @@ -330,14 +198,13 @@ object StormLaws extends Specification { val fnA = sample[Int => List[(Int, Int)]] val fnB = sample[Int => List[Int]] val returnedState = - runOnce(original)( + StormTestRun.simpleRun[Int, Int, Int](original, TestGraphs.singleStepMapKeysJob[Storm, Int, Int, Int, Int](_,_)(fnA, fnB) ) Equiv[Map[Int, Int]].equiv( TestGraphs.singleStepMapKeysInScala(original)(fnA, fnB), - returnedState.store.asScala.toMap - .collect { case ((k, batchID), Some(v)) => (k, v) } + returnedState.toScala ) must beTrue } @@ -345,21 +212,20 @@ object StormLaws extends Specification { val original = sample[List[Int]] val staticFunc = { i: Int => List((i -> i)) } val returnedState = - runOnce(original)( + StormTestRun.simpleRun[Int, Int, Int](original, TestGraphs.leftJoinJob[Storm, Int, Int, Int, Int, Int](_, service, _)(staticFunc)(nextFn) ) Equiv[Map[Int, Int]].equiv( TestGraphs.leftJoinInScala(original)(serviceFn) (staticFunc)(nextFn), - returnedState.store.asScala.toMap - .collect { case ((k, batchID), Some(v)) => (k, v) } + returnedState.toScala ) must beTrue } "StormPlatform matches Scala for optionMap only jobs" in { val original = sample[List[Int]] - val (id, store) = genStore + val (id, storeSupplier) = genStore val cluster = new LocalCluster() @@ -367,16 +233,13 @@ object StormLaws extends Specification { Storm.source(TraversableSpout(original)) .filter(_ % 2 == 0) .map(_ -> 10) - .sumByKey(store) + .sumByKey(storeSupplier) - val topo = storm.plan(producer) - StormRunner.run(topo) + StormTestRun(producer) Equiv[Map[Int, Int]].equiv( MapAlgebra.sumByKey(original.filter(_ % 2 == 0).map(_ -> 10)), - globalState(id).store.asScala - .toMap - .collect { case ((k, batchID), Some(v)) => (k, v) } + TestStore[Int, Int](id).get.toScala ) must beTrue } @@ -407,23 +270,20 @@ object StormLaws extends Specification { val tail = TestGraphs.multipleSummerJob[Storm, Int, Int, Int, Int, Int, Int](source, store1, store2)(simpleOp, doubler, doubler) - val topo = storm.plan(tail) - StormRunner.run(topo) + StormTestRun(tail) val (scalaA, scalaB) = TestGraphs.multipleSummerJobInScala(original)(simpleOp, doubler, doubler) - val store1Map = globalState(store1Id).store.asScala.toMap - val store2Map = globalState(store2Id).store.asScala.toMap + val store1Map = TestStore[Int, Int](store1Id).get.toScala + val store2Map = TestStore[Int, Int](store2Id).get.toScala Equiv[Map[Int, Int]].equiv( scalaA, store1Map - .collect { case ((k, batchID), Some(v)) => (k, v) } ) must beTrue Equiv[Map[Int, Int]].equiv( scalaB, store2Map - .collect { case ((k, batchID), Some(v)) => (k, v) } ) must beTrue } @@ -452,20 +312,17 @@ object StormLaws extends Specification { val tail = TestGraphs.realJoinTestJob[Storm, Int, Int, Int, Int, Int, Int, Int, Int, Int](source1, source2, source3, source4, service, store1, fn1, fn2, fn3, preJoinFn, postJoinFn) - val topo = storm.plan(tail) OnlinePlan(tail).nodes.size must beLessThan(10) - - StormRunner.run(topo) + StormTestRun(tail) val scalaA = TestGraphs.realJoinTestJobInScala(original1, original2, original3, original4, serviceFn, fn1, fn2, fn3, preJoinFn, postJoinFn) - val store1Map = globalState(store1Id).store.asScala.toMap + val store1Map = TestStore[Int, Int](store1Id).get.toScala Equiv[Map[Int, Int]].equiv( scalaA, store1Map - .collect { case ((k, batchID), Some(v)) => (k, v) } ) must beTrue } diff --git a/summingbird-storm/src/test/scala/com/twitter/summingbird/storm/StormPlanTopology.scala b/summingbird-storm-test/src/test/scala/com/twitter/summingbird/storm/StormPlanTopology.scala similarity index 100% rename from summingbird-storm/src/test/scala/com/twitter/summingbird/storm/StormPlanTopology.scala rename to summingbird-storm-test/src/test/scala/com/twitter/summingbird/storm/StormPlanTopology.scala diff --git a/summingbird-storm/src/test/scala/com/twitter/summingbird/storm/TopologyTests.scala b/summingbird-storm-test/src/test/scala/com/twitter/summingbird/storm/TopologyTests.scala similarity index 73% rename from summingbird-storm/src/test/scala/com/twitter/summingbird/storm/TopologyTests.scala rename to summingbird-storm-test/src/test/scala/com/twitter/summingbird/storm/TopologyTests.scala index 409134d04..fe730b03d 100644 --- a/summingbird-storm/src/test/scala/com/twitter/summingbird/storm/TopologyTests.scala +++ b/summingbird-storm-test/src/test/scala/com/twitter/summingbird/storm/TopologyTests.scala @@ -56,44 +56,6 @@ object TopologyTests extends Specification { implicit def extractor[T]: TimeExtractor[T] = TimeExtractor(_ => 0L) implicit val batcher = Batcher.unit - def createGlobalState[T, K, V] = - new MutableHashMap[String, TestState[T, K, V]] - with SynchronizedMap[String, TestState[T, K, V]] - - /** - * Global state shared by all tests. - */ - val globalState = createGlobalState[Int, Int, Int] - - /** - * Returns a MergeableStore that routes get, put and merge calls - * through to the backing store in the proper globalState entry. - */ - def testingStore(id: String) = - new MergeableStore[(Int, BatchID), Int] with java.io.Serializable { - val semigroup = implicitly[Semigroup[Int]] - def wrappedStore = globalState(id).store - private def getOpt(k: (Int, BatchID)) = Option(wrappedStore.get(k)).flatMap(i => i) - override def get(k: (Int, BatchID)) = Future.value(getOpt(k)) - override def put(pair: ((Int, BatchID), Option[Int])) = { - val (k, optV) = pair - if (optV.isDefined) - wrappedStore.put(k, optV) - else - wrappedStore.remove(k) - globalState(id).placed.incrementAndGet - Future.Unit - } - override def merge(pair: ((Int, BatchID), Int)) = { - val (k, v) = pair - val oldV = getOpt(k) - val newV = Semigroup.plus(Some(v), oldV) - wrappedStore.put(k, newV) - globalState(id).placed.incrementAndGet - Future.value(oldV) - } - } - /** * The function tested below. We can't generate a function with * ScalaCheck, as we need to know the number of tuples that the @@ -101,7 +63,7 @@ object TopologyTests extends Specification { */ val testFn = { i: Int => List((i -> i)) } - val storm = Storm.local() + implicit val storm = Storm.local() def sample[T: Arbitrary]: T = Arbitrary.arbitrary[T].sample.get @@ -112,12 +74,10 @@ object TopologyTests extends Specification { def funcToPlan(mkJob: (Producer[Storm, Int], Storm#Store[Int, Int]) => TailProducer[Storm, Any]) : StormTopology = { val original = sample[List[Int]] - val id = UUID.randomUUID.toString - globalState += (id -> TestState()) val job = mkJob( Storm.source(TraversableSpout(original)), - MergeableStoreSupplier(() => testingStore(id), Batcher.unit) + TestStore.createStore[Int, Int]()._2 ) storm.plan(job).topology @@ -145,7 +105,7 @@ object TopologyTests extends Specification { val nodeName = "super dooper node" val p = Storm.source(TraversableSpout(sample[List[Int]])) .flatMap(testFn).name(nodeName) - .sumByKey(MergeableStoreSupplier(() => testingStore(UUID.randomUUID.toString), Batcher.unit)) + .sumByKey(TestStore.createStore[Int, Int]()._2) val opts = Map(nodeName -> Options().set(FlatMapParallelism(50))) val storm = Storm.local(opts) @@ -156,7 +116,7 @@ object TopologyTests extends Specification { // Tail will have 1 -, distance from there should be onwards val TDistMap = bolts.map{case (k, v) => (k.split("-").size - 1, v)} - TDistMap(1).get_common.get_parallelism_hint must_== 50 + TDistMap(1).get_common.get_parallelism_hint must_== 50 } "With 2 names in a row we take the closest name" in { @@ -164,7 +124,7 @@ object TopologyTests extends Specification { val otherNodeName = "super dooper node" val p = Storm.source(TraversableSpout(sample[List[Int]])) .flatMap(testFn).name(nodeName).name(otherNodeName) - .sumByKey(MergeableStoreSupplier(() => testingStore(UUID.randomUUID.toString), Batcher.unit)) + .sumByKey(TestStore.createStore[Int, Int]()._2) val opts = Map(otherNodeName -> Options().set(FlatMapParallelism(40)), nodeName -> Options().set(FlatMapParallelism(50))) @@ -177,7 +137,7 @@ object TopologyTests extends Specification { // Tail will have 1 -, distance from there should be onwards val TDistMap = bolts.map{case (k, v) => (k.split("-").size - 1, v)} - TDistMap(1).get_common.get_parallelism_hint must_== 50 + TDistMap(1).get_common.get_parallelism_hint must_== 50 } "If the closes doesnt contain the option we keep going" in { @@ -185,7 +145,7 @@ object TopologyTests extends Specification { val otherNodeName = "super dooper node" val p = Storm.source(TraversableSpout(sample[List[Int]])) .flatMap(testFn).name(otherNodeName).name(nodeName) - .sumByKey(MergeableStoreSupplier(() => testingStore(UUID.randomUUID.toString), Batcher.unit)) + .sumByKey(TestStore.createStore[Int, Int]()._2) val opts = Map(otherNodeName -> Options().set(SpoutParallelism(30)), nodeName -> Options().set(FlatMapParallelism(50))) @@ -197,14 +157,14 @@ object TopologyTests extends Specification { // Tail will have 1 -, distance from there should be onwards val TDistMap = bolts.map{case (k, v) => (k.split("-").size - 1, v)} - TDistMap(1).get_common.get_parallelism_hint must_== 50 + TDistMap(1).get_common.get_parallelism_hint must_== 50 } "Options propagate backwards" in { val nodeName = "super dooper node" val p = Storm.source(TraversableSpout(sample[List[Int]])) .flatMap(testFn).name(nodeName).name("Throw away name") - .sumByKey(MergeableStoreSupplier(() => testingStore(UUID.randomUUID.toString), Batcher.unit)) + .sumByKey(TestStore.createStore[Int, Int]()._2) val opts = Map(nodeName -> Options().set(FlatMapParallelism(50)).set(SpoutParallelism(30))) val storm = Storm.local(opts) @@ -214,7 +174,7 @@ object TopologyTests extends Specification { val spouts = stormTopo.get_spouts val spout = spouts.head._2 - spout.get_common.get_parallelism_hint must_== 30 + spout.get_common.get_parallelism_hint must_== 30 } "Options don't propagate forwards" in { @@ -222,7 +182,7 @@ object TopologyTests extends Specification { val otherNodeName = "super dooper node" val p = Storm.source(TraversableSpout(sample[List[Int]])) .flatMap(testFn).name(otherNodeName).name(nodeName) - .sumByKey(MergeableStoreSupplier(() => testingStore(UUID.randomUUID.toString), Batcher.unit)) + .sumByKey(TestStore.createStore[Int, Int]()._2) val opts = Map(otherNodeName -> Options().set(SpoutParallelism(30)).set(SummerParallelism(50)), nodeName -> Options().set(FlatMapParallelism(50))) diff --git a/summingbird-storm/src/main/scala/com/twitter/summingbird/storm/BaseBolt.scala b/summingbird-storm/src/main/scala/com/twitter/summingbird/storm/BaseBolt.scala index 566181564..87b1761cf 100644 --- a/summingbird-storm/src/main/scala/com/twitter/summingbird/storm/BaseBolt.scala +++ b/summingbird-storm/src/main/scala/com/twitter/summingbird/storm/BaseBolt.scala @@ -25,8 +25,7 @@ import backtype.storm.tuple.{Tuple, TupleImpl, Fields} import java.util.{ Map => JMap } -import com.twitter.summingbird.batch.Timestamp -import com.twitter.summingbird.storm.option.{AckOnEntry, AnchorTuples, MaxWaitingFutures} +import com.twitter.summingbird.storm.option.{AckOnEntry, AnchorTuples} import com.twitter.summingbird.online.executor.OperationContainer import com.twitter.summingbird.online.executor.{InflightTuples, InputState} @@ -95,7 +94,7 @@ case class BaseBolt[I,O](metrics: () => TraversableOnce[StormMetric[_]], } } - private def finish(inputs: List[InputState[Tuple]], results: TraversableOnce[(Timestamp, O)]) { + private def finish(inputs: List[InputState[Tuple]], results: TraversableOnce[O]) { var emitCount = 0 if(hasDependants) { if(anchorTuples.anchor) { diff --git a/summingbird-storm/src/main/scala/com/twitter/summingbird/storm/Constants.scala b/summingbird-storm/src/main/scala/com/twitter/summingbird/storm/Constants.scala index 92906a028..bfdb59fdf 100644 --- a/summingbird-storm/src/main/scala/com/twitter/summingbird/storm/Constants.scala +++ b/summingbird-storm/src/main/scala/com/twitter/summingbird/storm/Constants.scala @@ -16,11 +16,7 @@ limitations under the License. package com.twitter.summingbird.storm -import com.twitter.summingbird.option.MonoidIsCommutative -import com.twitter.summingbird.storm.option.{OnlineSuccessHandler => SOnlineSuccessHandler, - OnlineExceptionHandler => SOnlineExceptionHandler, - MaxWaitingFutures => SMaxWaitingFutures, - CacheSize => DEPCacheSize, _ } +import com.twitter.summingbird.storm.option._ import com.twitter.summingbird.option._ import com.twitter.summingbird.online.option._ @@ -55,4 +51,6 @@ object Constants { val DEFAULT_SOFT_MEMORY_FLUSH_PERCENT = SoftMemoryFlushPercent(80.0F) val DEFAULT_VALUE_COMBINER_CACHE_SIZE = ValueCombinerCacheSize(100) val DEFAULT_ACK_ON_ENTRY = AckOnEntry(false) + val DEFAULT_MAX_EMIT_PER_EXECUTE = MaxEmitPerExecute(Int.MaxValue) + val DEFAULT_SUMMER_BATCH_MULTIPLIER = SummerBatchMultiplier(1) } diff --git a/summingbird-storm/src/main/scala/com/twitter/summingbird/storm/FlatMapBoltProvider.scala b/summingbird-storm/src/main/scala/com/twitter/summingbird/storm/FlatMapBoltProvider.scala new file mode 100644 index 000000000..e83e495c1 --- /dev/null +++ b/summingbird-storm/src/main/scala/com/twitter/summingbird/storm/FlatMapBoltProvider.scala @@ -0,0 +1,204 @@ +/* + Copyright 2013 Twitter, Inc. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + */ + +package com.twitter.summingbird.storm + +import Constants._ +import backtype.storm.topology.TopologyBuilder +import backtype.storm.tuple.Fields +import backtype.storm.tuple.Tuple + +import com.twitter.algebird.{Semigroup, Monoid} +import com.twitter.summingbird._ +import com.twitter.summingbird.chill._ +import com.twitter.summingbird.batch.{BatchID, Batcher, Timestamp} +import com.twitter.summingbird.storm.option.{AckOnEntry, AnchorTuples} +import com.twitter.summingbird.online.{MultiTriggerCache, SummingQueueCache, CacheBuilder} +import com.twitter.summingbird.online.executor.InputState +import com.twitter.summingbird.online.option.{IncludeSuccessHandler, MaxWaitingFutures, MaxFutureWaitTime} +import com.twitter.summingbird.option.CacheSize +import com.twitter.summingbird.planner._ +import com.twitter.summingbird.online.executor +import com.twitter.summingbird.online.FlatMapOperation +import com.twitter.summingbird.storm.planner._ +import org.slf4j.LoggerFactory + +object FlatMapBoltProvider { + @transient private val logger = LoggerFactory.getLogger(FlatMapBoltProvider.getClass) + private def wrapTimeBatchIDKV[T, K, V](existingOp: FlatMapOperation[T, (K, V)])(batcher: Batcher): + FlatMapOperation[(Timestamp, T), ((K, BatchID), (Timestamp, V))] = { + FlatMapOperation.generic({case (ts: Timestamp, data: T) => + existingOp.apply(data).map { vals => + vals.map{ tup => + ((tup._1, batcher.batchOf(ts)), (ts, tup._2)) + } + } + }) + } + + def wrapTime[T, U](existingOp: FlatMapOperation[T, U]): FlatMapOperation[(Timestamp, T), (Timestamp, U)] = { + FlatMapOperation.generic({x: (Timestamp, T) => + existingOp.apply(x._2).map { vals => + vals.map((x._1, _)) + } + }) + } +} + +case class FlatMapBoltProvider(storm: Storm, stormDag: Dag[Storm], node: StormNode)(implicit topologyBuilder: TopologyBuilder) { + import FlatMapBoltProvider._ + + def getOrElse[T <: AnyRef : Manifest](default: T, queryNode: StormNode = node) = storm.getOrElse(stormDag, queryNode, default) + /** + * Keep the crazy casts localized in here + */ + private def foldOperations[T, U](producers: List[Producer[Storm, _]]): FlatMapOperation[T, U] = + producers.foldLeft(FlatMapOperation.identity[Any]) { + case (acc, p) => + p match { + case LeftJoinedProducer(_, wrapper) => + val newService = wrapper.store + FlatMapOperation.combine( + acc.asInstanceOf[FlatMapOperation[Any, (Any, Any)]], + newService.asInstanceOf[StoreFactory[Any, Any]]).asInstanceOf[FlatMapOperation[Any, Any]] + case OptionMappedProducer(_, op) => acc.andThen(FlatMapOperation[Any, Any](op.andThen(_.iterator).asInstanceOf[Any => TraversableOnce[Any]])) + case FlatMappedProducer(_, op) => acc.andThen(FlatMapOperation(op).asInstanceOf[FlatMapOperation[Any, Any]]) + case WrittenProducer(_, sinkSupplier) => + acc.andThen(FlatMapOperation.write(() => sinkSupplier.toFn)) + case IdentityKeyedProducer(_) => acc + case MergedProducer(_, _) => acc + case NamedProducer(_, _) => acc + case AlsoProducer(_, _) => acc + case Source(_) => sys.error("Should not schedule a source inside a flat mapper") + case Summer(_, _, _) => sys.error("Should not schedule a Summer inside a flat mapper") + case KeyFlatMappedProducer(_, op) => acc.andThen(FlatMapOperation.keyFlatMap[Any, Any, Any](op).asInstanceOf[FlatMapOperation[Any, Any]]) + } + }.asInstanceOf[FlatMapOperation[T, U]] + + // Boilerplate extracting of the options from the DAG + private val nodeName = stormDag.getNodeName(node) + private val metrics = getOrElse(DEFAULT_FM_STORM_METRICS) + private val anchorTuples = getOrElse(AnchorTuples.default) + logger.info("[{}] Anchoring: {}", nodeName, anchorTuples.anchor) + + private val maxWaiting = getOrElse(DEFAULT_MAX_WAITING_FUTURES) + private val maxWaitTime = getOrElse(DEFAULT_MAX_FUTURE_WAIT_TIME) + logger.info("[{}] maxWaiting: {}", nodeName, maxWaiting.get) + + private val flushFrequency = getOrElse(DEFAULT_FLUSH_FREQUENCY) + logger.info("[{}] maxWaiting: {}", nodeName, flushFrequency.get) + + private val cacheSize = getOrElse(DEFAULT_FM_CACHE) + logger.info("[{}] cacheSize lowerbound: {}", nodeName, cacheSize.lowerBound) + + private val useAsyncCache = getOrElse(DEFAULT_USE_ASYNC_CACHE) + logger.info("[{}] useAsyncCache : {}", nodeName, useAsyncCache.get) + + private val ackOnEntry = getOrElse(DEFAULT_ACK_ON_ENTRY) + logger.info("[{}] ackOnEntry : {}", nodeName, ackOnEntry.get) + + private val maxEmitPerExecute = getOrElse(DEFAULT_MAX_EMIT_PER_EXECUTE) + logger.info("[{}] maxEmitPerExecute : {}", nodeName, maxEmitPerExecute.get) + + private def getCacheBuilder[K, V]: CacheBuilder[K, V] = + if(useAsyncCache.get) { + val softMemoryFlush = getOrElse(DEFAULT_SOFT_MEMORY_FLUSH_PERCENT) + logger.info("[{}] softMemoryFlush : {}", nodeName, softMemoryFlush.get) + + val asyncPoolSize = getOrElse(DEFAULT_ASYNC_POOL_SIZE) + logger.info("[{}] asyncPoolSize : {}", nodeName, asyncPoolSize.get) + + val valueCombinerCrushSize = getOrElse(DEFAULT_VALUE_COMBINER_CACHE_SIZE) + logger.info("[{}] valueCombinerCrushSize : {}", nodeName, valueCombinerCrushSize.get) + MultiTriggerCache.builder[K, V](cacheSize, valueCombinerCrushSize, flushFrequency, + softMemoryFlush, asyncPoolSize) + } else { + SummingQueueCache.builder[K, V](cacheSize, flushFrequency) + } + + private def getFFMBolt[T, K, V](summer: SummerNode[Storm]) = { + type ExecutorInput = (Timestamp, T) + type ExecutorKey = Int + type InnerValue = (Timestamp, V) + type ExecutorValue = Map[(K, BatchID), InnerValue] + val summerProducer = summer.members.collect { case s: Summer[_, _, _] => s }.head.asInstanceOf[Summer[Storm, K, V]] + // When emitting tuples between the Final Flat Map and the summer we encode the timestamp in the value + // The monoid we use in aggregation is timestamp max. + val batcher = summerProducer.store.batcher + implicit val valueMonoid: Semigroup[V] = summerProducer.monoid + + // Query to get the summer paralellism of the summer down stream of us we are emitting to + // to ensure no edge case between what we might see for its parallelism and what it would see/pass to storm. + val summerParalellism = getOrElse(DEFAULT_SUMMER_PARALLELISM, summer) + val summerBatchMultiplier = getOrElse(DEFAULT_SUMMER_BATCH_MULTIPLIER, summer) + + // This option we report its value here, but its not user settable. + val keyValueShards = executor.KeyValueShards(summerParalellism.parHint * summerBatchMultiplier.get) + logger.info("[{}] keyValueShards : {}", nodeName, keyValueShards.get) + + val operation = foldOperations[T, (K, V)](node.members.reverse) + val wrappedOperation = wrapTimeBatchIDKV(operation)(batcher) + BaseBolt( + metrics.metrics, + anchorTuples, + true, + new Fields(AGG_KEY, AGG_VALUE), + ackOnEntry, + new executor.FinalFlatMap( + wrappedOperation, + getCacheBuilder[ExecutorKey, (List[InputState[Tuple]], ExecutorValue)], + maxWaiting, + maxWaitTime, + maxEmitPerExecute, + keyValueShards, + new SingleItemInjection[ExecutorInput], + new KeyValueInjection[ExecutorKey, ExecutorValue] + )(implicitly[Semigroup[InnerValue]]) + ) + } + + def getIntermediateFMBolt[T, U] = { + type ExecutorInput = (Timestamp, T) + type ExecutorOutput = (Timestamp, U) + + val operation = foldOperations[T, U](node.members.reverse) + val wrappedOperation = wrapTime(operation) + + BaseBolt( + metrics.metrics, + anchorTuples, + stormDag.dependantsOf(node).size > 0, + new Fields(VALUE_FIELD), + ackOnEntry, + new executor.IntermediateFlatMap( + wrappedOperation, + maxWaiting, + maxWaitTime, + maxEmitPerExecute, + new SingleItemInjection[ExecutorInput], + new SingleItemInjection[ExecutorOutput] + ) + ) + } + + def apply: BaseBolt[Any, Any] = { + val summerOpt:Option[SummerNode[Storm]] = stormDag.dependantsOf(node).collect{case s: SummerNode[Storm] => s}.headOption + summerOpt match { + case Some(s) => getFFMBolt[Any, Any, Any](s).asInstanceOf[BaseBolt[Any, Any]] + case None => getIntermediateFMBolt[Any, Any].asInstanceOf[BaseBolt[Any, Any]] + } + } +} diff --git a/summingbird-storm/src/main/scala/com/twitter/summingbird/storm/StormPlatform.scala b/summingbird-storm/src/main/scala/com/twitter/summingbird/storm/StormPlatform.scala index d6854701b..35e14c1d1 100644 --- a/summingbird-storm/src/main/scala/com/twitter/summingbird/storm/StormPlatform.scala +++ b/summingbird-storm/src/main/scala/com/twitter/summingbird/storm/StormPlatform.scala @@ -27,7 +27,7 @@ import com.twitter.bijection.{Base64String, Injection} import com.twitter.algebird.{Monoid, Semigroup} import com.twitter.chill.IKryoRegistrar import com.twitter.storehaus.{ReadableStore, WritableStore} -import com.twitter.storehaus.algebra.MergeableStore +import com.twitter.storehaus.algebra.{MergeableStore, Mergeable} import com.twitter.storehaus.algebra.MergeableStore.enrich import com.twitter.summingbird._ import com.twitter.summingbird.viz.VizGraph @@ -37,13 +37,13 @@ import com.twitter.summingbird.storm.option.{AckOnEntry, AnchorTuples} import com.twitter.summingbird.online.{MultiTriggerCache, SummingQueueCache} import com.twitter.summingbird.online.executor.InputState import com.twitter.summingbird.online.option.{IncludeSuccessHandler, MaxWaitingFutures, MaxFutureWaitTime} -import com.twitter.summingbird.util.CacheSize +import com.twitter.summingbird.option.CacheSize import com.twitter.tormenta.spout.Spout import com.twitter.summingbird.planner._ import com.twitter.summingbird.online.executor import com.twitter.summingbird.online.FlatMapOperation import com.twitter.summingbird.storm.planner._ -import com.twitter.util.Future +import com.twitter.util.{Future, Time} import scala.annotation.tailrec import backtype.storm.tuple.Values import org.slf4j.LoggerFactory @@ -58,7 +58,7 @@ sealed trait StormStore[-K, V] { } object MergeableStoreSupplier { - def from[K, V](store: => MergeableStore[(K, BatchID), V])(implicit batcher: Batcher): MergeableStoreSupplier[K, V] = + def from[K, V](store: => Mergeable[(K, BatchID), V])(implicit batcher: Batcher): MergeableStoreSupplier[K, V] = MergeableStoreSupplier(() => store, batcher) def fromOnlineOnly[K, V](store: => MergeableStore[K, V]): MergeableStoreSupplier[K, V] = { @@ -68,7 +68,7 @@ object MergeableStoreSupplier { } -case class MergeableStoreSupplier[K, V](store: () => MergeableStore[(K, BatchID), V], batcher: Batcher) extends StormStore[K, V] +case class MergeableStoreSupplier[K, V](store: () => Mergeable[(K, BatchID), V], batcher: Batcher) extends StormStore[K, V] trait StormService[-K, +V] { def store: StoreFactory[K, V] @@ -99,7 +99,7 @@ object Storm { def onlineOnlyStore[K, V](store: => MergeableStore[K, V]): StormStore[K, V] = MergeableStoreSupplier.fromOnlineOnly(store) - def store[K, V](store: => MergeableStore[(K, BatchID), V])(implicit batcher: Batcher): StormStore[K, V] = + def store[K, V](store: => Mergeable[(K, BatchID), V])(implicit batcher: Batcher): StormStore[K, V] = MergeableStoreSupplier.from(store) def service[K, V](serv: => ReadableStore[K, V]): StormService[K, V] = StoreWrapper(() => serv) @@ -132,7 +132,7 @@ abstract class Storm(options: Map[String, Options], transformConfig: Summingbird private type Prod[T] = Producer[Storm, T] - private def getOrElse[T <: AnyRef : Manifest](dag: Dag[Storm], node: StormNode, default: T): T = { + private[storm] def getOrElse[T <: AnyRef : Manifest](dag: Dag[Storm], node: StormNode, default: T): T = { val producer = node.members.last val namedNodes = dag.producerToPriorityNames(producer) @@ -162,120 +162,21 @@ abstract class Storm(options: Map[String, Options], transformConfig: Summingbird } private def scheduleFlatMapper(stormDag: Dag[Storm], node: StormNode)(implicit topologyBuilder: TopologyBuilder) = { - /** - * Only exists because of the crazy casts we needed. - */ - def foldOperations(producers: List[Producer[Storm, _]]): FlatMapOperation[Any, Any] = { - producers.foldLeft(FlatMapOperation.identity[Any]) { - case (acc, p) => - p match { - case LeftJoinedProducer(_, wrapper) => - val newService = wrapper.store - FlatMapOperation.combine( - acc.asInstanceOf[FlatMapOperation[Any, (Any, Any)]], - newService.asInstanceOf[StoreFactory[Any, Any]]).asInstanceOf[FlatMapOperation[Any, Any]] - case OptionMappedProducer(_, op) => acc.andThen(FlatMapOperation[Any, Any](op.andThen(_.iterator).asInstanceOf[Any => TraversableOnce[Any]])) - case FlatMappedProducer(_, op) => acc.andThen(FlatMapOperation(op).asInstanceOf[FlatMapOperation[Any, Any]]) - case WrittenProducer(_, sinkSupplier) => - acc.andThen(FlatMapOperation.write(() => sinkSupplier.toFn)) - case IdentityKeyedProducer(_) => acc - case MergedProducer(_, _) => acc - case NamedProducer(_, _) => acc - case AlsoProducer(_, _) => acc - case Source(_) => sys.error("Should not schedule a source inside a flat mapper") - case Summer(_, _, _) => sys.error("Should not schedule a Summer inside a flat mapper") - case KeyFlatMappedProducer(_, op) => acc.andThen(FlatMapOperation.keyFlatMap[Any, Any, Any](op).asInstanceOf[FlatMapOperation[Any, Any]]) - } - } - } val nodeName = stormDag.getNodeName(node) - val operation = foldOperations(node.members.reverse) - val metrics = getOrElse(stormDag, node, DEFAULT_FM_STORM_METRICS) - val anchorTuples = getOrElse(stormDag, node, AnchorTuples.default) - logger.info("[{}] Anchoring: {}", nodeName, anchorTuples.anchor) - - val maxWaiting = getOrElse(stormDag, node, DEFAULT_MAX_WAITING_FUTURES) - val maxWaitTime = getOrElse(stormDag, node, DEFAULT_MAX_FUTURE_WAIT_TIME) - logger.info("[{}] maxWaiting: {}", nodeName, maxWaiting.get) - val usePreferLocalDependency = getOrElse(stormDag, node, DEFAULT_FM_PREFER_LOCAL_DEPENDENCY) logger.info("[{}] usePreferLocalDependency: {}", nodeName, usePreferLocalDependency.get) - val flushFrequency = getOrElse(stormDag, node, DEFAULT_FLUSH_FREQUENCY) - logger.info("[{}] maxWaiting: {}", nodeName, flushFrequency.get) - - val cacheSize = getOrElse(stormDag, node, DEFAULT_FM_CACHE) - logger.info("[{}] cacheSize lowerbound: {}", nodeName, cacheSize.lowerBound) - - val summerOpt:Option[SummerNode[Storm]] = stormDag.dependantsOf(node).collect{case s: SummerNode[Storm] => s}.headOption - - val useAsyncCache = getOrElse(stormDag, node, DEFAULT_USE_ASYNC_CACHE) - logger.info("[{}] useAsyncCache : {}", nodeName, useAsyncCache.get) - - val ackOnEntry = getOrElse(stormDag, node, DEFAULT_ACK_ON_ENTRY) - logger.info("[{}] ackOnEntry : {}", nodeName, ackOnEntry.get) - - val bolt = summerOpt match { - case Some(s) => - val summerProducer = s.members.collect { case s: Summer[_, _, _] => s }.head.asInstanceOf[Summer[Storm, _, _]] - val cacheBuilder = if(useAsyncCache.get) { - val softMemoryFlush = getOrElse(stormDag, node, DEFAULT_SOFT_MEMORY_FLUSH_PERCENT) - logger.info("[{}] softMemoryFlush : {}", nodeName, softMemoryFlush.get) - - val asyncPoolSize = getOrElse(stormDag, node, DEFAULT_ASYNC_POOL_SIZE) - logger.info("[{}] asyncPoolSize : {}", nodeName, asyncPoolSize.get) - - val valueCombinerCrushSize = getOrElse(stormDag, node, DEFAULT_VALUE_COMBINER_CACHE_SIZE) - logger.info("[{}] valueCombinerCrushSize : {}", nodeName, valueCombinerCrushSize.get) - - MultiTriggerCache.builder[(Any, BatchID), (List[InputState[Tuple]], Timestamp, Any)](cacheSize, valueCombinerCrushSize, flushFrequency, softMemoryFlush, asyncPoolSize) - } else { - SummingQueueCache.builder[(Any, BatchID), (List[InputState[Tuple]], Timestamp, Any)](cacheSize, flushFrequency) - } - - BaseBolt( - metrics.metrics, - anchorTuples, - true, - new Fields(AGG_KEY, AGG_VALUE), - ackOnEntry, - new executor.FinalFlatMap( - operation.asInstanceOf[FlatMapOperation[Any, (Any, Any)]], - cacheBuilder, - maxWaiting, - maxWaitTime, - new SingleItemInjection[Any], - new KeyValueInjection[(Any, BatchID), Any] - )(summerProducer.monoid.asInstanceOf[Monoid[Any]], summerProducer.store.batcher) - ) - case None => - BaseBolt( - metrics.metrics, - anchorTuples, - stormDag.dependantsOf(node).size > 0, - new Fields(VALUE_FIELD), - ackOnEntry, - new executor.IntermediateFlatMap( - operation, - maxWaiting, - maxWaitTime, - new SingleItemInjection[Any], - new SingleItemInjection[Any] - ) - ) - } + val bolt: BaseBolt[Any, Any] = FlatMapBoltProvider(this, stormDag, node).apply val parallelism = getOrElse(stormDag, node, DEFAULT_FM_PARALLELISM).parHint val declarer = topologyBuilder.setBolt(nodeName, bolt, parallelism).addConfigurations(tickConfig) - val dependenciesNames = stormDag.dependenciesOf(node).collect { case x: StormNode => stormDag.getNodeName(x) } if (usePreferLocalDependency.get) { dependenciesNames.foreach { declarer.localOrShuffleGrouping(_) } } else { dependenciesNames.foreach { declarer.shuffleGrouping(_) } } - } private def scheduleSpout[K](stormDag: Dag[Storm], node: StormNode)(implicit topologyBuilder: TopologyBuilder) = { @@ -304,11 +205,55 @@ abstract class Storm(options: Map[String, Options], transformConfig: Summingbird private def scheduleSummerBolt[K, V](stormDag: Dag[Storm], node: StormNode)(implicit topologyBuilder: TopologyBuilder) = { val summer: Summer[Storm, K, V] = node.members.collect { case c: Summer[Storm, K, V] => c }.head implicit val monoid = summer.monoid + implicit val batcher = summer.store.batcher val nodeName = stormDag.getNodeName(node) + type ExecutorKeyType = (K, BatchID) + type ExecutorValueType = (Timestamp, V) + type ExecutorOutputType = (Timestamp, (K, (Option[V], V))) + val supplier = summer.store match { case MergeableStoreSupplier(contained, _) => contained } + + def wrapMergable(supplier: () => Mergeable[ExecutorKeyType, V]) = + () => { + new Mergeable[ExecutorKeyType, ExecutorValueType] { + val innerMergable: Mergeable[ExecutorKeyType, V] = supplier() + implicit val innerSG = innerMergable.semigroup + + // Since we don't keep a timestamp in the store + // this makes it clear that the 'right' or newest timestamp from the stream + // will always be the timestamp outputted + val semigroup = { + implicit val tsSg = new Semigroup[Timestamp] { + def plus(a: Timestamp, b: Timestamp) = b + override def sumOption(ti: TraversableOnce[Timestamp]) = + if(ti.isEmpty) None + else { + var last: Timestamp = null + ti.foreach { last = _ } + Some(last) + } + } + implicitly[Semigroup[ExecutorValueType]] + } + + override def close(time: Time) = innerMergable.close(time) + + override def multiMerge[K1 <: ExecutorKeyType](kvs: Map[K1, ExecutorValueType]): Map[K1, Future[Option[ExecutorValueType]]] = + innerMergable.multiMerge(kvs.mapValues(_._2)).map { case (k, futOpt) => + (k, futOpt.map { opt => + opt.map { v => + (kvs(k)._1, v) + } + }) + } + } + } + + val wrappedStore = wrapMergable(supplier) + val anchorTuples = getOrElse(stormDag, node, AnchorTuples.default) val metrics = getOrElse(stormDag, node, DEFAULT_SUMMER_STORM_METRICS) val shouldEmit = stormDag.dependantsOf(node).size > 0 @@ -325,6 +270,19 @@ abstract class Storm(options: Map[String, Options], transformConfig: Summingbird val useAsyncCache = getOrElse(stormDag, node, DEFAULT_USE_ASYNC_CACHE) logger.info("[{}] useAsyncCache : {}", nodeName, useAsyncCache.get) + + val maxEmitPerExecute = getOrElse(stormDag, node, DEFAULT_MAX_EMIT_PER_EXECUTE) + logger.info("[{}] maxEmitPerExecute : {}", nodeName, maxEmitPerExecute.get) + + val storeBaseFMOp = { op: (ExecutorKeyType, (Option[ExecutorValueType], ExecutorValueType)) => + val ((k, batchID), (optiVWithTS, (ts, v))) = op + val optiV = optiVWithTS.map(_._2) + List((ts, (k, (optiV, v)))) + } + + val flatmapOp: FlatMapOperation[(ExecutorKeyType, (Option[ExecutorValueType], ExecutorValueType)), ExecutorOutputType] = + FlatMapOperation.apply(storeBaseFMOp) + val cacheBuilder = if(useAsyncCache.get) { val softMemoryFlush = getOrElse(stormDag, node, DEFAULT_SOFT_MEMORY_FLUSH_PERCENT) logger.info("[{}] softMemoryFlush : {}", nodeName, softMemoryFlush.get) @@ -335,9 +293,9 @@ abstract class Storm(options: Map[String, Options], transformConfig: Summingbird val valueCombinerCrushSize = getOrElse(stormDag, node, DEFAULT_VALUE_COMBINER_CACHE_SIZE) logger.info("[{}] valueCombinerCrushSize : {}", nodeName, valueCombinerCrushSize.get) - MultiTriggerCache.builder[(K, BatchID), (List[InputState[Tuple]], Timestamp, V)](cacheSize, valueCombinerCrushSize, flushFrequency, softMemoryFlush, asyncPoolSize) + MultiTriggerCache.builder[ExecutorKeyType, (List[InputState[Tuple]], ExecutorValueType)](cacheSize, valueCombinerCrushSize, flushFrequency, softMemoryFlush, asyncPoolSize) } else { - SummingQueueCache.builder[(K, BatchID), (List[InputState[Tuple]], Timestamp, V)](cacheSize, flushFrequency) + SummingQueueCache.builder[ExecutorKeyType, (List[InputState[Tuple]], ExecutorValueType)](cacheSize, flushFrequency) } val sinkBolt = BaseBolt( @@ -346,16 +304,18 @@ abstract class Storm(options: Map[String, Options], transformConfig: Summingbird shouldEmit, new Fields(VALUE_FIELD), ackOnEntry, - new executor.Summer( - supplier, + new executor.Summer ( + wrappedStore, + flatmapOp, getOrElse(stormDag, node, DEFAULT_ONLINE_SUCCESS_HANDLER), getOrElse(stormDag, node, DEFAULT_ONLINE_EXCEPTION_HANDLER), cacheBuilder, getOrElse(stormDag, node, DEFAULT_MAX_WAITING_FUTURES), getOrElse(stormDag, node, DEFAULT_MAX_FUTURE_WAIT_TIME), + maxEmitPerExecute, getOrElse(stormDag, node, IncludeSuccessHandler.default), - new KeyValueInjection[(K,BatchID), V], - new SingleItemInjection[(K, (Option[V], V))]) + new KeyValueInjection[Int, Map[ExecutorKeyType, ExecutorValueType]], + new SingleItemInjection[ExecutorOutputType]) ) val parallelism = getOrElse(stormDag, node, DEFAULT_SUMMER_PARALLELISM).parHint diff --git a/summingbird-storm/src/main/scala/com/twitter/summingbird/storm/TupleInjections.scala b/summingbird-storm/src/main/scala/com/twitter/summingbird/storm/TupleInjections.scala index 81df65457..65fe6fb24 100644 --- a/summingbird-storm/src/main/scala/com/twitter/summingbird/storm/TupleInjections.scala +++ b/summingbird-storm/src/main/scala/com/twitter/summingbird/storm/TupleInjections.scala @@ -18,37 +18,36 @@ package com.twitter.summingbird.storm import com.twitter.bijection.{Injection, Inversion, AbstractInjection} -import com.twitter.summingbird.batch.Timestamp import java.util.{List => JList, ArrayList => JAList} import scala.util.Try -class SingleItemInjection[T] extends Injection[(Timestamp, T), JList[AnyRef]] { +class SingleItemInjection[T] extends Injection[T, JList[AnyRef]] { - override def apply(t: (Timestamp, T)) = { + override def apply(t: T) = { val list = new JAList[AnyRef](1) - list.add(t) + list.add(t.asInstanceOf[AnyRef]) list } override def invert(vin: JList[AnyRef]) = Inversion.attempt(vin) { v => - v.get(0).asInstanceOf[(Timestamp, T)] + v.get(0).asInstanceOf[T] } } class KeyValueInjection[K, V] - extends Injection[(Timestamp, (K, V)), JList[AnyRef]] { + extends Injection[(K, V), JList[AnyRef]] { - override def apply(item: (Timestamp, (K, V))) = { - val (ts, (key, v)) = item + override def apply(item: (K, V)) = { + val (key, v) = item val list = new JAList[AnyRef](2) list.add(key.asInstanceOf[AnyRef]) - list.add((ts, v)) + list.add(v.asInstanceOf[AnyRef]) list } override def invert(vin: JList[AnyRef]) = Inversion.attempt(vin) { v => val key = v.get(0).asInstanceOf[K] - val (ts, value) = v.get(1).asInstanceOf[(Timestamp, V)] - (ts, (key, value)) + val value = v.get(1).asInstanceOf[V] + (key, value) } } diff --git a/summingbird-storm/src/main/scala/com/twitter/summingbird/storm/option/FlatMapOptions.scala b/summingbird-storm/src/main/scala/com/twitter/summingbird/storm/option/FlatMapOptions.scala index a7f467adc..ac8e96ba0 100644 --- a/summingbird-storm/src/main/scala/com/twitter/summingbird/storm/option/FlatMapOptions.scala +++ b/summingbird-storm/src/main/scala/com/twitter/summingbird/storm/option/FlatMapOptions.scala @@ -60,4 +60,4 @@ class SpoutStormMetrics(val metrics: () => TraversableOnce[StormMetric[IMetric]] case class PreferLocalDependency(get: Boolean) -case class AckOnEntry(get: Boolean) \ No newline at end of file +case class AckOnEntry(get: Boolean) diff --git a/summingbird-storm/src/main/scala/com/twitter/summingbird/storm/option/SummerOptions.scala b/summingbird-storm/src/main/scala/com/twitter/summingbird/storm/option/SummerOptions.scala index bfada9a76..b9ee1b88b 100644 --- a/summingbird-storm/src/main/scala/com/twitter/summingbird/storm/option/SummerOptions.scala +++ b/summingbird-storm/src/main/scala/com/twitter/summingbird/storm/option/SummerOptions.scala @@ -29,7 +29,6 @@ import com.twitter.util.Duration */ - /** * SinkParallelism controls the number of executors storm allocates to * the groupAndSum bolts. Each of these bolt executors is responsible @@ -39,22 +38,6 @@ import com.twitter.util.Duration */ case class SummerParallelism(parHint: Int) -case class OnlineSuccessHandler(handlerFn: Unit => Unit) - -/** - * Kryo serialization problems have been observed with using - * OnlineSuccessHandler. This enables easy disabling of the handler. - * TODO (https://github.com/twitter/summingbird/issues/82): remove - * once we know what the hell is going on with this - */ -case class IncludeSuccessHandler(get: Boolean) - -object IncludeSuccessHandler { - val default = IncludeSuccessHandler(true) -} - -case class OnlineExceptionHandler(handlerFn: PartialFunction[Throwable, Unit]) - /** * See FlatMapOptions.scala for an explanation. */ @@ -63,3 +46,6 @@ object SummerStormMetrics { def unapply(metrics: SummerStormMetrics) = Some(metrics.metrics) } class SummerStormMetrics(val metrics: () => TraversableOnce[StormMetric[_]]) + + +case class SummerBatchMultiplier(get: Int) \ No newline at end of file diff --git a/summingbird-storm/src/main/scala/com/twitter/summingbird/storm/option/package.scala b/summingbird-storm/src/main/scala/com/twitter/summingbird/storm/option/package.scala deleted file mode 100644 index 8fcebfe22..000000000 --- a/summingbird-storm/src/main/scala/com/twitter/summingbird/storm/option/package.scala +++ /dev/null @@ -1,55 +0,0 @@ -/* - Copyright 2013 Twitter, Inc. - - Licensed under the Apache License, Version 2.0 (the "License"); - you may not use this file except in compliance with the License. - You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. - */ - -package com.twitter.summingbird.storm - -import com.twitter.summingbird -import com.twitter.summingbird.online - - -package object option { - @deprecated("Use com.twitter.summingbird.option.CacheSize", "0.2.5") - type CacheSize = summingbird.option.CacheSize - - @deprecated("Use com.twitter.summingbird.option.CacheSize", "0.2.5") - val CacheSize = summingbird.option.CacheSize - - @deprecated("Use com.twitter.summingbird.option.SummerParallelism", "0.3.0") - type SinkParallelism = SummerParallelism - val SinkParallelism = SummerParallelism - - @deprecated("Use com.twitter.summingbird.option.SummerStormMetrics", "0.3.0") - type SinkStormMetrics = SummerStormMetrics - val SinkStormMetrics = SummerStormMetrics - - @deprecated("Use com.twitter.summingbird.online.option.OnlineSuccessHandler", "0.2.6") - type OnlineSuccessHandler = online.option.OnlineSuccessHandler - val OnlineSuccessHandler = online.option.OnlineSuccessHandler - - - @deprecated("Use com.twitter.summingbird.online.option.IncludeSuccessHandler", "0.2.6") - type IncludeSuccessHandler = online.option.IncludeSuccessHandler - val IncludeSuccessHandler = online.option.IncludeSuccessHandler - - @deprecated("Use com.twitter.summingbird.online.option.OnlineExceptionHandler", "0.2.6") - type OnlineExceptionHandler = online.option.OnlineExceptionHandler - val OnlineExceptionHandler = online.option.OnlineExceptionHandler - - @deprecated("Use com.twitter.summingbird.online.option.MaxWaitingFutures", "0.2.6") - type MaxWaitingFutures = online.option.MaxWaitingFutures - val MaxWaitingFutures = online.option.MaxWaitingFutures - -}