This repository was archived by the owner on Jan 20, 2022. It is now read-only.
-
Couldn't load subscription status.
- Fork 266
Yet another cache #470
Merged
Merged
Yet another cache #470
Changes from all commits
Commits
Show all changes
12 commits
Select commit
Hold shift + click to select a range
2fa69bd
Add's another cache that has out-preformed the MultiTriggerCache and …
ianoc b2f7d28
Swap the default for the async cache to the YAC
ianoc 17624bc
Made def abstract
ianoc f184083
add missing needed Algebird upgrade that fixes some sparse monoid act…
ianoc 2e912f5
Rename new cache over to a more sensible descriptive name
ianoc 38eadc0
Review comments
ianoc c263321
Merge branch 'develop' of github.com:twitter/summingbird into YetAnot…
ianoc bb4f423
Realise lazy stores on init
ianoc 64f72d2
Fix typo in naming
ianoc 6ff43e5
Update year
ianoc 2e33ec0
Keep the current caches the default
ianoc a9f8c09
Merge branch 'develop' of github.com:twitter/summingbird into YetAnot…
ianoc File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
122 changes: 122 additions & 0 deletions
122
...bird-online/src/main/scala/com/twitter/summingbird/online/BackgroundCompactionCache.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,122 @@ | ||
| /* | ||
| 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.online | ||
|
|
||
| import com.twitter.algebird.Semigroup | ||
| import com.twitter.summingbird.option.CacheSize | ||
| import com.twitter.util.{Future, Promise, FuturePool, Await} | ||
| import com.twitter.summingbird.online.option.{FlushFrequency, SoftMemoryFlushPercent} | ||
| import scala.collection.JavaConverters._ | ||
| import scala.collection.JavaConversions._ | ||
| import scala.collection.mutable.ListBuffer | ||
| import java.util.concurrent._ | ||
| import org.slf4j.{LoggerFactory, Logger} | ||
|
|
||
| object BackgroundCompactionCache { | ||
| def builder[Key, Value](cacheSize: CacheSize, flushFrequency: FlushFrequency, softMemoryFlush: SoftMemoryFlushPercent): CacheBuilder[Key, Value] = | ||
| new CacheBuilder[Key, Value] { | ||
| def apply(sg: Semigroup[Value]) = { | ||
| BackgroundCompactionCache(cacheSize, flushFrequency, softMemoryFlush)(sg) | ||
| } | ||
| } | ||
| def apply[Key, Value](cacheSize: CacheSize, | ||
| flushFrequency: FlushFrequency, | ||
| softMemoryFlush: SoftMemoryFlushPercent) | ||
| (implicit sg: Semigroup[Value]): AsyncCache[Key, Value] = { | ||
| cacheSize.size.map { _ => | ||
| new NonEmptyBackgroundCompactionCache[Key, Value](cacheSize, flushFrequency, softMemoryFlush)(sg) | ||
| }.getOrElse(new EmptyBackgroundCompactionCache[Key, Value]()(sg)) | ||
| } | ||
| } | ||
|
|
||
| private[summingbird] trait WithFlushConditions[Key, Value] extends AsyncCache[Key, Value] { | ||
| protected var lastDump:Long = System.currentTimeMillis | ||
| protected def softMemoryFlush: SoftMemoryFlushPercent | ||
| protected def flushFrequency: FlushFrequency | ||
|
|
||
| protected def timedOut = (System.currentTimeMillis - lastDump) >= flushFrequency.get.inMilliseconds | ||
| protected lazy val runtime = Runtime.getRuntime | ||
|
|
||
| protected def didFlush {lastDump = System.currentTimeMillis} | ||
|
|
||
| protected def memoryWaterMark = { | ||
| val used = ((runtime.totalMemory - runtime.freeMemory).toDouble * 100) / runtime.maxMemory | ||
| used > softMemoryFlush.get | ||
| } | ||
| def tick: Future[Map[Key, Value]] = { | ||
| if (timedOut || memoryWaterMark) { | ||
| forceTick | ||
| } | ||
| else { | ||
| Future.value(Map.empty) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| private[summingbird] trait ParallelCleanup[Key, Value] extends AsyncCache[Key, Value] { | ||
| protected def executor: ExecutorService | ||
| protected lazy val futurePool = FuturePool(executor) | ||
|
|
||
| override def cleanup = { | ||
| Future { | ||
| executor.shutdown | ||
| executor.awaitTermination(10, TimeUnit.SECONDS) | ||
| }.flatMap(f => super.cleanup) | ||
| } | ||
| } | ||
|
|
||
| class EmptyBackgroundCompactionCache[Key, Value](implicit semigroup: Semigroup[Value]) | ||
| extends AsyncCache[Key, Value] { | ||
| def forceTick: Future[Map[Key, Value]] = Future.value(Map.empty) | ||
| def tick: Future[Map[Key, Value]] = Future.value(Map.empty) | ||
| def insert(vals: TraversableOnce[(Key, Value)]): Future[Map[Key, Value]] = Future.value(Semigroup.sumOption(vals.map(Map(_))).getOrElse(Map.empty)) | ||
| } | ||
|
|
||
|
|
||
| class NonEmptyBackgroundCompactionCache[Key, Value](cacheSizeOpt: CacheSize, | ||
| override val flushFrequency: FlushFrequency, | ||
| override val softMemoryFlush: SoftMemoryFlushPercent) | ||
| (implicit semigroup: Semigroup[Value]) | ||
| extends AsyncCache[Key, Value] with ParallelCleanup[Key, Value] with WithFlushConditions[Key, Value] { | ||
|
|
||
| protected override val executor = Executors.newFixedThreadPool(2) | ||
| protected val logger: Logger = LoggerFactory.getLogger(getClass) | ||
| protected val cacheSize = cacheSizeOpt.size.get | ||
|
|
||
| private val queue: ArrayBlockingQueue[Map[Key, Value]] = new ArrayBlockingQueue[Map[Key, Value]](cacheSize, true) | ||
|
|
||
| override def forceTick: Future[Map[Key, Value]] = { | ||
| didFlush // bumps timeout on the flush conditions | ||
| val toSum = ListBuffer[Map[Key, Value]]() | ||
| queue.drainTo(toSum.asJava) | ||
| futurePool { | ||
| Semigroup.sumOption(toSum).getOrElse(Map.empty) | ||
| } | ||
| } | ||
|
|
||
| def insert(vals: TraversableOnce[(Key, Value)]): Future[Map[Key, Value]] = { | ||
| val curData = Semigroup.sumOption(vals.map(Map(_))).getOrElse(Map.empty) | ||
| if(!queue.offer(curData)) { | ||
| forceTick.map { flushRes => | ||
| Semigroup.plus(flushRes, curData) | ||
| } | ||
| } | ||
| else { | ||
| Future.value(Map.empty) | ||
| } | ||
| } | ||
| } |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
94 changes: 94 additions & 0 deletions
94
...ne/src/test/scala/com/twitter/summingbird/online/BackroundCompactionCacheProperties.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,94 @@ | ||
| /* | ||
| 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.online | ||
|
|
||
| import com.twitter.summingbird.online.option._ | ||
| import com.twitter.summingbird.option._ | ||
| import com.twitter.summingbird.planner._ | ||
| import com.twitter.summingbird.memory.Memory | ||
| import com.twitter.algebird.{MapAlgebra, Semigroup} | ||
| import com.twitter.util.{Future, Await} | ||
| import scala.collection.mutable.{Map => MMap} | ||
| import org.scalacheck._ | ||
| import Gen._ | ||
| import Arbitrary._ | ||
| import org.scalacheck.Prop._ | ||
| import scala.util.Random | ||
| import com.twitter.util.Duration | ||
|
|
||
| object BackgroundCompactionCacheProperties extends Properties("BackgroundCompactionCache") { | ||
|
|
||
| implicit def arbFlushFreq = Arbitrary { | ||
| Gen.choose(1, 4000) | ||
| .map { x: Int => FlushFrequency(Duration.fromMilliseconds(x)) } | ||
| } | ||
|
|
||
| implicit def arbCacheSize = Arbitrary { | ||
| Gen.choose(0, 10) | ||
| .map { x => | ||
| CacheSize(x) } | ||
| } | ||
|
|
||
| def sample[T: Arbitrary]: T = Arbitrary.arbitrary[T].sample.get | ||
|
|
||
| property("Summing with and without the cache should match") = forAll { inputs: List[List[(Int, Int)]] => | ||
| val cache = BackgroundCompactionCache[Int, Int](sample[CacheSize], sample[FlushFrequency], SoftMemoryFlushPercent(80.0F)) | ||
| val reference = MapAlgebra.sumByKey(inputs.flatten) | ||
| val resA = Await.result(Future.collect(inputs.map(cache.insert(_)))).map(_.toList).flatten | ||
| val resB = Await.result(cache.forceTick) | ||
| val other = MapAlgebra.sumByKey(resA.toList ++ resB.toList) | ||
| val res = Equiv[Map[Int, Int]].equiv( | ||
| reference, | ||
| other | ||
| ) | ||
| Await.ready(cache.cleanup) | ||
| res | ||
| } | ||
|
|
||
| property("Input Set must not get duplicates") = forAll { (ids: Set[Int], inputs: List[List[(Int, Int)]]) => | ||
| val cache = BackgroundCompactionCache[Int, (List[Int], Int)](sample[CacheSize], sample[FlushFrequency], SoftMemoryFlushPercent(80.0F)) | ||
| val idList = (ids ++ Set(1)).toList | ||
| var refCount = MMap[Int, Int]() | ||
| val realInputs = inputs.map{ iList => | ||
| iList.map{ case (k, v) => | ||
| val id = idList(Random.nextInt(idList.size)) | ||
| refCount += (id -> (refCount.getOrElse(id, 0) + 1)) | ||
| (k, (List(id), v)) | ||
| } | ||
| }.toList | ||
|
|
||
| val reference = MapAlgebra.sumByKey(realInputs.flatten).mapValues(tupV => (tupV._1.sorted, tupV._2)) | ||
| val resA = realInputs.map(cache.insert(_)).map(Await.result(_)).map(_.toList).flatten | ||
| val resB = Await.result(cache.forceTick) | ||
| val other = MapAlgebra.sumByKey(resA.toList ++ resB.toList).mapValues(tupV => (tupV._1.sorted, tupV._2)) | ||
| Await.ready(cache.cleanup) | ||
|
|
||
| val equiv = Equiv[Map[Int, (List[Int], Int)]].equiv( | ||
| reference, | ||
| other | ||
| ) | ||
| if(equiv) { | ||
| val postFreq = MapAlgebra.sumByKey(other.map(_._2._1).flatten.map((_, 1))) | ||
| Equiv[Map[Int, Int]].equiv( | ||
| refCount.toMap, | ||
| postFreq | ||
| ) | ||
| } else { | ||
| equiv | ||
| } | ||
| } | ||
| } | ||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -67,9 +67,9 @@ object StormTestRun { | |
| try { | ||
| val cluster = new LocalCluster() | ||
| cluster.submitTopology("test topology", plannedTopology.config, plannedTopology.topology) | ||
| Thread.sleep(4000) | ||
| Thread.sleep(4500) | ||
| cluster.killTopology("test topology") | ||
| Thread.sleep(1000) | ||
| Thread.sleep(1500) | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. flaky test fix? Is there a better way? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Nah, not with the storm local cluster modes as they are alas. We can't have both tick tuples and the storm feature of run test till all acked. It was still flaky with that, but less so. And had less code coverage for us. |
||
| cluster.shutdown | ||
| } finally { | ||
| System.setSecurityManager(oldSecManager) | ||
|
|
||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
2014?