From f3b809422c1db8f15271a66af3c284c5a02293e6 Mon Sep 17 00:00:00 2001 From: Tianshuo Deng Date: Fri, 20 Mar 2015 12:48:31 -0700 Subject: [PATCH 01/12] ensure at least one batch before merging --- .../scalding/batch/BatchedStore.scala | 32 +++++++++++-------- 1 file changed, 18 insertions(+), 14 deletions(-) diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedStore.scala b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedStore.scala index 7ca6eb687..fccce4338 100644 --- a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedStore.scala +++ b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedStore.scala @@ -143,9 +143,8 @@ trait BatchedStore[K, V] extends scalding.Store[K, V] { self => // get the batches read from the readTimespan val batchIntr = batcher.batchesCoveredBy(readTimespan) - logger.info("readTimeSpan {}", readTimespan) val batches = BatchID.toIterable(batchIntr).toList - val finalBatch = batches.last // batches won't be empty. + val finalBatch = batches.last // batches won't be empty, ensured by atLeastOneBatch method val filteredBatches = select(batches).sorted assert(filteredBatches.contains(finalBatch), "select must not remove the final batch.") @@ -284,8 +283,6 @@ trait BatchedStore[K, V] extends scalding.Store[K, V] { self => // Now get the first timestamp that we need input data for. firstDeltaTimestamp = lastTimeWrittenToStore.next - firstDeltaBatch = lastBatch.next - // Get the requested timeSpan. tsMode <- getState[FactoryInput] (timeSpan, mode) = tsMode @@ -295,7 +292,7 @@ trait BatchedStore[K, V] extends scalding.Store[K, V] { self => // Get the total time we want to cover. If the lower bound of the requested timeSpan // is not the firstDeltaTimestamp, adjust it to that. - deltaTimes: Interval[Timestamp] = setLower(InclusiveLower(firstDeltaTimestamp), timeSpan) + deltaTimes = setLower(InclusiveLower(firstDeltaTimestamp), timeSpan) // Try to read the range covering the time we want; get the time we can completely // cover and the data from input in that range. @@ -303,17 +300,11 @@ trait BatchedStore[K, V] extends scalding.Store[K, V] { self => (readDeltaTimestamps, readFlow) = readTimeFlow - firstDeltaBatchInterval: Interval[Timestamp] = batcher.toInterval(firstDeltaBatch) // Make sure that the time we can read includes the time just after the last // snapshot. We can't roll the store forward without this. - _ <- fromEither[FactoryInput] { - logger.info("firstBatchInterval is {}", firstDeltaBatchInterval) - if (readDeltaTimestamps.intersect(firstDeltaBatchInterval) == firstDeltaBatchInterval) //readDeltaTimestamps should include the firstDeltaBatchInterval - Right(()) - else - Left(List("Cannot load initial timestamp interval " + firstDeltaBatchInterval.toString + " of deltas " + - " at " + this.toString + " only " + readDeltaTimestamps.toString)) - } + _ <- fromEither[FactoryInput](if (readDeltaTimestamps.contains(firstDeltaTimestamp)) Right(()) else + Left(List("Cannot load initial timestamp " + firstDeltaTimestamp.toString + " of deltas " + + " at " + this.toString + " only " + readDeltaTimestamps.toString))) // Record the timespan we actually read. _ <- putState((readDeltaTimestamps, mode)) @@ -339,6 +330,18 @@ trait BatchedStore[K, V] extends scalding.Store[K, V] { self => deltaFlow2Pipe) } + /** + * This is for ensuring there is at least one batch coverd by readTimespan. This is + * required by mergeBatched + */ + private def atLeastOneBatch(readTimespan: Interval[Timestamp]): Either[List[FailureReason], Unit] = { + if (batcher.batchesCoveredBy(readTimespan) == Empty()) { + Left(List("readTimespan is not convering at least one batch: " + readTimespan.toString)) + } else { + Right() + } + } + /** * 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 @@ -360,6 +363,7 @@ trait BatchedStore[K, V] extends scalding.Store[K, V] { self => // get the actual timespan read by readAfterLastBatch tsModeRead <- getState[FactoryInput] (tsRead, _) = tsModeRead + _ <- fromEither[FactoryInput](atLeastOneBatch(tsRead)) /** * Once we have read the last snapshot and the available batched blocks of delta, just merge From cfebfa391017fa66f204664e46de179d4c1243e6 Mon Sep 17 00:00:00 2001 From: Tianshuo Deng Date: Mon, 23 Mar 2015 10:37:04 -0700 Subject: [PATCH 02/12] try to add some tests --- .../com/twitter/summingbird/scalding/TestStore.scala | 2 +- .../com/twitter/summingbird/scalding/ScaldingLaws.scala | 8 ++------ 2 files changed, 3 insertions(+), 7 deletions(-) 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 index 44f8de226..d3d8eb9c3 100644 --- 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 @@ -40,7 +40,7 @@ object TestStore { } } -class TestStore[K, V](store: String, inBatcher: Batcher, initBatch: BatchID, initStore: Iterable[(K, V)], lastBatch: BatchID, override val pruning: PrunedSpace[(K, V)])(implicit ord: Ordering[K], tset: TupleSetter[(K, V)], tconv: TupleConverter[(K, V)]) +class TestStore[K, V](store: String, inBatcher: Batcher, val initBatch: BatchID, initStore: Iterable[(K, V)], lastBatch: BatchID, override val pruning: PrunedSpace[(K, V)])(implicit ord: Ordering[K], tset: TupleSetter[(K, V)], tconv: TupleConverter[(K, V)]) extends batch.BatchedStore[K, V] { var writtenBatches = Set[BatchID](initBatch) 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 index ddb752e96..64ce50174 100644 --- 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 @@ -16,7 +16,7 @@ package com.twitter.summingbird.scalding -import com.twitter.algebird.{ MapAlgebra, Monoid, Group, Interval, Last } +import com.twitter.algebird.{ MapAlgebra, Monoid, Group, Interval, Last, Intersection, InclusiveLower, ExclusiveUpper, Empty } import com.twitter.algebird.monad._ import com.twitter.summingbird.{ Producer, TimeExtractor, TestGraphs } import com.twitter.summingbird.batch._ @@ -42,7 +42,7 @@ import scala.util.{ Try => ScalaTry } import cascading.scheme.local.{ TextDelimited => CLTextDelimited } import cascading.tuple.{ Tuple, Fields, TupleEntry } -import cascading.flow.Flow +import cascading.flow.{ Flow, FlowDef } import cascading.stats.FlowStats import cascading.tap.Tap import cascading.scheme.NullScheme @@ -52,10 +52,6 @@ import org.apache.hadoop.mapred.OutputCollector import org.specs2.mutable._ -/** - * Tests for Summingbird's Scalding planner. - */ - object ScaldingLaws extends Specification { import MapAlgebra.sparseEquiv From 164d6ef2a8888a5e5f6fc728e441ebc5e0e1b63c Mon Sep 17 00:00:00 2001 From: Tianshuo Deng Date: Mon, 23 Mar 2015 10:39:03 -0700 Subject: [PATCH 03/12] add a test --- .../scalding/BatchedStoreProperties.scala | 124 ++++++++++++++++++ 1 file changed, 124 insertions(+) create mode 100644 summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/BatchedStoreProperties.scala diff --git a/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/BatchedStoreProperties.scala b/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/BatchedStoreProperties.scala new file mode 100644 index 000000000..ca65cb37a --- /dev/null +++ b/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/BatchedStoreProperties.scala @@ -0,0 +1,124 @@ +/* + 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 cascading.flow.{ Flow, FlowDef } + +import com.twitter.algebird._ +import com.twitter.algebird.monad._ +import com.twitter.summingbird.batch._ +import com.twitter.scalding.{ Source => ScaldingSource, Test => TestMode, _ } + +import org.scalacheck._ +import org.scalacheck.Prop._ +import org.scalacheck.Properties + +/** + * Tests for Summingbird's Scalding planner. + */ + +object BatchedStoreProperties extends Properties("BatchedStore's Properties") { + + implicit def intervalArb[T: Arbitrary: Ordering]: Arbitrary[Interval[T]] = + Arbitrary { + for { + l <- Arbitrary.arbitrary[T] + u <- Arbitrary.arbitrary[T] + } yield Interval.leftClosedRightOpen(l, u) + } + + implicit val arbTimestamp: Arbitrary[Timestamp] = Arbitrary { + // a relevant 200 or so year range + Gen.choose(1L, 100000L) + .map { Timestamp(_) } + } + + def sample[T: Arbitrary]: T = Arbitrary.arbitrary[T].sample.get + + implicit def arbitraryPipeFactory: Arbitrary[PipeFactory[(Int, Int)]] = { + Arbitrary { + Gen.const { + StateWithError[(Interval[Timestamp], Mode), List[FailureReason], FlowToPipe[(Int, Int)]] { + (timeMode: (Interval[Timestamp], Mode)) => + { + val (time: Interval[Timestamp], mode: Mode) = timeMode + val a: FlowToPipe[(Int, Int)] = Reader { (fdM: (FlowDef, Mode)) => TypedPipe.from[(Timestamp, (Int, Int))](Seq((Timestamp(10), (2, 3)))) } + Right((timeMode, a)) + } + } + } + } + } + + implicit def timeExtractor[T <: (Long, _)] = TestUtil.simpleTimeExtractor[T] + + implicit def arbitraryInputWithTimeStampAndBatcher: Arbitrary[(List[(Long, Int)], Batcher, TestStore[Int, Int])] = Arbitrary { + Arbitrary.arbitrary[List[Int]] + .map { in => in.zipWithIndex.map { case (item: Int, time: Int) => (time.toLong, item) } } + .map { in => + val batcher = TestUtil.randomBatcher(in) + val lastTimeStamp = in.size + val testStore = TestStore[Int, Int]("test", batcher, sample[Map[Int, Int]], lastTimeStamp) + (in, batcher, testStore) + } + } + + implicit def arbitraryLocalMode: Arbitrary[Mode] = Arbitrary { Gen.const(Local(true)) } + + property("readAfterLastBatch should return interval starting from the last batch written") = { + forAll { + ( + a: PipeFactory[(Int, Int)], + interval: Interval[Timestamp], + inputWithTimeStampAndBatcherAndStore: (List[(Long, Int)], Batcher, TestStore[Int, Int]), + mode: Mode + ) => + // println(s"requested interval is ${interval}") + + val (inputWithTimeStamp, batcher, testStore) = inputWithTimeStampAndBatcherAndStore + + val result: PlannerOutput[(BatchID, FlowProducer[TypedPipe[(Int, Int)]], FlowToPipe[(Int, Int)])] = testStore.readAfterLastBatch(a) + + result((interval, mode)) match { + case Right(((readInterval: Intersection[InclusiveLower, ExclusiveUpper, Timestamp], _), _)) => { + println(s"readInterval is ${readInterval}, intersection is ${interval.intersect(readInterval)}") + //readInterval should start from the last written interval + val start: InclusiveLower[Timestamp] = InclusiveLower(batcher.earliestTimeOf(testStore.initBatch.next)) + readInterval.lower == start + } + case Left(_) => interval == Empty() + } + } + } + + property("should merge Properly") = { + forAll { + ( + a: PipeFactory[(Int, Int)], + interval: Interval[Timestamp], + inputWithTimeStampAndBatcherAndStore: (List[(Long, Int)], Batcher, TestStore[Int, Int]), + mode: Mode + ) => + //println(s"requested interval is ${interval}") + + val (inputWithTimeStamp, batcher, testStore) = inputWithTimeStampAndBatcherAndStore + val s: Semigroup[Int] = implicitly[Semigroup[Int]] + println(testStore.merge(a, s, com.twitter.summingbird.option.Commutative, 10)) + 1 == 1 + } + } +} From 9290d219d8895b0d104491c2f40211455df62b74 Mon Sep 17 00:00:00 2001 From: Tianshuo Deng Date: Mon, 23 Mar 2015 10:53:36 -0700 Subject: [PATCH 04/12] minor fixes --- .../twitter/summingbird/scalding/BatchedStoreProperties.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/BatchedStoreProperties.scala b/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/BatchedStoreProperties.scala index ca65cb37a..1f1571cc9 100644 --- a/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/BatchedStoreProperties.scala +++ b/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/BatchedStoreProperties.scala @@ -42,7 +42,6 @@ object BatchedStoreProperties extends Properties("BatchedStore's Properties") { } implicit val arbTimestamp: Arbitrary[Timestamp] = Arbitrary { - // a relevant 200 or so year range Gen.choose(1L, 100000L) .map { Timestamp(_) } } @@ -64,7 +63,7 @@ object BatchedStoreProperties extends Properties("BatchedStore's Properties") { } } - implicit def timeExtractor[T <: (Long, _)] = TestUtil.simpleTimeExtractor[T] + implicit def timeExtractor[T <: (Long, Any)] = TestUtil.simpleTimeExtractor[T] implicit def arbitraryInputWithTimeStampAndBatcher: Arbitrary[(List[(Long, Int)], Batcher, TestStore[Int, Int])] = Arbitrary { Arbitrary.arbitrary[List[Int]] From 16b538ae4ead155086b0f00f8835463f7702dc1c Mon Sep 17 00:00:00 2001 From: Tianshuo Deng Date: Mon, 23 Mar 2015 11:10:52 -0700 Subject: [PATCH 05/12] atLeastOneBatch returns PipeFactory --- .../summingbird/scalding/batch/BatchedStore.scala | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedStore.scala b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedStore.scala index fccce4338..e36b93164 100644 --- a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedStore.scala +++ b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedStore.scala @@ -334,11 +334,13 @@ trait BatchedStore[K, V] extends scalding.Store[K, V] { self => * This is for ensuring there is at least one batch coverd by readTimespan. This is * required by mergeBatched */ - private def atLeastOneBatch(readTimespan: Interval[Timestamp]): Either[List[FailureReason], Unit] = { - if (batcher.batchesCoveredBy(readTimespan) == Empty()) { - Left(List("readTimespan is not convering at least one batch: " + readTimespan.toString)) - } else { - Right() + private def atLeastOneBatch(readTimespan: Interval[Timestamp]) = { + fromEither[FactoryInput] { + if (batcher.batchesCoveredBy(readTimespan) == Empty()) { + Left(List("readTimespan is not convering at least one batch: " + readTimespan.toString)) + } else { + Right() + } } } @@ -363,7 +365,7 @@ trait BatchedStore[K, V] extends scalding.Store[K, V] { self => // get the actual timespan read by readAfterLastBatch tsModeRead <- getState[FactoryInput] (tsRead, _) = tsModeRead - _ <- fromEither[FactoryInput](atLeastOneBatch(tsRead)) + _ <- atLeastOneBatch(tsRead) /** * Once we have read the last snapshot and the available batched blocks of delta, just merge From c935dd8a352b758534fb6d98cdf213dcd63fb145 Mon Sep 17 00:00:00 2001 From: Tianshuo Deng Date: Mon, 23 Mar 2015 16:53:46 -0700 Subject: [PATCH 06/12] add unit tests --- .../scalding/BatchedStoreProperties.scala | 88 +++++++++++++------ .../scalding/batch/BatchedStore.scala | 2 +- 2 files changed, 60 insertions(+), 30 deletions(-) diff --git a/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/BatchedStoreProperties.scala b/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/BatchedStoreProperties.scala index 1f1571cc9..0977c0d96 100644 --- a/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/BatchedStoreProperties.scala +++ b/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/BatchedStoreProperties.scala @@ -21,6 +21,7 @@ import cascading.flow.{ Flow, FlowDef } import com.twitter.algebird._ import com.twitter.algebird.monad._ import com.twitter.summingbird.batch._ +import com.twitter.summingbird.option.Commutative import com.twitter.scalding.{ Source => ScaldingSource, Test => TestMode, _ } import org.scalacheck._ @@ -28,17 +29,17 @@ import org.scalacheck.Prop._ import org.scalacheck.Properties /** - * Tests for Summingbird's Scalding planner. + * Tests for BatchedStore */ - object BatchedStoreProperties extends Properties("BatchedStore's Properties") { - implicit def intervalArb[T: Arbitrary: Ordering]: Arbitrary[Interval[T]] = + implicit def intersectionArb[T: Arbitrary: Ordering]: Arbitrary[Intersection[InclusiveLower, ExclusiveUpper, T]] = Arbitrary { for { l <- Arbitrary.arbitrary[T] u <- Arbitrary.arbitrary[T] - } yield Interval.leftClosedRightOpen(l, u) + if implicitly[Ordering[T]].lt(l, u) + } yield Intersection(InclusiveLower(l), ExclusiveUpper(u)) } implicit val arbTimestamp: Arbitrary[Timestamp] = Arbitrary { @@ -48,7 +49,7 @@ object BatchedStoreProperties extends Properties("BatchedStore's Properties") { def sample[T: Arbitrary]: T = Arbitrary.arbitrary[T].sample.get - implicit def arbitraryPipeFactory: Arbitrary[PipeFactory[(Int, Int)]] = { + implicit val arbitraryPipeFactory: Arbitrary[PipeFactory[(Int, Int)]] = { Arbitrary { Gen.const { StateWithError[(Interval[Timestamp], Mode), List[FailureReason], FlowToPipe[(Int, Int)]] { @@ -65,11 +66,11 @@ object BatchedStoreProperties extends Properties("BatchedStore's Properties") { implicit def timeExtractor[T <: (Long, Any)] = TestUtil.simpleTimeExtractor[T] - implicit def arbitraryInputWithTimeStampAndBatcher: Arbitrary[(List[(Long, Int)], Batcher, TestStore[Int, Int])] = Arbitrary { + implicit val arbitraryInputWithTimeStampAndBatcher: Arbitrary[(List[(Long, Int)], Batcher, TestStore[Int, Int])] = Arbitrary { Arbitrary.arbitrary[List[Int]] .map { in => in.zipWithIndex.map { case (item: Int, time: Int) => (time.toLong, item) } } .map { in => - val batcher = TestUtil.randomBatcher(in) + val batcher: Batcher = TestUtil.randomBatcher(in) val lastTimeStamp = in.size val testStore = TestStore[Int, Int]("test", batcher, sample[Map[Int, Int]], lastTimeStamp) (in, batcher, testStore) @@ -80,22 +81,16 @@ object BatchedStoreProperties extends Properties("BatchedStore's Properties") { property("readAfterLastBatch should return interval starting from the last batch written") = { forAll { - ( - a: PipeFactory[(Int, Int)], - interval: Interval[Timestamp], + (diskPipeFactory: PipeFactory[(Int, Int)], + interval: Intersection[InclusiveLower, ExclusiveUpper, Timestamp], inputWithTimeStampAndBatcherAndStore: (List[(Long, Int)], Batcher, TestStore[Int, Int]), - mode: Mode - ) => - // println(s"requested interval is ${interval}") - + mode: Mode) => val (inputWithTimeStamp, batcher, testStore) = inputWithTimeStampAndBatcherAndStore + val result = testStore.readAfterLastBatch(diskPipeFactory)((interval, mode)) - val result: PlannerOutput[(BatchID, FlowProducer[TypedPipe[(Int, Int)]], FlowToPipe[(Int, Int)])] = testStore.readAfterLastBatch(a) - - result((interval, mode)) match { + result match { case Right(((readInterval: Intersection[InclusiveLower, ExclusiveUpper, Timestamp], _), _)) => { - println(s"readInterval is ${readInterval}, intersection is ${interval.intersect(readInterval)}") - //readInterval should start from the last written interval + //readInterval should start from the last written interval in the store val start: InclusiveLower[Timestamp] = InclusiveLower(batcher.earliestTimeOf(testStore.initBatch.next)) readInterval.lower == start } @@ -104,20 +99,55 @@ object BatchedStoreProperties extends Properties("BatchedStore's Properties") { } } - property("should merge Properly") = { + property("the end of merged interval is never extended") = { forAll { - ( - a: PipeFactory[(Int, Int)], - interval: Interval[Timestamp], + (diskPipeFactory: PipeFactory[(Int, Int)], + interval: Intersection[InclusiveLower, ExclusiveUpper, Timestamp], inputWithTimeStampAndBatcherAndStore: (List[(Long, Int)], Batcher, TestStore[Int, Int]), - mode: Mode - ) => - //println(s"requested interval is ${interval}") + mode: Mode) => + val (inputWithTimeStamp, batcher, testStore) = inputWithTimeStampAndBatcherAndStore + val mergeResult = testStore.merge(diskPipeFactory, implicitly[Semigroup[Int]], com.twitter.summingbird.option.Commutative, 10)((interval, mode)) + mergeResult.isRight ==> { + val Right(((readInterval: Intersection[InclusiveLower, ExclusiveUpper, Timestamp], _), _)) = mergeResult + val requestedEndingTimestamp: Timestamp = interval.upper.upper + val readIntervalEndingTimestamp: Timestamp = readInterval.upper.upper + implicitly[Ordering[Timestamp]].lteq(readIntervalEndingTimestamp, requestedEndingTimestamp) + } + } + } + property("should not merge if the time interval on disk(from diskPipeFactory) is smaller than one batch") = { + forAll { + (interval: Intersection[InclusiveLower, ExclusiveUpper, Timestamp], + inputWithTimeStampAndBatcherAndStore: (List[(Long, Int)], Batcher, TestStore[Int, Int]), + mode: Mode) => val (inputWithTimeStamp, batcher, testStore) = inputWithTimeStampAndBatcherAndStore - val s: Semigroup[Int] = implicitly[Semigroup[Int]] - println(testStore.merge(a, s, com.twitter.summingbird.option.Commutative, 10)) - 1 == 1 + val nextBatchEnding = batcher.latestTimeOf(testStore.initBatch.next) + + //this diskPipeFactory returns a timeinterval that ends before the ending of next batch, meaning there is not enough data for a new batch + val diskPipeFactory = StateWithError[(Interval[Timestamp], Mode), List[FailureReason], FlowToPipe[(Int, Int)]] { + (timeMode: (Interval[Timestamp], Mode)) => + { + val (time: Interval[Timestamp], mode: Mode) = timeMode + val Intersection(InclusiveLower(startRequestedTime), ExclusiveUpper(_)) = time + + //shrink the endTime so it does not cover a whole batch + val onDiskEndTime: Long = Gen.choose(startRequestedTime.milliSinceEpoch, nextBatchEnding.milliSinceEpoch).sample.get + + val readTime: Interval[Timestamp] = if (startRequestedTime == nextBatchEnding) + Empty() + else + Intersection(InclusiveLower(startRequestedTime), ExclusiveUpper(nextBatchEnding)) + + val flowToPipe: FlowToPipe[(Int, Int)] = Reader { (fdM: (FlowDef, Mode)) => TypedPipe.from[(Timestamp, (Int, Int))](Seq((Timestamp(10), (2, 3)))) } + Right(((readTime, mode), flowToPipe)) + } + } + + val mergeResult = testStore.merge(diskPipeFactory, implicitly[Semigroup[Int]], Commutative, 10)((interval, mode)) + + mergeResult.left.get.mkString.contains("readTimespan is not convering at least one batch").label("fail with right reason") + mergeResult.isInstanceOf[Left[_, _]].label("returns Left when data is smaller than one batch") } } } diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedStore.scala b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedStore.scala index e36b93164..de581634e 100644 --- a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedStore.scala +++ b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedStore.scala @@ -151,7 +151,7 @@ trait BatchedStore[K, V] extends scalding.Store[K, V] { self => import IteratorSums._ // get the groupedSum, partials function - logger.info("Previous written batch: {}, computing: {}", inBatch.asInstanceOf[Any], 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))) } From a86ec31fd6b01a0ec1ad0877f2abc0af9f5bdcd2 Mon Sep 17 00:00:00 2001 From: Tianshuo Deng Date: Mon, 23 Mar 2015 17:03:31 -0700 Subject: [PATCH 07/12] format --- .../twitter/summingbird/scalding/batch/BatchedStore.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedStore.scala b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedStore.scala index de581634e..45c893bb5 100644 --- a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedStore.scala +++ b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedStore.scala @@ -151,7 +151,7 @@ trait BatchedStore[K, V] extends scalding.Store[K, V] { self => import IteratorSums._ // get the groupedSum, partials function - // logger.info("Previous written batch: {}, computing: {}", inBatch.asInstanceOf[Any], 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))) } @@ -292,7 +292,7 @@ trait BatchedStore[K, V] extends scalding.Store[K, V] { self => // Get the total time we want to cover. If the lower bound of the requested timeSpan // is not the firstDeltaTimestamp, adjust it to that. - deltaTimes = setLower(InclusiveLower(firstDeltaTimestamp), timeSpan) + deltaTimes: Interval[Timestamp] = setLower(InclusiveLower(firstDeltaTimestamp), timeSpan) // Try to read the range covering the time we want; get the time we can completely // cover and the data from input in that range. @@ -302,7 +302,9 @@ trait BatchedStore[K, V] extends scalding.Store[K, V] { self => // Make sure that the time we can read includes the time just after the last // snapshot. We can't roll the store forward without this. - _ <- fromEither[FactoryInput](if (readDeltaTimestamps.contains(firstDeltaTimestamp)) Right(()) else + _ <- fromEither[FactoryInput](if (readDeltaTimestamps.contains(firstDeltaTimestamp)) + Right(()) + else Left(List("Cannot load initial timestamp " + firstDeltaTimestamp.toString + " of deltas " + " at " + this.toString + " only " + readDeltaTimestamps.toString))) From 870dea170153a2d2d7d97aad318732491c371ec9 Mon Sep 17 00:00:00 2001 From: Tianshuo Deng Date: Mon, 23 Mar 2015 17:31:12 -0700 Subject: [PATCH 08/12] fix --- .../scalding/BatchedStoreProperties.scala | 29 +++++++++---------- .../summingbird/scalding/ScaldingLaws.scala | 8 +++-- .../scalding/batch/BatchedStore.scala | 2 +- 3 files changed, 21 insertions(+), 18 deletions(-) diff --git a/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/BatchedStoreProperties.scala b/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/BatchedStoreProperties.scala index 0977c0d96..18343d8b8 100644 --- a/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/BatchedStoreProperties.scala +++ b/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/BatchedStoreProperties.scala @@ -47,8 +47,6 @@ object BatchedStoreProperties extends Properties("BatchedStore's Properties") { .map { Timestamp(_) } } - def sample[T: Arbitrary]: T = Arbitrary.arbitrary[T].sample.get - implicit val arbitraryPipeFactory: Arbitrary[PipeFactory[(Int, Int)]] = { Arbitrary { Gen.const { @@ -67,14 +65,14 @@ object BatchedStoreProperties extends Properties("BatchedStore's Properties") { implicit def timeExtractor[T <: (Long, Any)] = TestUtil.simpleTimeExtractor[T] implicit val arbitraryInputWithTimeStampAndBatcher: Arbitrary[(List[(Long, Int)], Batcher, TestStore[Int, Int])] = Arbitrary { - Arbitrary.arbitrary[List[Int]] - .map { in => in.zipWithIndex.map { case (item: Int, time: Int) => (time.toLong, item) } } - .map { in => - val batcher: Batcher = TestUtil.randomBatcher(in) - val lastTimeStamp = in.size - val testStore = TestStore[Int, Int]("test", batcher, sample[Map[Int, Int]], lastTimeStamp) - (in, batcher, testStore) - } + for { + arbInt <- Arbitrary.arbitrary[List[Int]] + in = arbInt.zipWithIndex.map { case (item: Int, time: Int) => (time.toLong, item) } + arbMap <- Arbitrary.arbitrary[Map[Int, Int]] + batcher = TestUtil.randomBatcher(in) + lastTimeStamp = in.size + testStore = TestStore[Int, Int]("test", batcher, arbMap, lastTimeStamp) + } yield (in, batcher, testStore) } implicit def arbitraryLocalMode: Arbitrary[Mode] = Arbitrary { Gen.const(Local(true)) } @@ -89,11 +87,12 @@ object BatchedStoreProperties extends Properties("BatchedStore's Properties") { val result = testStore.readAfterLastBatch(diskPipeFactory)((interval, mode)) result match { - case Right(((readInterval: Intersection[InclusiveLower, ExclusiveUpper, Timestamp], _), _)) => { + case Right(((Intersection(InclusiveLower(readIntervalLower), ExclusiveUpper(_)), _), _)) => { //readInterval should start from the last written interval in the store - val start: InclusiveLower[Timestamp] = InclusiveLower(batcher.earliestTimeOf(testStore.initBatch.next)) - readInterval.lower == start + val start: Timestamp = batcher.earliestTimeOf(testStore.initBatch.next) + implicitly[Ordering[Timestamp]].equiv(readIntervalLower, start) } + case Right(_) => false case Left(_) => interval == Empty() } } @@ -108,9 +107,9 @@ object BatchedStoreProperties extends Properties("BatchedStore's Properties") { val (inputWithTimeStamp, batcher, testStore) = inputWithTimeStampAndBatcherAndStore val mergeResult = testStore.merge(diskPipeFactory, implicitly[Semigroup[Int]], com.twitter.summingbird.option.Commutative, 10)((interval, mode)) mergeResult.isRight ==> { - val Right(((readInterval: Intersection[InclusiveLower, ExclusiveUpper, Timestamp], _), _)) = mergeResult + val Right(((Intersection(InclusiveLower(_), ExclusiveUpper(readIntervalUpper)), _), _)) = mergeResult val requestedEndingTimestamp: Timestamp = interval.upper.upper - val readIntervalEndingTimestamp: Timestamp = readInterval.upper.upper + val readIntervalEndingTimestamp: Timestamp = readIntervalUpper implicitly[Ordering[Timestamp]].lteq(readIntervalEndingTimestamp, requestedEndingTimestamp) } } 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 index 64ce50174..ddb752e96 100644 --- 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 @@ -16,7 +16,7 @@ package com.twitter.summingbird.scalding -import com.twitter.algebird.{ MapAlgebra, Monoid, Group, Interval, Last, Intersection, InclusiveLower, ExclusiveUpper, Empty } +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._ @@ -42,7 +42,7 @@ import scala.util.{ Try => ScalaTry } import cascading.scheme.local.{ TextDelimited => CLTextDelimited } import cascading.tuple.{ Tuple, Fields, TupleEntry } -import cascading.flow.{ Flow, FlowDef } +import cascading.flow.Flow import cascading.stats.FlowStats import cascading.tap.Tap import cascading.scheme.NullScheme @@ -52,6 +52,10 @@ import org.apache.hadoop.mapred.OutputCollector import org.specs2.mutable._ +/** + * Tests for Summingbird's Scalding planner. + */ + object ScaldingLaws extends Specification { import MapAlgebra.sparseEquiv diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedStore.scala b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedStore.scala index 45c893bb5..a4486a636 100644 --- a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedStore.scala +++ b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedStore.scala @@ -151,7 +151,7 @@ trait BatchedStore[K, V] extends scalding.Store[K, V] { self => import IteratorSums._ // get the groupedSum, partials function - logger.info("Previous written batch: {}, computing: {}", inBatch.asInstanceOf[Any], 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))) } From 8b9d97d286bbc4a4d97d7a4fe6a8835d7da867a6 Mon Sep 17 00:00:00 2001 From: Tianshuo Deng Date: Mon, 23 Mar 2015 17:35:24 -0700 Subject: [PATCH 09/12] add test for testing upper bound of readAfterLastBatch --- .../scalding/BatchedStoreProperties.scala | 20 +++++++++++++++++++ .../scalding/batch/BatchedStore.scala | 5 ++--- 2 files changed, 22 insertions(+), 3 deletions(-) diff --git a/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/BatchedStoreProperties.scala b/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/BatchedStoreProperties.scala index 18343d8b8..258bf105d 100644 --- a/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/BatchedStoreProperties.scala +++ b/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/BatchedStoreProperties.scala @@ -98,6 +98,26 @@ object BatchedStoreProperties extends Properties("BatchedStore's Properties") { } } + property("readAfterLastBatch should not extend the end of interval requested") = { + forAll { + (diskPipeFactory: PipeFactory[(Int, Int)], + interval: Intersection[InclusiveLower, ExclusiveUpper, Timestamp], + inputWithTimeStampAndBatcherAndStore: (List[(Long, Int)], Batcher, TestStore[Int, Int]), + mode: Mode) => + val (inputWithTimeStamp, batcher, testStore) = inputWithTimeStampAndBatcherAndStore + val result = testStore.readAfterLastBatch(diskPipeFactory)((interval, mode)) + + result match { + case Right(((Intersection(InclusiveLower(_), ExclusiveUpper(readIntervalUpper)), _), _)) => { + //readInterval should start from the last written interval in the store + implicitly[Ordering[Timestamp]].lteq(readIntervalUpper, interval.upper.upper) + } + case Right(_) => false + case Left(_) => interval == Empty() + } + } + } + property("the end of merged interval is never extended") = { forAll { (diskPipeFactory: PipeFactory[(Int, Int)], diff --git a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedStore.scala b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedStore.scala index a4486a636..2fb6ff2e4 100644 --- a/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedStore.scala +++ b/summingbird-scalding/src/main/scala/com/twitter/summingbird/scalding/batch/BatchedStore.scala @@ -151,7 +151,7 @@ trait BatchedStore[K, V] extends scalding.Store[K, V] { self => import IteratorSums._ // get the groupedSum, partials function - // logger.info("Previous written batch: {}, computing: {}", inBatch.asInstanceOf[Any], batches) + logger.debug("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))) } @@ -336,7 +336,7 @@ trait BatchedStore[K, V] extends scalding.Store[K, V] { self => * This is for ensuring there is at least one batch coverd by readTimespan. This is * required by mergeBatched */ - private def atLeastOneBatch(readTimespan: Interval[Timestamp]) = { + private def atLeastOneBatch(readTimespan: Interval[Timestamp]) = fromEither[FactoryInput] { if (batcher.batchesCoveredBy(readTimespan) == Empty()) { Left(List("readTimespan is not convering at least one batch: " + readTimespan.toString)) @@ -344,7 +344,6 @@ trait BatchedStore[K, V] extends scalding.Store[K, V] { self => Right() } } - } /** * instances of this trait MAY NOT change the logic here. This always follows the rule From 20d9be5986e76db4b490fb636eb723de1132cb0a Mon Sep 17 00:00:00 2001 From: Tianshuo Deng Date: Mon, 23 Mar 2015 18:22:23 -0700 Subject: [PATCH 10/12] fix tests for testing input data is smaller than one batch, it requires the batcher to be at least 2ms --- .../scalding/BatchedStoreProperties.scala | 59 ++++++++++++------- 1 file changed, 37 insertions(+), 22 deletions(-) diff --git a/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/BatchedStoreProperties.scala b/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/BatchedStoreProperties.scala index 258bf105d..a24c9126a 100644 --- a/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/BatchedStoreProperties.scala +++ b/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/BatchedStoreProperties.scala @@ -136,37 +136,52 @@ object BatchedStoreProperties extends Properties("BatchedStore's Properties") { } property("should not merge if the time interval on disk(from diskPipeFactory) is smaller than one batch") = { + //To test this property, it requires the length of the batcher is at least 2 millis, since we want + //to create data that fits a batch partially + def atLeast2MsBatcher(batcher: Batcher): Boolean = { + batcher match { + case b: MillisecondBatcher => b.durationMillis >= 2 + case _ => true + } + } forAll { (interval: Intersection[InclusiveLower, ExclusiveUpper, Timestamp], inputWithTimeStampAndBatcherAndStore: (List[(Long, Int)], Batcher, TestStore[Int, Int]), mode: Mode) => val (inputWithTimeStamp, batcher, testStore) = inputWithTimeStampAndBatcherAndStore - val nextBatchEnding = batcher.latestTimeOf(testStore.initBatch.next) - - //this diskPipeFactory returns a timeinterval that ends before the ending of next batch, meaning there is not enough data for a new batch - val diskPipeFactory = StateWithError[(Interval[Timestamp], Mode), List[FailureReason], FlowToPipe[(Int, Int)]] { - (timeMode: (Interval[Timestamp], Mode)) => - { - val (time: Interval[Timestamp], mode: Mode) = timeMode - val Intersection(InclusiveLower(startRequestedTime), ExclusiveUpper(_)) = time - - //shrink the endTime so it does not cover a whole batch - val onDiskEndTime: Long = Gen.choose(startRequestedTime.milliSinceEpoch, nextBatchEnding.milliSinceEpoch).sample.get + (atLeast2MsBatcher(batcher)) ==> { + val nextBatchEnding = batcher.latestTimeOf(testStore.initBatch.next) + + //this diskPipeFactory returns a time interval that ends before the ending of next batch, meaning there is not enough data for a new batch + val diskPipeFactory = StateWithError[(Interval[Timestamp], Mode), List[FailureReason], FlowToPipe[(Int, Int)]] { + (timeMode: (Interval[Timestamp], Mode)) => + { + val (time: Interval[Timestamp], mode: Mode) = timeMode + val Intersection(InclusiveLower(startRequestedTime), ExclusiveUpper(_)) = time + + //shrink the endTime so it does not cover a whole batch + val onDiskEndTime: Long = Gen.choose(startRequestedTime.milliSinceEpoch, nextBatchEnding.milliSinceEpoch).sample.get + + val readTime: Interval[Timestamp] = if (startRequestedTime == nextBatchEnding) + Empty() + else + Intersection(InclusiveLower(startRequestedTime), ExclusiveUpper(nextBatchEnding)) + + val flowToPipe: FlowToPipe[(Int, Int)] = Reader { (fdM: (FlowDef, Mode)) => TypedPipe.from[(Timestamp, (Int, Int))](Seq((Timestamp(10), (2, 3)))) } + Right(((readTime, mode), flowToPipe)) + } + } - val readTime: Interval[Timestamp] = if (startRequestedTime == nextBatchEnding) - Empty() - else - Intersection(InclusiveLower(startRequestedTime), ExclusiveUpper(nextBatchEnding)) + val mergeResult = testStore.merge(diskPipeFactory, implicitly[Semigroup[Int]], Commutative, 10)((interval, mode)) - val flowToPipe: FlowToPipe[(Int, Int)] = Reader { (fdM: (FlowDef, Mode)) => TypedPipe.from[(Timestamp, (Int, Int))](Seq((Timestamp(10), (2, 3)))) } - Right(((readTime, mode), flowToPipe)) + mergeResult match { + case Left(l) => { + println(l) + l.mkString.contains("readTimespan is not convering at least one batch").label("fail with right reason") } + case Right(_) => false.label("should fail when readTimespan is not covering at least one batch") + } } - - val mergeResult = testStore.merge(diskPipeFactory, implicitly[Semigroup[Int]], Commutative, 10)((interval, mode)) - - mergeResult.left.get.mkString.contains("readTimespan is not convering at least one batch").label("fail with right reason") - mergeResult.isInstanceOf[Left[_, _]].label("returns Left when data is smaller than one batch") } } } From fc93f6b66929fde0857f0c366ab217c3d58830d2 Mon Sep 17 00:00:00 2001 From: Tianshuo Deng Date: Tue, 24 Mar 2015 11:14:44 -0700 Subject: [PATCH 11/12] gen arbitrary commutativity --- .../summingbird/scalding/BatchedStoreProperties.scala | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/BatchedStoreProperties.scala b/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/BatchedStoreProperties.scala index a24c9126a..3ae332be8 100644 --- a/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/BatchedStoreProperties.scala +++ b/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/BatchedStoreProperties.scala @@ -21,7 +21,7 @@ import cascading.flow.{ Flow, FlowDef } import com.twitter.algebird._ import com.twitter.algebird.monad._ import com.twitter.summingbird.batch._ -import com.twitter.summingbird.option.Commutative +import com.twitter.summingbird.option.{ Commutative, NonCommutative, Commutativity } import com.twitter.scalding.{ Source => ScaldingSource, Test => TestMode, _ } import org.scalacheck._ @@ -76,6 +76,9 @@ object BatchedStoreProperties extends Properties("BatchedStore's Properties") { } implicit def arbitraryLocalMode: Arbitrary[Mode] = Arbitrary { Gen.const(Local(true)) } + implicit def arbitraryCommutativity: Arbitrary[Commutativity] = Arbitrary { + Gen.oneOf(Seq(Commutative, NonCommutative)) + } property("readAfterLastBatch should return interval starting from the last batch written") = { forAll { @@ -123,9 +126,10 @@ object BatchedStoreProperties extends Properties("BatchedStore's Properties") { (diskPipeFactory: PipeFactory[(Int, Int)], interval: Intersection[InclusiveLower, ExclusiveUpper, Timestamp], inputWithTimeStampAndBatcherAndStore: (List[(Long, Int)], Batcher, TestStore[Int, Int]), + commutativity: Commutativity, mode: Mode) => val (inputWithTimeStamp, batcher, testStore) = inputWithTimeStampAndBatcherAndStore - val mergeResult = testStore.merge(diskPipeFactory, implicitly[Semigroup[Int]], com.twitter.summingbird.option.Commutative, 10)((interval, mode)) + val mergeResult = testStore.merge(diskPipeFactory, implicitly[Semigroup[Int]], commutativity, 10)((interval, mode)) mergeResult.isRight ==> { val Right(((Intersection(InclusiveLower(_), ExclusiveUpper(readIntervalUpper)), _), _)) = mergeResult val requestedEndingTimestamp: Timestamp = interval.upper.upper @@ -176,7 +180,6 @@ object BatchedStoreProperties extends Properties("BatchedStore's Properties") { mergeResult match { case Left(l) => { - println(l) l.mkString.contains("readTimespan is not convering at least one batch").label("fail with right reason") } case Right(_) => false.label("should fail when readTimespan is not covering at least one batch") From a860e84f9dc4a08467dd2f49df4fb68cca6bf328 Mon Sep 17 00:00:00 2001 From: Tianshuo Deng Date: Tue, 24 Mar 2015 14:18:47 -0700 Subject: [PATCH 12/12] arbPipefactory returns PipeFactory[Nothing] --- .../scalding/BatchedStoreProperties.scala | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/BatchedStoreProperties.scala b/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/BatchedStoreProperties.scala index 3ae332be8..ffc71b8f4 100644 --- a/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/BatchedStoreProperties.scala +++ b/summingbird-scalding-test/src/test/scala/com/twitter/summingbird/scalding/BatchedStoreProperties.scala @@ -47,14 +47,14 @@ object BatchedStoreProperties extends Properties("BatchedStore's Properties") { .map { Timestamp(_) } } - implicit val arbitraryPipeFactory: Arbitrary[PipeFactory[(Int, Int)]] = { + implicit val arbitraryPipeFactory: Arbitrary[PipeFactory[Nothing]] = { Arbitrary { Gen.const { - StateWithError[(Interval[Timestamp], Mode), List[FailureReason], FlowToPipe[(Int, Int)]] { + StateWithError[(Interval[Timestamp], Mode), List[FailureReason], FlowToPipe[Nothing]] { (timeMode: (Interval[Timestamp], Mode)) => { val (time: Interval[Timestamp], mode: Mode) = timeMode - val a: FlowToPipe[(Int, Int)] = Reader { (fdM: (FlowDef, Mode)) => TypedPipe.from[(Timestamp, (Int, Int))](Seq((Timestamp(10), (2, 3)))) } + val a: FlowToPipe[Nothing] = Reader { (fdM: (FlowDef, Mode)) => TypedPipe.empty } Right((timeMode, a)) } } @@ -82,7 +82,7 @@ object BatchedStoreProperties extends Properties("BatchedStore's Properties") { property("readAfterLastBatch should return interval starting from the last batch written") = { forAll { - (diskPipeFactory: PipeFactory[(Int, Int)], + (diskPipeFactory: PipeFactory[Nothing], interval: Intersection[InclusiveLower, ExclusiveUpper, Timestamp], inputWithTimeStampAndBatcherAndStore: (List[(Long, Int)], Batcher, TestStore[Int, Int]), mode: Mode) => @@ -103,7 +103,7 @@ object BatchedStoreProperties extends Properties("BatchedStore's Properties") { property("readAfterLastBatch should not extend the end of interval requested") = { forAll { - (diskPipeFactory: PipeFactory[(Int, Int)], + (diskPipeFactory: PipeFactory[Nothing], interval: Intersection[InclusiveLower, ExclusiveUpper, Timestamp], inputWithTimeStampAndBatcherAndStore: (List[(Long, Int)], Batcher, TestStore[Int, Int]), mode: Mode) => @@ -123,7 +123,7 @@ object BatchedStoreProperties extends Properties("BatchedStore's Properties") { property("the end of merged interval is never extended") = { forAll { - (diskPipeFactory: PipeFactory[(Int, Int)], + (diskPipeFactory: PipeFactory[Nothing], interval: Intersection[InclusiveLower, ExclusiveUpper, Timestamp], inputWithTimeStampAndBatcherAndStore: (List[(Long, Int)], Batcher, TestStore[Int, Int]), commutativity: Commutativity, @@ -151,6 +151,7 @@ object BatchedStoreProperties extends Properties("BatchedStore's Properties") { forAll { (interval: Intersection[InclusiveLower, ExclusiveUpper, Timestamp], inputWithTimeStampAndBatcherAndStore: (List[(Long, Int)], Batcher, TestStore[Int, Int]), + commutativity: Commutativity, mode: Mode) => val (inputWithTimeStamp, batcher, testStore) = inputWithTimeStampAndBatcherAndStore (atLeast2MsBatcher(batcher)) ==> { @@ -176,7 +177,7 @@ object BatchedStoreProperties extends Properties("BatchedStore's Properties") { } } - val mergeResult = testStore.merge(diskPipeFactory, implicitly[Semigroup[Int]], Commutative, 10)((interval, mode)) + val mergeResult = testStore.merge(diskPipeFactory, implicitly[Semigroup[Int]], commutativity, 10)((interval, mode)) mergeResult match { case Left(l) => {