├── project ├── build.properties ├── plugins.sbt ├── GenTupleAggregators.scala ├── DocGen.scala └── Unidoc.scala ├── version.sbt ├── .travis.yml ├── NOTICE ├── algebird-util └── src │ ├── main │ └── scala │ │ └── com │ │ └── twitter │ │ └── algebird │ │ └── util │ │ ├── summer │ │ ├── Incrementor.scala │ │ ├── MemoryFlushPercent.scala │ │ ├── BufferSize.scala │ │ ├── FlushFrequency.scala │ │ ├── NullSummer.scala │ │ ├── AsyncMapSum.scala │ │ ├── AsyncSummer.scala │ │ └── AsyncListMMapSum.scala │ │ ├── UtilAlgebras.scala │ │ └── PromiseLinkMonoid.scala │ └── test │ └── scala │ └── com │ └── twitter │ └── algebird │ └── util │ ├── summer │ ├── Counter.scala │ ├── NullSummerProperties.scala │ ├── AsyncListMMapSumProperties.scala │ ├── AsyncMapSumProperties.scala │ ├── SyncSummingQueueProperties.scala │ ├── HeavyHittersCachingSummerProperties.scala │ ├── AsyncSummerLaws.scala │ └── AsyncListSumProperties.scala │ ├── PromiseLinkMonoidProperties.scala │ ├── UtilAlgebraProperties.scala │ └── TunnelMonoidProperties.scala ├── algebird-test └── src │ ├── test │ └── scala │ │ └── com │ │ └── twitter │ │ └── algebird │ │ ├── OperatorTest.scala │ │ ├── TupleRing.scala │ │ ├── MonadFoldMTest.scala │ │ ├── RightFoldedTest.scala │ │ ├── PredecessibleTests.scala │ │ ├── SuccessibleProperties.scala │ │ ├── FunctionMonoidTests.scala │ │ ├── MinMaxAggregatorTest.scala │ │ ├── VectorSpaceProperties.scala │ │ ├── AdJoinedUnitRing.scala │ │ ├── FunctorProperties.scala │ │ ├── ApplicativeProperties.scala │ │ ├── MinPlusLaws.scala │ │ ├── ResetTest.scala │ │ ├── AggregationMonoids.scala │ │ ├── SpaceSaverTest.scala │ │ ├── DecayedVectorProperties.scala │ │ ├── MinHasherTest.scala │ │ ├── AggregatorLaws.scala │ │ ├── JavaBoxedTests.scala │ │ ├── MomentsGroupTest.scala │ │ ├── SummingIteratorTest.scala │ │ ├── statistics │ │ └── StatisticsTests.scala │ │ ├── MonadProperties.scala │ │ ├── Generators.scala │ │ ├── TopKTests.scala │ │ ├── SGDTest.scala │ │ ├── RightFolded2Test.scala │ │ ├── FoldTest.scala │ │ ├── MonadInstanceLaws.scala │ │ └── QTreeTest.scala │ └── main │ └── scala │ └── com │ └── twitter │ └── algebird │ ├── StatefulSummerLaws.scala │ ├── FunctorLaws.scala │ ├── ApplicativeLaws.scala │ ├── SucPredLaws.scala │ ├── BaseVectorSpaceProperties.scala │ └── MonadLaws.scala ├── CONTRIBUTING.md ├── .gitignore ├── algebird-caliper ├── src │ └── test │ │ └── scala │ │ └── com │ │ └── twitter │ │ └── algebird │ │ └── caliper │ │ ├── HllBatchCreateBenchmark.scala │ │ ├── HLLBenchmark.scala │ │ └── CMSBenchmark.scala └── README.md ├── algebird-core └── src │ └── main │ ├── scala │ └── com │ │ └── twitter │ │ └── algebird │ │ ├── legacy │ │ ├── CountMinSketchMonoid.scala │ │ └── package.scala │ │ ├── package.scala │ │ ├── monad │ │ ├── EitherMonad.scala │ │ ├── Trampoline.scala │ │ └── Reader.scala │ │ ├── HashingTrick.scala │ │ ├── ResetAlgebra.scala │ │ ├── matrix │ │ ├── DenseMatrix.scala │ │ └── SparseColumnMatrix.scala │ │ ├── statistics │ │ ├── Counter.scala │ │ └── IterCallStatistics.scala │ │ ├── mutable │ │ ├── PriorityQueueAggregator.scala │ │ └── PriorityQueueMonoid.scala │ │ ├── Operators.scala │ │ ├── StatefulSummer.scala │ │ ├── Functor.scala │ │ ├── RightFolded.scala │ │ ├── MurmurHash.scala │ │ ├── VectorSpace.scala │ │ ├── BufferedOperation.scala │ │ ├── AffineFunction.scala │ │ ├── Combinator.scala │ │ ├── SummingIterator.scala │ │ ├── AdjoinedUnitRing.scala │ │ ├── MinPlus.scala │ │ ├── SummingCache.scala │ │ ├── IndexedSeq.scala │ │ ├── SummingQueue.scala │ │ ├── AveragedValue.scala │ │ ├── DecayedValue.scala │ │ ├── DecayedVector.scala │ │ └── Predecessible.scala │ └── java │ └── com │ └── twitter │ └── algebird │ └── javaapi │ └── Monoids.java └── algebird-bijection └── src └── test └── scala └── com └── twitter └── algebird └── bijection └── AlgebirdBijectionLaws.scala /project/build.properties: -------------------------------------------------------------------------------- 1 | sbt.version=0.13.5 2 | -------------------------------------------------------------------------------- /version.sbt: -------------------------------------------------------------------------------- 1 | version in ThisBuild := "0.8.1" 2 | -------------------------------------------------------------------------------- /.travis.yml: -------------------------------------------------------------------------------- 1 | language: scala 2 | scala: 3 | - 2.10.4 4 | - 2.11.4 5 | -------------------------------------------------------------------------------- /NOTICE: -------------------------------------------------------------------------------- 1 | MurmurHash implementation provided by software developed by The Apache Software 2 | Foundation (http://www.apache.org/). 3 | 4 | 5 | -------------------------------------------------------------------------------- /algebird-util/src/main/scala/com/twitter/algebird/util/summer/Incrementor.scala: -------------------------------------------------------------------------------- 1 | package com.twitter.algebird.util.summer 2 | 3 | /** 4 | * @author Mansur Ashraf. 5 | */ 6 | trait Incrementor { 7 | def incr: Unit 8 | def incrBy(amount: Long): Unit 9 | } -------------------------------------------------------------------------------- /algebird-util/src/test/scala/com/twitter/algebird/util/summer/Counter.scala: -------------------------------------------------------------------------------- 1 | package com.twitter.algebird.util.summer 2 | 3 | import java.util.concurrent.atomic.AtomicLong 4 | 5 | /** 6 | * @author Mansur Ashraf. 7 | */ 8 | case class Counter(name: String) extends Incrementor { 9 | private val counter = new AtomicLong() 10 | 11 | override def incr: Unit = counter.incrementAndGet() 12 | 13 | override def incrBy(amount: Long): Unit = counter.addAndGet(amount) 14 | 15 | def size = counter.get() 16 | 17 | override def toString: String = s"$name: size:$size" 18 | } 19 | -------------------------------------------------------------------------------- /project/plugins.sbt: -------------------------------------------------------------------------------- 1 | resolvers ++= Seq( 2 | "jgit-repo" at "http://download.eclipse.org/jgit/maven", 3 | "sonatype-releases" at "http://oss.sonatype.org/content/repositories/releases", 4 | Resolver.url("bintray-sbt-plugin-releases", 5 | url("http://dl.bintray.com/content/sbt/sbt-plugin-releases"))(Resolver.ivyStylePatterns) 6 | ) 7 | 8 | addSbtPlugin("me.lessis" % "cappi" % "0.1.1") 9 | 10 | addSbtPlugin("com.typesafe.sbt" % "sbt-ghpages" % "0.5.2") 11 | 12 | addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.1.6") 13 | 14 | addSbtPlugin("com.typesafe.sbt" % "sbt-scalariform" % "1.3.0") 15 | -------------------------------------------------------------------------------- /algebird-test/src/test/scala/com/twitter/algebird/OperatorTest.scala: -------------------------------------------------------------------------------- 1 | package com.twitter.algebird 2 | 3 | import org.scalatest._ 4 | 5 | import Operators._ 6 | 7 | class OperatorTest extends WordSpec with Matchers { 8 | 9 | "Operators" should { 10 | "allow plus" in { 11 | assert(List(1, 2) + List(3, 4) == List(1, 2, 3, 4)) 12 | } 13 | "allow -" in { 14 | assert((1, 3) - (2, 4) == (-1, -1)) 15 | } 16 | "allow *" in { 17 | assert(Map(1 -> 3) * Map(2 -> 4) == Map[Int, Int]()) 18 | assert(Map(1 -> 3) * Map(1 -> 4) == Map(1 -> 12)) 19 | } 20 | "allow /" in { 21 | assert(true / true == true) 22 | } 23 | } 24 | } 25 | -------------------------------------------------------------------------------- /algebird-test/src/test/scala/com/twitter/algebird/TupleRing.scala: -------------------------------------------------------------------------------- 1 | package com.twitter.algebird 2 | 3 | import org.scalatest.{ PropSpec, Matchers } 4 | import org.scalatest.prop.PropertyChecks 5 | import org.scalacheck.Arbitrary 6 | 7 | class TupleRingSpecification extends PropSpec with PropertyChecks with Matchers { 8 | import BaseProperties._ 9 | 10 | property("(Int,Int) is a Ring") { 11 | ringLaws[(Int, Int)] 12 | } 13 | 14 | property("(Int,Long) is a Ring") { 15 | ringLaws[(Int, Long)] 16 | } 17 | 18 | property("(Long,Int,Int) is a Ring") { 19 | ringLaws[(Long, Int, Int)] 20 | } 21 | 22 | property("(Long,Int,Int,Long) is a Ring") { 23 | ringLaws[(Long, Int, Int, Long)] 24 | } 25 | } 26 | -------------------------------------------------------------------------------- /CONTRIBUTING.md: -------------------------------------------------------------------------------- 1 | Recommendations and requirements for how to best contribute to Algebird. 2 | 3 | We strive to obey these as best as possible. As always, thanks for contributing--we hope these guidelines make it easier and shed some light on our approach and processes. 4 | 5 | ### Key branches 6 | - `master` is the latest, deployed version 7 | - `develop` is where development happens and all pull requests should be submitted 8 | 9 | ### Pull requests 10 | - Submit pull requests against the `develop` branch 11 | - Try not to pollute your pull request with unintended changes--keep them simple and small 12 | 13 | ### License 14 | By contributing your code, you agree to license your contribution under the terms of the APLv2: 15 | https://github.com/twitter/algebird/blob/master/LICENSE 16 | -------------------------------------------------------------------------------- /algebird-util/src/main/scala/com/twitter/algebird/util/summer/MemoryFlushPercent.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | package com.twitter.algebird.util.summer 17 | 18 | /** 19 | * @author Ian O Connell 20 | */ 21 | 22 | case class MemoryFlushPercent(v: Float) -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | # see also test/files/.gitignore 2 | /test/files/.gitignore 3 | 4 | *.jar 5 | *~ 6 | 7 | #sbt 8 | /project/target/ 9 | /project/project/target 10 | 11 | /target/ 12 | /src/jline/target/ 13 | 14 | # target directories for ant build 15 | /build/ 16 | /dists/ 17 | 18 | # other 19 | /out/ 20 | /bin/ 21 | /sandbox/ 22 | 23 | # eclipse, intellij 24 | /.classpath 25 | /.project 26 | /src/intellij/*.iml 27 | /src/intellij/*.ipr 28 | /src/intellij/*.iws 29 | /.cache 30 | /.idea 31 | /.settings 32 | */.classpath 33 | */.project 34 | */.cache 35 | */.settings 36 | 37 | # bak files produced by ./cleanup-commit 38 | *.bak 39 | *.swp 40 | 41 | # from Scalding 42 | BUILD 43 | target/ 44 | lib_managed/ 45 | project/boot/ 46 | project/build/target/ 47 | project/plugins/target/ 48 | project/plugins/lib_managed/ 49 | project/plugins/src_managed/ 50 | -------------------------------------------------------------------------------- /algebird-util/src/main/scala/com/twitter/algebird/util/summer/BufferSize.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | package com.twitter.algebird.util.summer 17 | 18 | import com.twitter.util.Duration 19 | 20 | /** 21 | * @author Ian O Connell 22 | */ 23 | 24 | case class BufferSize(v: Int) -------------------------------------------------------------------------------- /algebird-util/src/main/scala/com/twitter/algebird/util/summer/FlushFrequency.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | package com.twitter.algebird.util.summer 17 | 18 | import com.twitter.util.Duration 19 | 20 | /** 21 | * @author Ian O Connell 22 | */ 23 | 24 | case class FlushFrequency(v: Duration) -------------------------------------------------------------------------------- /algebird-test/src/test/scala/com/twitter/algebird/MonadFoldMTest.scala: -------------------------------------------------------------------------------- 1 | package com.twitter.algebird 2 | 3 | import org.scalatest._ 4 | 5 | class MonadFoldMTest extends WordSpec with Matchers { 6 | 7 | def binSmalls(x: Int, y: Int): Option[Int] = if (y > 9) None else Some(x + y) 8 | "A monad foldM" should { 9 | "fold correctly" in { 10 | 11 | // nice easy example from Learn You a Haskell 12 | 13 | val first = Monad.foldM(0, List(2, 8, 3, 1))(binSmalls) 14 | assert(first == Some(14)) 15 | def binSmalls2(x: Int, y: String): Option[Int] = if (y == "11") None else Some(x + y.toInt) 16 | 17 | val second = Monad.foldM(0, List("2", "11", "3", "1"))(binSmalls2) 18 | assert(second == None) 19 | } 20 | "handle an empty list" in { 21 | val third = Monad.foldM(0, List.empty)(binSmalls) 22 | assert(third == Some(0)) 23 | } 24 | } 25 | } -------------------------------------------------------------------------------- /algebird-caliper/src/test/scala/com/twitter/algebird/caliper/HllBatchCreateBenchmark.scala: -------------------------------------------------------------------------------- 1 | package com.twitter.algebird.caliper 2 | 3 | import com.google.caliper.{ SimpleBenchmark, Param } 4 | import com.twitter.algebird.HyperLogLogMonoid 5 | import com.twitter.bijection._ 6 | import java.nio.ByteBuffer 7 | 8 | class HllBatchCreateBenchmark extends SimpleBenchmark { 9 | @Param(Array("5", "10", "17", "25")) 10 | val bits: Int = 0 11 | 12 | @Param(Array("10", "20", "30")) 13 | val max: Long = 0 14 | 15 | var set: Set[Long] = _ 16 | 17 | implicit val byteEncoder = implicitly[Injection[Long, Array[Byte]]] 18 | 19 | override def setUp { 20 | set = (0L until max).toSet 21 | } 22 | 23 | def timeBatchCreate(reps: Int): Int = { 24 | val hllMonoid = new HyperLogLogMonoid(bits) 25 | var dummy = 0 26 | while (dummy < reps) { 27 | val hll = hllMonoid.batchCreate(set)(byteEncoder.toFunction) 28 | dummy += 1 29 | } 30 | dummy 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /algebird-core/src/main/scala/com/twitter/algebird/legacy/CountMinSketchMonoid.scala: -------------------------------------------------------------------------------- 1 | package com.twitter.algebird.legacy 2 | 3 | import com.twitter.algebird.TopPctCMS 4 | 5 | /** 6 | * Creates instances of type `CountMinSketchMonoid` (which is a type alias for the legacy class of the same name in 7 | * Algebird versions < 0.8.1). 8 | * 9 | * =Deprecated since 0.8.1= 10 | * 11 | * Only use this object when transitioning legacy code. Newer code should use [[TopPctCMS.monoid()]]: 12 | * 13 | * {{{ 14 | * import com.twitter.algebird.CMSHasherImplicits._ 15 | * val cmsMonoid = TopPctCMS.monoid[Long](...) 16 | * }}} 17 | */ 18 | object CountMinSketchMonoid { 19 | 20 | import com.twitter.algebird.CMSHasherImplicits._ 21 | 22 | @deprecated( 23 | "You should use TopPctCMS.monoid[Long]() instead of legacy.CountMinSketchMonoid, and import CMSHasherImplicits._", 24 | since = "0.8.1") 25 | def apply(eps: Double, delta: Double, seed: Int, heavyHittersPct: Double = 0.01): CountMinSketchMonoid = 26 | TopPctCMS.monoid[Long](eps, delta, seed, heavyHittersPct) 27 | 28 | } -------------------------------------------------------------------------------- /algebird-core/src/main/scala/com/twitter/algebird/package.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2014 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter 18 | 19 | package object algebird { 20 | /** 21 | * TODO remove these in scala 2.11 and use the standard there. 22 | * these are here to avoid massive bloat around these classes 23 | * https://github.com/twitter/algebird/issues/263 24 | */ 25 | private[algebird] abstract class AbstractIterable[T] extends Iterable[T] 26 | private[algebird] abstract class AbstractIterator[T] extends Iterator[T] 27 | } 28 | -------------------------------------------------------------------------------- /algebird-bijection/src/test/scala/com/twitter/algebird/bijection/AlgebirdBijectionLaws.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2010 Twitter Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); you may 5 | * not use this file except in compliance with the License. You may obtain 6 | * a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird.bijection 18 | 19 | import org.scalatest.{ PropSpec, Matchers } 20 | import org.scalatest.prop.PropertyChecks 21 | import org.scalacheck.{ Arbitrary, Properties } 22 | 23 | class AlgebirdBijectionLaws extends PropSpec with PropertyChecks with Matchers { 24 | // TODO: Fill in tests. Ideally we'd publish an algebird-testing 25 | // module before merging this in. 26 | } 27 | -------------------------------------------------------------------------------- /algebird-test/src/test/scala/com/twitter/algebird/RightFoldedTest.scala: -------------------------------------------------------------------------------- 1 | package com.twitter.algebird 2 | 3 | import org.scalatest.{ PropSpec, Matchers } 4 | import org.scalatest.prop.PropertyChecks 5 | import org.scalacheck.{ Gen, Arbitrary } 6 | 7 | class RightFoldedTest extends PropSpec with PropertyChecks with Matchers { 8 | import BaseProperties._ 9 | 10 | implicit def rightFoldedValue[Out: Arbitrary]: Arbitrary[RightFoldedValue[Out]] = 11 | Arbitrary { 12 | for (v <- implicitly[Arbitrary[Out]].arbitrary) yield RightFoldedValue(v) 13 | } 14 | implicit def rightFoldedToFold[In: Arbitrary]: Arbitrary[RightFoldedToFold[In]] = 15 | Arbitrary { 16 | for (v <- implicitly[Arbitrary[In]].arbitrary) yield RightFoldedToFold(List(v)) 17 | } 18 | implicit def rightFolded[In: Arbitrary, Out: Arbitrary]: Arbitrary[RightFolded[In, Out]] = 19 | Arbitrary { Gen.oneOf(rightFoldedValue[Out].arbitrary, rightFoldedToFold[In].arbitrary) } 20 | 21 | implicit val rightFoldedMonoid = RightFolded.monoid[Int, Long] { (i, l) => l + i.toLong } 22 | 23 | property("RightFolded is a monoid") { 24 | monoidLaws[RightFolded[Int, Long]] 25 | } 26 | 27 | } 28 | -------------------------------------------------------------------------------- /algebird-test/src/test/scala/com/twitter/algebird/PredecessibleTests.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2014 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | package com.twitter.algebird 17 | 18 | import org.scalatest.{ PropSpec, Matchers } 19 | import org.scalatest.prop.PropertyChecks 20 | import org.scalacheck.{ Arbitrary, Properties } 21 | 22 | class PredecessibleTests extends PropSpec with PropertyChecks with Matchers { 23 | import PredecessibleLaws.{ predessibleLaws => laws } 24 | 25 | property("Int is Predecessible") { 26 | laws[Int] 27 | } 28 | 29 | property("Long is Predecessible") { 30 | laws[Long] 31 | } 32 | 33 | property("BigInt is Predecessible") { 34 | laws[BigInt] 35 | } 36 | 37 | } 38 | -------------------------------------------------------------------------------- /algebird-test/src/test/scala/com/twitter/algebird/SuccessibleProperties.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird 18 | 19 | import org.scalatest.{ PropSpec, Matchers } 20 | import org.scalatest.prop.PropertyChecks 21 | import org.scalacheck.{ Arbitrary, Properties } 22 | 23 | class SuccessibleProperties extends PropSpec with PropertyChecks with Matchers { 24 | import SuccessibleLaws.{ successibleLaws => laws } 25 | 26 | property("Int is Successible") { 27 | laws[Int] 28 | } 29 | 30 | property("Long is Successible") { 31 | laws[Long] 32 | } 33 | 34 | property("BigInt is Successible") { 35 | laws[BigInt] 36 | } 37 | 38 | } 39 | -------------------------------------------------------------------------------- /algebird-util/src/test/scala/com/twitter/algebird/util/summer/NullSummerProperties.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2013 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird.util.summer 18 | 19 | import org.scalatest.{ PropSpec, Matchers } 20 | import org.scalatest.prop.PropertyChecks 21 | 22 | class NullSummerProperties extends PropSpec with PropertyChecks with Matchers { 23 | import AsyncSummerLaws._ 24 | 25 | property("Summing with and without the summer should match") { 26 | forAll { (inputs: List[List[(Int, Long)]]) => 27 | val tuplesIn = Counter("tuplesIn") 28 | val tuplesOut = Counter("tuplesOut") 29 | val summer = new NullSummer[Int, Long](tuplesIn, tuplesOut) 30 | assert(summingWithAndWithoutSummerShouldMatch(summer, inputs)) 31 | } 32 | } 33 | 34 | } 35 | -------------------------------------------------------------------------------- /algebird-util/src/test/scala/com/twitter/algebird/util/PromiseLinkMonoidProperties.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | package com.twitter.algebird.util 17 | 18 | import com.twitter.algebird._ 19 | import org.scalatest.{ PropSpec, Matchers } 20 | import org.scalatest.prop.PropertyChecks 21 | 22 | class PromiseLinkMonoidProperties extends PropSpec with PropertyChecks with Matchers { 23 | property("associative") { 24 | def makeTunnel(seed: Int) = PromiseLink.toPromiseLink(seed) 25 | def collapseFinalValues(finalTunnel: PromiseLink[Int], tunnels: Seq[PromiseLink[Int]], toFeed: Int) = { 26 | finalTunnel.completeWithStartingValue(toFeed) 27 | finalTunnel.promise +: tunnels.map { _.promise } 28 | } 29 | 30 | TunnelMonoidProperties.testTunnelMonoid(identity, makeTunnel, collapseFinalValues) 31 | } 32 | } -------------------------------------------------------------------------------- /algebird-test/src/test/scala/com/twitter/algebird/FunctionMonoidTests.scala: -------------------------------------------------------------------------------- 1 | package com.twitter.algebird 2 | 3 | import org.scalatest.{ PropSpec, Matchers } 4 | import org.scalatest.prop.PropertyChecks 5 | import org.scalacheck.{ Prop, Arbitrary } 6 | 7 | class FunctionMonoidTests extends PropSpec with PropertyChecks with Matchers { 8 | import BaseProperties._ 9 | 10 | // Generates an arbitrary linear function of the form f(x) = a * x + b, 11 | // where a and b are arbitrary integers. 12 | // TODO: add more types of functions (e.g., polynomials of degree two). 13 | implicit def arbLinearFunc1: Arbitrary[Function1[Int, Int]] = Arbitrary[Function[Int, Int]] { 14 | for ( 15 | a <- Arbitrary.arbInt.arbitrary; 16 | b <- Arbitrary.arbInt.arbitrary 17 | ) yield ((x: Int) => a * x + b) 18 | } 19 | 20 | property("Linear functions over the integers form a monoid under function composition") { 21 | // TODO: switch the scope of the quantification? 22 | Prop.forAll { (n: Int) => 23 | monoidLawsEq[Function1[Int, Int]] { (f1, f2) => f1(n) == f2(n) } 24 | } 25 | } 26 | 27 | implicit def arbAffine: Arbitrary[AffineFunction[Int]] = Arbitrary[AffineFunction[Int]] { 28 | for ( 29 | a <- Arbitrary.arbInt.arbitrary; 30 | b <- Arbitrary.arbInt.arbitrary 31 | ) yield AffineFunction(a, b) 32 | } 33 | 34 | property("AffineFunctions are monoids") { 35 | monoidLaws[AffineFunction[Int]] 36 | } 37 | 38 | } 39 | -------------------------------------------------------------------------------- /algebird-core/src/main/scala/com/twitter/algebird/monad/EitherMonad.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2013 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird.monad 18 | 19 | import com.twitter.algebird.Monad 20 | 21 | // Monad for either, used for modeling Error where L is the type of the error 22 | object EitherMonad { 23 | class Error[L] extends Monad[({ type RightType[R] = Either[L, R] })#RightType] { 24 | def apply[R](r: R) = Right(r) 25 | 26 | def flatMap[T, U](self: Either[L, T])(next: T => Either[L, U]): Either[L, U] = 27 | self.right.flatMap(next) 28 | 29 | override def map[T, U](self: Either[L, T])(fn: T => U): Either[L, U] = 30 | self.right.map(fn) 31 | } 32 | implicit def monad[L]: Monad[({ type RightT[R] = Either[L, R] })#RightT] = new Error[L] 33 | 34 | def assert[L](truth: Boolean, failure: => L): Either[L, Unit] = if (truth) Right(()) else Left(failure) 35 | } 36 | -------------------------------------------------------------------------------- /algebird-core/src/main/scala/com/twitter/algebird/HashingTrick.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2013 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird 18 | 19 | class HashingTrickMonoid[V: Group](bits: Int, seed: Int = 123456) extends Monoid[AdaptiveVector[V]] { 20 | val vectorSize = 1 << bits 21 | val bitMask = vectorSize - 1 22 | val hash = MurmurHash128(seed) 23 | 24 | val zero = AdaptiveVector.fill[V](vectorSize)(Monoid.zero[V]) 25 | 26 | def plus(left: AdaptiveVector[V], right: AdaptiveVector[V]) = Monoid.plus(left, right) 27 | 28 | def init[K <% Array[Byte]](kv: (K, V)): AdaptiveVector[V] = { 29 | val (long1, long2) = hash(kv._1) 30 | val index = (long1 & bitMask).toInt 31 | val isNegative = (long2 & 1) == 1 32 | 33 | val signedValue = if (isNegative) Group.negate(kv._2) else kv._2 34 | AdaptiveVector.fromMap[V](Map(index -> signedValue), Monoid.zero[V], vectorSize) 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /algebird-test/src/test/scala/com/twitter/algebird/MinMaxAggregatorTest.scala: -------------------------------------------------------------------------------- 1 | package com.twitter.algebird 2 | 3 | import org.scalatest._ 4 | 5 | class MinMaxAggregatorTest extends WordSpec with Matchers { 6 | val data = List(1, 3, 5, 0, 7, 6) 7 | 8 | sealed trait TestElementParent 9 | case object TestElementA extends TestElementParent 10 | case object TestElementB extends TestElementParent 11 | case object TestElementC extends TestElementParent 12 | 13 | implicit val testOrdering = Ordering.fromLessThan[TestElementParent]((x, y) => (x, y) match { 14 | case (TestElementA, TestElementA) => false 15 | case (TestElementA, _) => true 16 | case (TestElementB, TestElementB) => false 17 | case (TestElementB, TestElementA) => false 18 | case (TestElementB, TestElementC) => true 19 | case (TestElementC, _) => false 20 | }) 21 | 22 | val data2 = List(TestElementC, TestElementA, TestElementB) 23 | 24 | "MinAggregator" should { 25 | "produce the minimum value" in { 26 | val agg = Min.aggregator[Int] 27 | assert(agg(data) == 0) 28 | 29 | val agg2 = Min.aggregator[TestElementParent] 30 | assert(agg2(data2) == TestElementA) 31 | } 32 | } 33 | 34 | "MaxAggregator" should { 35 | "produce the maximum value" in { 36 | val agg = Max.aggregator[Int] 37 | assert(agg(data) == 7) 38 | 39 | val agg2 = Max.aggregator[TestElementParent] 40 | assert(agg2(data2) == TestElementC) 41 | } 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /algebird-util/src/main/scala/com/twitter/algebird/util/summer/NullSummer.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | package com.twitter.algebird.util.summer 17 | 18 | import com.twitter.algebird._ 19 | import com.twitter.util.Future 20 | 21 | /** 22 | * @author Ian O Connell 23 | */ 24 | 25 | class NullSummer[Key, Value](tuplesIn: Incrementor, tuplesOut: Incrementor)(implicit semigroup: Semigroup[Value]) 26 | extends AsyncSummer[(Key, Value), Map[Key, Value]] { 27 | def flush: Future[Map[Key, Value]] = Future.value(Map.empty) 28 | def tick: Future[Map[Key, Value]] = Future.value(Map.empty) 29 | def addAll(vals: TraversableOnce[(Key, Value)]): Future[Map[Key, Value]] = { 30 | 31 | val r = Semigroup.sumOption(vals.map { inV => 32 | tuplesIn.incr 33 | Map(inV) 34 | }).getOrElse(Map.empty) 35 | tuplesOut.incrBy(r.size) 36 | Future.value(r) 37 | } 38 | override val isFlushed: Boolean = true 39 | } -------------------------------------------------------------------------------- /algebird-test/src/test/scala/com/twitter/algebird/VectorSpaceProperties.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird 18 | 19 | import org.scalatest.{ PropSpec, Matchers } 20 | import org.scalatest.prop.PropertyChecks 21 | import org.scalacheck.Prop._ 22 | 23 | class VectorSpaceProperties extends PropSpec with PropertyChecks with Matchers { 24 | import BaseVectorSpaceProperties._ 25 | 26 | // TODO: we won't need this when we have an Equatable trait 27 | def mapEqFn(a: Map[Int, Double], b: Map[Int, Double]) = { 28 | (a.keySet ++ b.keySet).forall { key => 29 | (a.get(key), b.get(key)) match { 30 | case (Some(aVal), Some(bVal)) => beCloseTo(aVal, bVal) 31 | case (Some(aVal), None) => beCloseTo(aVal, 0.0) 32 | case (None, Some(bVal)) => beCloseTo(bVal, 0.0) 33 | case _ => true 34 | } 35 | } 36 | } 37 | 38 | property("map int double scaling") { 39 | vectorSpaceLaws[Double, ({ type x[a] = Map[Int, a] })#x](mapEqFn(_, _)) 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /algebird-test/src/test/scala/com/twitter/algebird/AdJoinedUnitRing.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird 18 | 19 | import org.scalacheck.Arbitrary 20 | import org.scalatest.{ PropSpec, Matchers } 21 | import org.scalatest.prop.PropertyChecks 22 | 23 | class AdjoinedRingSpecification extends PropSpec with PropertyChecks with Matchers { 24 | import BaseProperties._ 25 | 26 | implicit def adjoined[T: Arbitrary]: Arbitrary[AdjoinedUnit[T]] = Arbitrary { 27 | implicitly[Arbitrary[T]].arbitrary.map { t => AdjoinedUnit(t) } 28 | } 29 | // AdjoinedUnit requires this method to be correct, so it is tested here: 30 | property("intTimes works correctly") { 31 | forAll { (bi0: BigInt, bi1: BigInt) => 32 | assert(Group.intTimes(bi0, bi1) == (bi0 * bi1)) 33 | } 34 | } 35 | 36 | property("AdjoinedUnit[Int] is a Ring") { 37 | ringLaws[AdjoinedUnit[Int]] 38 | } 39 | 40 | property("AdjoinedUnit[Long] is a Ring") { 41 | ringLaws[AdjoinedUnit[Long]] 42 | } 43 | 44 | } 45 | -------------------------------------------------------------------------------- /algebird-test/src/test/scala/com/twitter/algebird/FunctorProperties.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2014 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird 18 | 19 | import org.scalatest.{ PropSpec, Matchers } 20 | import org.scalatest.prop.PropertyChecks 21 | import org.scalacheck.Properties 22 | 23 | class FunctorProperties extends PropSpec with PropertyChecks with Matchers { 24 | import FunctorLaws._ 25 | import Monad._ // for Monad instances 26 | import MonadLaws._ // for Arbitrary instances 27 | 28 | property("list") { 29 | functorLaws[List, Int, String, Long]() 30 | } 31 | 32 | property("option") { 33 | functorLaws[Option, Int, String, Long]() 34 | } 35 | 36 | property("indexedseq") { 37 | functorLaws[IndexedSeq, Int, String, Long]() 38 | } 39 | 40 | property("vector") { 41 | functorLaws[Vector, Int, String, Long]() 42 | } 43 | 44 | property("set") { 45 | functorLaws[Set, Int, String, Long]() 46 | } 47 | 48 | property("seq") { 49 | functorLaws[Seq, Int, String, Long]() 50 | } 51 | 52 | } 53 | -------------------------------------------------------------------------------- /algebird-core/src/main/scala/com/twitter/algebird/ResetAlgebra.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | package com.twitter.algebird 17 | 18 | /** 19 | * Used to represent cases where we need to periodically reset 20 | * a + b = a + b 21 | * |a + b = |(a + b) 22 | * a + |b = |b 23 | * |a + |b = |b 24 | */ 25 | sealed trait ResetState[+A] { def get: A } 26 | case class SetValue[+A](get: A) extends ResetState[A] 27 | case class ResetValue[+A](get: A) extends ResetState[A] 28 | 29 | object ResetState { 30 | implicit def monoid[A: Monoid]: Monoid[ResetState[A]] = new ResetStateMonoid[A] 31 | } 32 | 33 | class ResetStateMonoid[A](implicit monoid: Monoid[A]) extends Monoid[ResetState[A]] { 34 | def zero = SetValue(monoid.zero) 35 | def plus(left: ResetState[A], right: ResetState[A]) = { 36 | (left, right) match { 37 | case (SetValue(l), SetValue(r)) => SetValue(monoid.plus(l, r)) 38 | case (ResetValue(l), SetValue(r)) => ResetValue(monoid.plus(l, r)) 39 | case (_, ResetValue(r)) => right 40 | } 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /algebird-test/src/test/scala/com/twitter/algebird/ApplicativeProperties.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2014 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird 18 | 19 | import org.scalatest.{ PropSpec, Matchers } 20 | import org.scalatest.prop.PropertyChecks 21 | import org.scalacheck.Properties 22 | 23 | class ApplicativeProperties extends PropSpec with PropertyChecks with Matchers { 24 | import ApplicativeLaws._ 25 | import Monad._ // for Monad instances 26 | import MonadLaws._ // for Arbitrary instances 27 | 28 | property("list") { 29 | applicativeLaws[List, Int, String, Long]() 30 | } 31 | 32 | property("option") { 33 | applicativeLaws[Option, Int, String, Long]() 34 | } 35 | 36 | property("indexedseq") { 37 | applicativeLaws[IndexedSeq, Int, String, Long]() 38 | } 39 | property("vector") { 40 | applicativeLaws[Vector, Int, String, Long]() 41 | } 42 | 43 | property("set") { 44 | applicativeLaws[Set, Int, String, Long]() 45 | } 46 | property("seq") { 47 | applicativeLaws[Seq, Int, String, Long]() 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /algebird-core/src/main/scala/com/twitter/algebird/matrix/DenseMatrix.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird.matrix 18 | import scala.collection.mutable.{ ArrayBuffer, Map => MMap } 19 | 20 | import com.twitter.algebird.Monoid 21 | 22 | case class DenseMatrix[V: Monoid](rows: Int, cols: Int, rowsByColumns: IndexedSeq[V]) extends AdaptiveMatrix[V] { 23 | val valueMonoid = implicitly[Monoid[V]] 24 | 25 | private[this] def tupToIndex(position: (Int, Int)) = position._1 * cols + position._2 26 | 27 | override def getValue(position: (Int, Int)): V = rowsByColumns(tupToIndex(position)) 28 | 29 | override def updated(position: (Int, Int), value: V): DenseMatrix[V] = 30 | DenseMatrix[V](rows, cols, rowsByColumns.updated(tupToIndex(position), value)) 31 | 32 | override def updateInto(buffer: ArrayBuffer[V]) { 33 | var indx = 0 34 | val lsize = size 35 | while (indx < lsize) { 36 | buffer(indx) = valueMonoid.plus(buffer(indx), rowsByColumns(indx)) 37 | indx += 1 38 | } 39 | } 40 | 41 | } 42 | 43 | -------------------------------------------------------------------------------- /algebird-test/src/test/scala/com/twitter/algebird/MinPlusLaws.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird 18 | 19 | import org.scalatest.{ PropSpec, Matchers } 20 | import org.scalatest.prop.PropertyChecks 21 | import org.scalacheck.Arbitrary 22 | import org.scalatest.{ PropSpec, Matchers } 23 | import org.scalatest.prop.PropertyChecks 24 | import org.scalacheck.Arbitrary.arbitrary 25 | import org.scalatest.{ PropSpec, Matchers } 26 | import org.scalatest.prop.PropertyChecks 27 | import org.scalacheck.Properties 28 | import org.scalatest.{ PropSpec, Matchers } 29 | import org.scalatest.prop.PropertyChecks 30 | import org.scalacheck.Gen.choose 31 | 32 | class MinPlusLaws extends PropSpec with PropertyChecks with Matchers { 33 | import BaseProperties._ 34 | 35 | implicit val mpint: Arbitrary[MinPlus[Int]] = Arbitrary { 36 | for { 37 | a <- choose(-1000, 1000) // Don't get too big and overflow 38 | } yield MinPlusValue(a) 39 | } 40 | 41 | property("is a semiring") { 42 | semiringLaws[MinPlus[Int]] 43 | } 44 | 45 | } 46 | -------------------------------------------------------------------------------- /algebird-core/src/main/scala/com/twitter/algebird/legacy/package.scala: -------------------------------------------------------------------------------- 1 | package com.twitter.algebird 2 | 3 | package object legacy { 4 | 5 | /** 6 | * For legacy code (Algebird versions < 0.8.1) that uses CMS we provide this functionally equivalent CMS type, which 7 | * is a CMS with [[Long]]-based keys. 8 | * 9 | * This type is an alias for `TopCMS[Long]` (see [[TopCMS]]). 10 | * 11 | * =Usage= 12 | * 13 | * You typically create instances of this type via [[CountMinSketchMonoid]]. 14 | */ 15 | @deprecated("You should use TopCMS[Long] instead of legacy.CMS", since = "0.8.1") 16 | type CMS = TopCMS[Long] 17 | 18 | /** 19 | * For legacy code (Algebird versions < 0.8.1) that uses CMS we provide this functionally equivalent CMS monoid, which 20 | * allows you to work with (e.g. create) top-% CMS instances with [[Long]]-based keys. 21 | * 22 | * =Usage= 23 | * 24 | * You typically create instances of this type via `CountMinSketchMonoid.apply()`, see example below. 25 | * 26 | * =Example= 27 | * 28 | * {{{ 29 | * import com.twitter.algebird.legacy.CountMinSketchMonoid 30 | * 31 | * // Pre-0.8.1 you would have used `new CountMinSketchMonoid(EPS, DELTA, SEED)` (note the `new`). 32 | * val cmsMonoid: CountMinSketchMonoid = CountMinSketchMonoid(EPS, DELTA, SEED) 33 | * }}} 34 | * 35 | * =Implementation details= 36 | * 37 | * This type is an alias for `TopPctCMSMonoid[Long]` (see [[TopPctCMSMonoid]]). 38 | */ 39 | @deprecated("You should use TopPctCMSMonoid[Long] instead of legacy.CountMinSketchMonoid", since = "0.8.1") 40 | type CountMinSketchMonoid = TopPctCMSMonoid[Long] 41 | 42 | } -------------------------------------------------------------------------------- /project/GenTupleAggregators.scala: -------------------------------------------------------------------------------- 1 | package algebird 2 | 3 | import sbt._ 4 | 5 | object GenTupleAggregators { 6 | def gen(dir: File) = { 7 | val place = dir / "com" / "twitter" / "algebird" / "GeneratedTupleAggregators.scala" 8 | IO.write(place, 9 | """package com.twitter.algebird 10 | 11 | object GeneratedTupleAggregator extends GeneratedTupleAggregator 12 | 13 | trait GeneratedTupleAggregator { 14 | """ + genAggregators(22) + "\n" + "}") 15 | 16 | Seq(place) 17 | } 18 | 19 | def genAggregators(max: Int): String = { 20 | (2 to max).map(i => { 21 | val nums = (1 to i) 22 | val bs = nums.map("B" + _).mkString(", ") 23 | val cs = nums.map("C" + _).mkString(", ") 24 | val aggs = nums.map(x => "Aggregator[A, B%s, C%s]".format(x, x)).mkString(", ") 25 | val prepares = nums.map(x => "aggs._%s.prepare(a)".format(x)).mkString(", ") 26 | val semigroups = nums.map(x => "aggs._%s.semigroup".format(x)).mkString(", ") 27 | val semigroup = "new Tuple%dSemigroup()(%s)".format(i, semigroups) 28 | val present = nums.map(x => "aggs._%s.present(b._%s)".format(x, x)).mkString(", ") 29 | val tupleBs = "Tuple%d[%s]".format(i, bs) 30 | val tupleCs = "Tuple%d[%s]".format(i, cs) 31 | 32 | """ 33 | implicit def from%d[A, %s, %s](aggs: Tuple%d[%s]): Aggregator[A, %s, %s] = { 34 | new Aggregator[A, %s, %s] { 35 | def prepare(a: A) = (%s) 36 | val semigroup = %s 37 | def present(b: %s) = (%s) 38 | } 39 | }""".format(i, bs, cs, i, aggs, tupleBs, tupleCs, 40 | tupleBs, tupleCs, 41 | prepares, 42 | semigroup, 43 | tupleBs, present) 44 | }).mkString("\n") 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /algebird-core/src/main/scala/com/twitter/algebird/statistics/Counter.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2014 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird.statistics 18 | 19 | import java.util.concurrent.atomic.AtomicLong 20 | 21 | /** 22 | * Counter abstraction that can optionally be thread safe 23 | * @author Julien Le Dem 24 | */ 25 | private object Counter { 26 | def apply(threadSafe: Boolean): Counter = if (threadSafe) AtomicCounter() else PlainCounter() 27 | } 28 | 29 | private sealed trait Counter { 30 | def increment: Unit 31 | def add(v: Long): Unit 32 | def get: Long 33 | def toDouble = get.toDouble 34 | override def toString = get.toString 35 | } 36 | 37 | /** thread safe */ 38 | private case class AtomicCounter() extends Counter { 39 | private[this] final val counter = new AtomicLong(0) 40 | override def increment = counter.incrementAndGet 41 | override def add(v: Long) = counter.addAndGet(v) 42 | override def get = counter.get 43 | } 44 | 45 | /** not thread safe */ 46 | private case class PlainCounter() extends Counter { 47 | private[this] final var counter: Long = 0 48 | override def increment = counter += 1 49 | override def add(v: Long) = counter += v 50 | override def get = counter 51 | } -------------------------------------------------------------------------------- /algebird-core/src/main/scala/com/twitter/algebird/mutable/PriorityQueueAggregator.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | package com.twitter.algebird.mutable 17 | 18 | import com.twitter.algebird.MonoidAggregator 19 | import scala.collection.JavaConverters._ 20 | 21 | import java.util.PriorityQueue 22 | 23 | /** 24 | * This gives you the `max` smallest items. If you want the biggest reverse the Ordering. 25 | * Note that PriorityQueue is mutable so it is a good idea to copy this into 26 | * an immutable view before using it, as is done in PriorityQueueToListAggregator 27 | */ 28 | abstract class PriorityQueueAggregator[A, +C](max: Int)(implicit ord: Ordering[A]) 29 | extends MonoidAggregator[A, PriorityQueue[A], C] { 30 | /* 31 | you need to override: 32 | def present(q: PriorityQueue[A]): B 33 | */ 34 | 35 | val monoid = new PriorityQueueMonoid[A](max) 36 | final def prepare(a: A) = monoid.build(a) 37 | } 38 | 39 | /** 40 | * Should probably be your default Top-K implementation 41 | */ 42 | class PriorityQueueToListAggregator[A](max: Int)(implicit ord: Ordering[A]) 43 | extends PriorityQueueAggregator[A, List[A]](max) { 44 | def present(q: PriorityQueue[A]) = q.iterator.asScala.toList.sorted 45 | } 46 | 47 | -------------------------------------------------------------------------------- /algebird-core/src/main/scala/com/twitter/algebird/Operators.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | package com.twitter.algebird 17 | 18 | object Operators { 19 | implicit def toPlus[T: Semigroup](t: T) = new PlusOp(t) 20 | implicit def toMinus[T: Group](t: T) = new MinusOp(t) 21 | implicit def toTimes[T: Ring](t: T) = new TimesOp(t) 22 | implicit def toDiv[T: Field](t: T) = new DivOp(t) 23 | implicit def toRichTraversable[T](t: Traversable[T]) = new RichTraversable(t) 24 | } 25 | 26 | class PlusOp[T: Semigroup](t: T) { 27 | def +(other: T) = implicitly[Semigroup[T]].plus(t, other) 28 | } 29 | 30 | class MinusOp[T: Group](t: T) { 31 | def -(other: T) = implicitly[Group[T]].minus(t, other) 32 | } 33 | 34 | class TimesOp[T: Ring](t: T) { 35 | def *(other: T) = implicitly[Ring[T]].times(t, other) 36 | } 37 | 38 | class DivOp[T: Field](t: T) { 39 | def /(other: T) = implicitly[Field[T]].div(t, other) 40 | } 41 | 42 | class RichTraversable[T](t: TraversableOnce[T]) { 43 | def sumByKey[K, V](implicit ev: <:<[T, (K, V)], sg: Semigroup[V]): Map[K, V] = 44 | MapAlgebra.sumByKey(t.map { _.asInstanceOf[(K, V)] }) 45 | 46 | def monoidSum(implicit monoid: Monoid[T]) = Monoid.sum(t) 47 | def ringProduct(implicit ring: Ring[T]) = Ring.product(t) 48 | } 49 | -------------------------------------------------------------------------------- /algebird-util/src/test/scala/com/twitter/algebird/util/summer/AsyncListMMapSumProperties.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2013 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird.util.summer 18 | 19 | import org.scalatest.{ PropSpec, Matchers } 20 | import org.scalatest.prop.PropertyChecks 21 | 22 | class AsyncListMMapSumProperties extends PropSpec with PropertyChecks with Matchers { 23 | 24 | import AsyncSummerLaws._ 25 | 26 | property("Summing with and without the summer should match") { 27 | forAll { (inputs: List[List[(Int, Long)]], 28 | flushFrequency: FlushFrequency, 29 | bufferSize: BufferSize, 30 | memoryFlushPercent: MemoryFlushPercent) => 31 | val timeOutCounter = Counter("timeOut") 32 | val sizeCounter = Counter("size") 33 | val memoryCounter = Counter("memory") 34 | val insertOp = Counter("insertOp") 35 | val tuplesOut = Counter("tuplesOut") 36 | val summer = new AsyncListMMapSum[Int, Long](bufferSize, 37 | flushFrequency, 38 | memoryFlushPercent, 39 | memoryCounter, 40 | timeOutCounter, 41 | tuplesOut, 42 | insertOp, 43 | sizeCounter, workPool) 44 | assert(summingWithAndWithoutSummerShouldMatch(summer, inputs)) 45 | } 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /algebird-util/src/test/scala/com/twitter/algebird/util/summer/AsyncMapSumProperties.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2013 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird.util.summer 18 | 19 | import org.scalatest.{ PropSpec, Matchers } 20 | import org.scalatest.prop.PropertyChecks 21 | 22 | class AsyncMapSumProperties extends PropSpec with PropertyChecks with Matchers { 23 | import AsyncSummerLaws._ 24 | 25 | property("Summing with and without the summer should match") { 26 | forAll { (inputs: List[List[(Int, Long)]], 27 | flushFrequency: FlushFrequency, 28 | bufferSize: BufferSize, 29 | memoryFlushPercent: MemoryFlushPercent) => 30 | val timeOutCounter = Counter("timeOut") 31 | val sizeCounter = Counter("size") 32 | val memoryCounter = Counter("memory") 33 | val insertOp = Counter("insertOp") 34 | val tuplesOut = Counter("tuplesOut") 35 | 36 | val summer = new AsyncMapSum[Int, Long](bufferSize, 37 | flushFrequency, 38 | memoryFlushPercent, 39 | memoryCounter, 40 | timeOutCounter, 41 | insertOp, 42 | tuplesOut, 43 | sizeCounter, 44 | workPool) 45 | assert(summingWithAndWithoutSummerShouldMatch(summer, inputs)) 46 | } 47 | } 48 | 49 | } 50 | -------------------------------------------------------------------------------- /project/DocGen.scala: -------------------------------------------------------------------------------- 1 | package algebird 2 | 3 | import sbt._ 4 | import Keys._ 5 | 6 | import com.typesafe.sbt.git.GitRunner 7 | import com.typesafe.sbt.SbtGit.GitKeys 8 | import com.typesafe.sbt.SbtSite.{ site, SiteKeys } 9 | import com.typesafe.sbt.SbtGhPages.{ ghpages, GhPagesKeys => ghkeys } 10 | import com.typesafe.sbt.SbtGit.GitKeys.gitRemoteRepo 11 | 12 | object DocGen { 13 | val docDirectory = "target/site" 14 | 15 | def syncLocal = (ghkeys.updatedRepository, GitKeys.gitRunner, streams) map { (repo, git, s) => 16 | cleanSite(repo, git, s) // First, remove 'stale' files. 17 | val rootPath = file(docDirectory) // Now copy files. 18 | IO.copyDirectory(rootPath, repo) 19 | IO.touch(repo / ".nojekyll") 20 | repo 21 | } 22 | 23 | private def cleanSite(dir: File, git: GitRunner, s: TaskStreams): Unit = { 24 | val toClean = IO.listFiles(dir).filterNot(_.getName == ".git").map(_.getAbsolutePath).toList 25 | if(!toClean.isEmpty) 26 | git(("rm" :: "-r" :: "-f" :: "--ignore-unmatch" :: toClean) :_*)(dir, s.log) 27 | () 28 | } 29 | 30 | lazy val unidocSettings: Seq[sbt.Setting[_]] = 31 | site.includeScaladoc(docDirectory) ++ Seq( 32 | scalacOptions in doc <++= (version, baseDirectory in LocalProject("algebird")).map { (v, rootBase) => 33 | val tagOrBranch = if (v.endsWith("-SNAPSHOT")) "develop" else v 34 | val docSourceUrl = "https://github.com/twitter/algebird/tree/" + tagOrBranch + "€{FILE_PATH}.scala" 35 | Seq("-sourcepath", rootBase.getAbsolutePath, "-doc-source-url", docSourceUrl) 36 | }, 37 | Unidoc.unidocDirectory := file(docDirectory), 38 | gitRemoteRepo := "git@github.com:twitter/algebird.git", 39 | ghkeys.synchLocal <<= syncLocal 40 | ) 41 | 42 | lazy val publishSettings = site.settings ++ Unidoc.settings ++ ghpages.settings ++ unidocSettings 43 | } 44 | -------------------------------------------------------------------------------- /algebird-util/src/test/scala/com/twitter/algebird/util/summer/SyncSummingQueueProperties.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2013 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird.util.summer 18 | 19 | import org.scalatest.{ PropSpec, Matchers } 20 | import org.scalatest.prop.PropertyChecks 21 | 22 | class SyncSummingQueueProperties extends PropSpec with PropertyChecks with Matchers { 23 | import AsyncSummerLaws._ 24 | property("Summing with and without the summer should match") { 25 | forAll { (inputs: List[List[(Int, Long)]], 26 | flushFrequency: FlushFrequency, 27 | bufferSize: BufferSize, 28 | memoryFlushPercent: MemoryFlushPercent) => 29 | val timeOutCounter = Counter("timeOut") 30 | val sizeCounter = Counter("size") 31 | val memoryCounter = Counter("memory") 32 | val tuplesIn = Counter("tuplesIn") 33 | val tuplesOut = Counter("tuplesOut") 34 | val putCounter = Counter("put") 35 | val summer = new SyncSummingQueue[Int, Long](bufferSize, 36 | flushFrequency, 37 | memoryFlushPercent, 38 | memoryCounter, 39 | timeOutCounter, 40 | sizeCounter, 41 | putCounter, 42 | tuplesIn, 43 | tuplesOut) 44 | assert(summingWithAndWithoutSummerShouldMatch(summer, inputs)) 45 | } 46 | } 47 | 48 | } 49 | -------------------------------------------------------------------------------- /algebird-util/src/test/scala/com/twitter/algebird/util/UtilAlgebraProperties.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2013 Twitter Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); you may 5 | * not use this file except in compliance with the License. You may obtain 6 | * a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird.util 18 | 19 | import com.twitter.algebird.MonadLaws.monadLaws 20 | import com.twitter.util.{ Await, Future, Throw, Return, Try } 21 | import org.scalatest.{ PropSpec, Matchers } 22 | import org.scalatest.prop.PropertyChecks 23 | import org.scalacheck.{ Arbitrary, Properties } 24 | 25 | class UtilAlgebraProperties extends PropSpec with PropertyChecks with Matchers { 26 | import UtilAlgebras._ 27 | 28 | def toOption[T](f: Future[T]): Option[T] = 29 | try { 30 | Some(Await.result(f)) 31 | } catch { 32 | case _: Exception => None 33 | } 34 | 35 | implicit def futureA[T: Arbitrary]: Arbitrary[Future[T]] = 36 | Arbitrary { 37 | Arbitrary.arbitrary[T].map { l => Future.value(l) } 38 | } 39 | 40 | implicit def returnA[T: Arbitrary]: Arbitrary[Try[T]] = 41 | Arbitrary { 42 | Arbitrary.arbitrary[T].map { l => Return(l) } 43 | } 44 | 45 | property("futureMonad") { 46 | monadLaws[Future, Int, String, Long] { (f1, f2) => 47 | toOption(f1) == toOption(f2) 48 | } 49 | } 50 | 51 | property("tryMonad") { 52 | monadLaws[Try, Int, String, Long]() 53 | } 54 | 55 | } 56 | -------------------------------------------------------------------------------- /algebird-test/src/main/scala/com/twitter/algebird/StatefulSummerLaws.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird 18 | 19 | object StatefulSummerLaws { 20 | import BaseProperties.isNonZero 21 | 22 | // Law1: 23 | def zeroEquiv[V: Semigroup: Equiv](v0: Option[V], v1: Option[V]): Boolean = { 24 | val v0NonZero = v0.exists(isNonZero(_)) 25 | val v1NonZero = v1.exists(isNonZero(_)) 26 | if (!(v0NonZero || v1NonZero)) { 27 | // neither are non-zero 28 | true 29 | } else { 30 | (for (a <- v0; b <- v1; eq = Equiv[V].equiv(a, b)) yield eq).getOrElse(false) 31 | } 32 | } 33 | def sumIsPreserved[V: Semigroup: Equiv](summer: StatefulSummer[V], items: Iterable[V]): Boolean = { 34 | summer.flush 35 | val sg = Semigroup.sumOption(items) 36 | val wsummer = Monoid.plus(Monoid.sum(items.map { summer.put(_) }.filter { _.isDefined }), summer.flush) 37 | zeroEquiv(sg, wsummer) && summer.isFlushed 38 | } 39 | // Law 2: 40 | def isFlushedIsConsistent[V](summer: StatefulSummer[V], items: Iterable[V]): Boolean = { 41 | items.forall { v => 42 | summer.put(v) 43 | (summer.isFlushed == summer.flush.isEmpty) && 44 | // Now flush should empty 45 | summer.isFlushed && 46 | summer.flush.isEmpty 47 | } 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /algebird-test/src/main/scala/com/twitter/algebird/FunctorLaws.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2014 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird 18 | 19 | import org.scalacheck.{ Arbitrary, Prop } 20 | import org.scalacheck.Prop.forAll 21 | 22 | /** 23 | * Basic Functor laws, useful for testing any functor. 24 | */ 25 | object FunctorLaws { 26 | import BaseProperties.{ HigherEq, DefaultHigherEq } 27 | 28 | def identityLaw[M[_], V]( 29 | eq: HigherEq[M] = new DefaultHigherEq[M])( 30 | implicit functor: Functor[M], 31 | arb: Arbitrary[M[V]]): Prop = 32 | forAll { (mv: M[V]) => eq(functor.map(mv) { x => x }, mv) } 33 | 34 | def composeLaw[M[_], T, U, V]( 35 | eq: HigherEq[M] = new DefaultHigherEq[M])( 36 | implicit functor: Functor[M], 37 | arb: Arbitrary[M[T]], 38 | arbFn1: Arbitrary[T => U], 39 | arbFn2: Arbitrary[U => V]): Prop = 40 | forAll { (mt: M[T], fn1: T => U, fn2: U => V) => 41 | eq( 42 | functor.map(mt)(fn1 andThen fn2), 43 | functor.map(functor.map(mt)(fn1))(fn2)) 44 | } 45 | 46 | def functorLaws[M[_], T, U, V]( 47 | eq: HigherEq[M] = new DefaultHigherEq[M])( 48 | implicit functor: Functor[M], 49 | arbMt: Arbitrary[M[T]], 50 | arbMv: Arbitrary[M[V]], 51 | arbFn1: Arbitrary[T => U], 52 | arbFn2: Arbitrary[U => V]): Prop = 53 | identityLaw[M, V](eq) && composeLaw[M, T, U, V](eq) 54 | } 55 | 56 | -------------------------------------------------------------------------------- /algebird-test/src/test/scala/com/twitter/algebird/ResetTest.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird 18 | 19 | import org.scalatest.{ PropSpec, Matchers } 20 | import org.scalatest.prop.PropertyChecks 21 | import org.scalacheck.Arbitrary 22 | 23 | class ResetTest extends PropSpec with PropertyChecks with Matchers { 24 | import BaseProperties._ 25 | 26 | implicit def resetArb[T: Arbitrary]: Arbitrary[ResetState[T]] = Arbitrary { 27 | Arbitrary.arbitrary[T].map { t => 28 | if (scala.math.random < 0.1) { 29 | ResetValue(t) 30 | } else { 31 | SetValue(t) 32 | } 33 | } 34 | } 35 | 36 | property("ResetState[Int] forms a Monoid") { 37 | monoidLaws[ResetState[Int]] 38 | } 39 | 40 | property("ResetState[String] forms a Monoid") { 41 | monoidLaws[ResetState[String]] 42 | } 43 | 44 | property("ResetState[Int] works as expected") { 45 | forAll { (a: ResetState[Int], b: ResetState[Int], c: ResetState[Int]) => 46 | val result = Monoid.plus(Monoid.plus(a, b), c) 47 | assert(((a, b, c) match { 48 | case (SetValue(x), SetValue(y), SetValue(z)) => SetValue(x + y + z) 49 | case (ResetValue(x), SetValue(y), SetValue(z)) => ResetValue(x + y + z) 50 | case (_, ResetValue(y), SetValue(z)) => ResetValue(y + z) 51 | case (_, _, ResetValue(z)) => ResetValue(z) 52 | }) == result) 53 | } 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /algebird-core/src/main/scala/com/twitter/algebird/StatefulSummer.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird 18 | 19 | /** 20 | * A Stateful summer is something that is potentially more efficient 21 | * (a buffer, a cache, etc...) that has the same result as a sum: 22 | * Law 1: Semigroup.sumOption(items) == 23 | * (Monoid.plus(items.map { stateful.put(_) }.filter { _.isDefined }, stateful.flush) && 24 | * stateful.isFlushed) 25 | * Law 2: isFlushed == flush.isEmpty 26 | * @author Oscar Boykin 27 | */ 28 | trait StatefulSummer[V] extends Buffered[V, V] { 29 | def semigroup: Semigroup[V] 30 | } 31 | 32 | /** 33 | * Sum the entire iterator one item at a time. Only emits on flush 34 | * you should probably prefer BufferedSumAll 35 | */ 36 | class SumAll[V](implicit override val semigroup: Semigroup[V]) extends StatefulSummer[V] { 37 | var summed: Option[V] = None 38 | def put(item: V) = { 39 | summed = Semigroup.plus(summed, Some(item)) 40 | None 41 | } 42 | def flush = { 43 | val res = summed 44 | summed = None 45 | res 46 | } 47 | def isFlushed = summed.isEmpty 48 | } 49 | 50 | class BufferedSumAll[V](size: Int)(implicit override val semigroup: Semigroup[V]) 51 | extends ArrayBufferedOperation[V, V](size) 52 | with StatefulSummer[V] 53 | with BufferedReduce[V] { 54 | 55 | def operate(nonEmpty: Seq[V]): V = semigroup.sumOption(nonEmpty).get 56 | } 57 | -------------------------------------------------------------------------------- /algebird-core/src/main/scala/com/twitter/algebird/Functor.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2014 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | package com.twitter.algebird 17 | 18 | import scala.annotation.implicitNotFound 19 | 20 | /** 21 | * Simple implementation of a Functor type-class. 22 | * 23 | * Laws Functors must follow: 24 | * map(m)(id) == m 25 | * map(m)(f andThen g) == map(map(m)(f))(g) 26 | */ 27 | @implicitNotFound(msg = "Cannot find Functor type class for ${M}") 28 | trait Functor[M[_]] { 29 | def map[T, U](m: M[T])(fn: (T) => U): M[U] 30 | } 31 | 32 | /** 33 | * For use from Java/minimizing code bloat in scala 34 | */ 35 | abstract class AbstractFunctor[M[_]] extends Functor[M] 36 | 37 | /** 38 | * Follows the type-class pattern for the Functor trait 39 | */ 40 | object Functor { 41 | /** Get the Functor for a type, e.g: Functor[List] */ 42 | def apply[M[_]](implicit functor: Functor[M]): Functor[M] = functor 43 | def map[M[_], T, U](m: M[T])(fn: (T) => U)(implicit functor: Functor[M]) = functor.map(m)(fn) 44 | 45 | implicit def operators[A, M[_]](m: M[A])(implicit functor: Functor[M]) = 46 | new FunctorOperators(m)(functor) 47 | } 48 | 49 | /** 50 | * This enrichment allows us to use our Functor instances in for expressions: 51 | * if (import Functor._) has been done 52 | */ 53 | class FunctorOperators[A, M[_]](m: M[A])(implicit functor: Functor[M]) { 54 | // This is called fmap in haskell 55 | def map[U](fn: (A) => U): M[U] = functor.map(m)(fn) 56 | } 57 | -------------------------------------------------------------------------------- /algebird-core/src/main/scala/com/twitter/algebird/RightFolded.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird 18 | 19 | /** 20 | * This is an associative, but not commutative monoid 21 | * Also, you must start on the right, with a value, and all subsequent RightFolded must 22 | * be RightFoldedToFold objects or zero 23 | * 24 | * If you add two Folded values together, you always get the one on the left, 25 | * so this forms a kind of reset of the fold. 26 | */ 27 | object RightFolded { 28 | def monoid[In, Out](foldfn: (In, Out) => Out) = 29 | new Monoid[RightFolded[In, Out]] { 30 | 31 | val zero = RightFoldedZero 32 | 33 | def plus(left: RightFolded[In, Out], right: RightFolded[In, Out]) = left match { 34 | case RightFoldedValue(_) => left 35 | case RightFoldedZero => right 36 | case RightFoldedToFold(lList) => right match { 37 | case RightFoldedZero => RightFoldedToFold(lList) 38 | case RightFoldedValue(vr) => RightFoldedValue(lList.foldRight(vr)(foldfn)) 39 | case RightFoldedToFold(rList) => RightFoldedToFold(lList ++ rList) 40 | } 41 | } 42 | } 43 | } 44 | 45 | sealed abstract class RightFolded[+In, +Out] 46 | case object RightFoldedZero extends RightFolded[Nothing, Nothing] 47 | case class RightFoldedValue[+Out](v: Out) extends RightFolded[Nothing, Out] 48 | case class RightFoldedToFold[+In](in: List[In]) extends RightFolded[In, Nothing] 49 | -------------------------------------------------------------------------------- /algebird-core/src/main/scala/com/twitter/algebird/MurmurHash.scala: -------------------------------------------------------------------------------- 1 | package com.twitter.algebird 2 | 3 | import java.nio._ 4 | 5 | case class MurmurHash128(seed: Long) extends AnyVal { 6 | def apply(buffer: ByteBuffer, offset: Int, length: Int): (Long, Long) = { 7 | val longs = CassandraMurmurHash.hash3_x64_128(buffer, offset, length, seed) 8 | (longs(0), longs(1)) 9 | } 10 | 11 | def apply(bytes: Array[Byte]): (Long, Long) = apply(ByteBuffer.wrap(bytes), 0, bytes.length) 12 | def apply(maxBytes: Int, fn: ByteBuffer => Unit): (Long, Long) = { 13 | val buffer = ByteBuffer.allocate(maxBytes) 14 | fn(buffer) 15 | apply(buffer, 0, maxBytes) 16 | } 17 | def apply(array: Array[Char]): (Long, Long) = apply(array.size * 2, { _.asCharBuffer.put(array) }) 18 | def apply(array: Array[Short]): (Long, Long) = apply(array.size * 2, { _.asShortBuffer.put(array) }) 19 | def apply(array: Array[Int]): (Long, Long) = apply(array.size * 4, { _.asIntBuffer.put(array) }) 20 | def apply(array: Array[Float]): (Long, Long) = apply(array.size * 4, { _.asFloatBuffer.put(array) }) 21 | def apply(array: Array[Long]): (Long, Long) = apply(array.size * 8, { _.asLongBuffer.put(array) }) 22 | def apply(array: Array[Double]): (Long, Long) = apply(array.size * 8, { _.asDoubleBuffer.put(array) }) 23 | 24 | def apply(value: Char): (Long, Long) = apply(2, { _.asCharBuffer.put(value) }) 25 | def apply(value: Short): (Long, Long) = apply(2, { _.asShortBuffer.put(value) }) 26 | def apply(value: Int): (Long, Long) = apply(4, { _.asIntBuffer.put(value) }) 27 | def apply(value: Float): (Long, Long) = apply(4, { _.asFloatBuffer.put(value) }) 28 | def apply(value: Long): (Long, Long) = apply(8, { _.asLongBuffer.put(value) }) 29 | def apply(value: Double): (Long, Long) = apply(8, { _.asDoubleBuffer.put(value) }) 30 | 31 | def apply(string: CharSequence): (Long, Long) = apply(string.length * 2, { buffer => 32 | val charBuffer = buffer.asCharBuffer 33 | 0.to(string.length - 1).foreach{ i => charBuffer.put(string.charAt(i)) } 34 | }) 35 | } 36 | -------------------------------------------------------------------------------- /algebird-core/src/main/scala/com/twitter/algebird/VectorSpace.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird 18 | 19 | import java.lang.{ Integer => JInt, Short => JShort, Long => JLong, Float => JFloat, Double => JDouble, Boolean => JBool } 20 | 21 | import scala.annotation.implicitNotFound 22 | /** 23 | * This class represents a vector space. For the required properties see: 24 | * 25 | * http://en.wikipedia.org/wiki/Vector_space#Definition 26 | * 27 | */ 28 | object VectorSpace { 29 | def scale[F, C[_]](v: F, c: C[F])(implicit vs: VectorSpace[F, C]): C[F] = vs.scale(v, c) 30 | def from[F, C[_]](scaleFn: (F, C[F]) => C[F])(implicit fField: Field[F], cGroup: Group[C[F]]) = new VectorSpace[F, C] { 31 | def field = fField 32 | def group = cGroup 33 | def scale(v: F, c: C[F]) = if (field.isNonZero(v)) scaleFn(v, c) else cGroup.zero 34 | } 35 | 36 | // Implicits 37 | implicit def indexedSeqSpace[T: Field] = 38 | from[T, IndexedSeq]{ (s, seq) => seq.map(Ring.times(s, _)) } 39 | 40 | implicit def mapSpace[K, T: Field] = 41 | from[T, ({ type x[a] = Map[K, a] })#x] { (s, m) => m.mapValues(Ring.times(s, _)) } 42 | 43 | // TODO: add implicits for java lists, arrays, and options 44 | } 45 | 46 | @implicitNotFound(msg = "Cannot find VectorSpace type class for Container: ${C} and Field: ${F}") 47 | trait VectorSpace[F, C[_]] extends java.io.Serializable { 48 | implicit def field: Field[F] 49 | implicit def group: Group[C[F]] 50 | def scale(v: F, c: C[F]): C[F] 51 | } 52 | -------------------------------------------------------------------------------- /algebird-test/src/test/scala/com/twitter/algebird/AggregationMonoids.scala: -------------------------------------------------------------------------------- 1 | package com.twitter.algebird 2 | 3 | import org.scalacheck.{ Gen, Arbitrary } 4 | import org.scalatest.{ PropSpec, Matchers } 5 | import org.scalatest.prop.PropertyChecks 6 | 7 | class AggregationMonoidSpecification extends PropSpec with PropertyChecks with Matchers { 8 | import BaseProperties._ 9 | import Gen.choose 10 | 11 | def approxEq(f1: Double, f2: Double) = (scala.math.abs(f1 - f2) / scala.math.abs(f2)) < 1e-10 12 | 13 | implicit val dcgen = Arbitrary { 14 | for { 15 | a <- choose(-1e100, 1e100) // Don't get too big and overflow 16 | b <- choose(-1e100, 1e100) // Don't get too big and overflow 17 | } yield DecayedValue(a, b) 18 | } 19 | 20 | implicit val decayedMonoid = DecayedValue.monoidWithEpsilon(0.001) 21 | 22 | property("DecayedValue Monoid laws") { 23 | monoidLawsEq[DecayedValue] { (dvl, dvr) => 24 | approxEq(dvl.value, dvr.value) && (dvl.scaledTime == dvr.scaledTime) 25 | } 26 | } 27 | 28 | implicit val avgen = Arbitrary { 29 | for { 30 | cnt <- choose(Int.MinValue.toLong, Int.MaxValue.toLong) 31 | v <- choose(-1e100, 1e100) // Don't get too big and overflow 32 | } yield AveragedValue(cnt, v) 33 | } 34 | 35 | property("AveragedValue Group laws") { 36 | groupLawsEq[AveragedValue] { (avl, avr) => 37 | ((avl.count == 0L) && (avr.count == 0L)) || { 38 | approxEq(avl.value, avr.value) && (avl.count == avr.count) 39 | } 40 | } 41 | } 42 | 43 | implicit val momentsgen = Arbitrary { 44 | for { 45 | m0 <- choose(1L, Int.MaxValue.toLong) 46 | m1 <- choose(-1e50, 1e50) 47 | m2 <- choose(0, 1e50) 48 | m3 <- choose(-1e10, 1e50) 49 | m4 <- choose(0, 1e50) 50 | } yield new Moments(m0, m1, m2, m3, m4) 51 | } 52 | 53 | property("Moments Group laws") { 54 | groupLawsEq[Moments] { (ml, mr) => 55 | (ml.m0 == mr.m0) && 56 | approxEq(ml.m1, mr.m1) && 57 | approxEq(ml.m2, mr.m2) && 58 | approxEq(ml.m3, mr.m3) && 59 | approxEq(ml.m4, mr.m4) 60 | } 61 | } 62 | } 63 | -------------------------------------------------------------------------------- /algebird-core/src/main/scala/com/twitter/algebird/BufferedOperation.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird 18 | 19 | import scala.collection.mutable.ArrayBuffer 20 | 21 | /** 22 | * Represents something that consumes I and may emit O. Has some internal 23 | * state that may be used to improve performance. 24 | * Generally used to model folds or reduces (see BufferedReduce) 25 | */ 26 | trait Buffered[I, O] extends java.io.Serializable { 27 | def put(i: I): Option[O] 28 | def flush: Option[O] 29 | def isFlushed: Boolean 30 | } 31 | 32 | abstract class ArrayBufferedOperation[I, O](size: Int) extends Buffered[I, O] { 33 | def operate(nonEmpty: Seq[I]): O 34 | 35 | require(size > 0, "buffer <= 0 not allowed") 36 | 37 | private val buffer = new ArrayBuffer[I](size) 38 | 39 | def put(item: I): Option[O] = { 40 | buffer += item 41 | if (buffer.size >= size) flush 42 | else None 43 | } 44 | 45 | def flush: Option[O] = 46 | if (buffer.isEmpty) None 47 | else { 48 | val res = operate(buffer) 49 | buffer.clear 50 | Some(res) 51 | } 52 | 53 | def isFlushed = buffer.isEmpty 54 | } 55 | 56 | /** 57 | * This never emits on put, you must call flush 58 | * designed to be use in the stackable pattern with ArrayBufferedOperation 59 | */ 60 | trait BufferedReduce[V] extends Buffered[V, V] { 61 | abstract override def put(item: V) = { 62 | val res = super.put(item) 63 | // avoiding closures for performance critical code: 64 | if (res.isDefined) put(res.get) 65 | else None 66 | } 67 | } 68 | -------------------------------------------------------------------------------- /algebird-util/src/main/scala/com/twitter/algebird/util/UtilAlgebras.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2013 Twitter Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); you may 5 | * not use this file except in compliance with the License. You may obtain 6 | * a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird.util 18 | 19 | import com.twitter.algebird._ 20 | import com.twitter.util.{ Future, Return, Try } 21 | 22 | object UtilAlgebras { 23 | implicit val futureMonad: Monad[Future] = new Monad[Future] { 24 | def apply[T](v: T) = Future.value(v) 25 | override def map[T, U](m: Future[T])(fn: T => U) = m.map(fn) 26 | def flatMap[T, U](m: Future[T])(fn: T => Future[U]) = m.flatMap(fn) 27 | } 28 | implicit val tryMonad: Monad[Try] = new Monad[Try] { 29 | def apply[T](v: T) = Return(v) 30 | override def map[T, U](m: Try[T])(fn: T => U) = m.map(fn) 31 | def flatMap[T, U](m: Try[T])(fn: T => Try[U]) = m.flatMap(fn) 32 | } 33 | 34 | implicit def futureSemigroup[T: Semigroup]: Semigroup[Future[T]] = new MonadSemigroup[T, Future] 35 | implicit def futureMonoid[T: Monoid]: Monoid[Future[T]] = new MonadMonoid[T, Future] 36 | implicit def futureGroup[T: Group]: Group[Future[T]] = new MonadGroup[T, Future] 37 | implicit def futureRing[T: Ring]: Ring[Future[T]] = new MonadRing[T, Future] 38 | implicit def futureField[T: Field]: Field[Future[T]] = new MonadField[T, Future] 39 | 40 | implicit def trySemigroup[T: Semigroup]: Semigroup[Try[T]] = new MonadSemigroup[T, Try] 41 | implicit def tryMonoid[T: Monoid]: Monoid[Try[T]] = new MonadMonoid[T, Try] 42 | implicit def tryGroup[T: Group]: Group[Try[T]] = new MonadGroup[T, Try] 43 | implicit def tryRing[T: Ring]: Ring[Try[T]] = new MonadRing[T, Try] 44 | implicit def tryField[T: Field]: Field[Try[T]] = new MonadField[T, Try] 45 | } 46 | -------------------------------------------------------------------------------- /algebird-test/src/test/scala/com/twitter/algebird/SpaceSaverTest.scala: -------------------------------------------------------------------------------- 1 | package com.twitter.algebird 2 | 3 | import org.scalatest._ 4 | import org.scalatest.prop.PropertyChecks 5 | import org.scalacheck.{ Gen, Arbitrary } 6 | 7 | class SpaceSaverLaws extends PropSpec with PropertyChecks with Matchers { 8 | import BaseProperties._ 9 | 10 | // limit sizes to 100 to avoid large data structures in tests 11 | property("SpaceSaver is a Semigroup") { 12 | forAll(Gen.choose(2, 100)){ capacity => 13 | forAll(Gen.choose(1, 100)){ range => 14 | 15 | // need a non-uniform distro 16 | implicit val ssGenOne: Arbitrary[SSOne[Int]] = Arbitrary { 17 | for (key <- Gen.frequency((1 to range).map{ x => (x * x, x: Gen[Int]) }: _*)) yield SpaceSaver(capacity, key).asInstanceOf[SSOne[Int]] 18 | } 19 | implicit def ssGen(implicit sg: Semigroup[SpaceSaver[Int]]): Arbitrary[SpaceSaver[Int]] = Arbitrary { 20 | Gen.oneOf( 21 | Arbitrary.arbitrary[SSOne[Int]], 22 | Gen.nonEmptyContainerOf[List, SSOne[Int]](Arbitrary.arbitrary[SSOne[Int]]).map(_.reduce(sg.plus))) 23 | } 24 | 25 | commutativeSemigroupLawsEq[SpaceSaver[Int]] { (left, right) => (left consistentWith right) && (right consistentWith left) } 26 | } 27 | } 28 | } 29 | } 30 | 31 | class SpaceSaverTest extends WordSpec with Matchers { 32 | "SpaceSaver" should { 33 | "produce a top 20 with exact bounds" in { 34 | val gen = Gen.frequency((1 to 100).map{ x => (x * x, x: Gen[Int]) }: _*) 35 | val items = (1 to 1000).map{ x => gen.sample.get } 36 | val exactCounts = items.groupBy(identity).mapValues(_.size) 37 | 38 | // simulate a distributed system with 10 mappers and 1 reducer 39 | val sg = new SpaceSaverSemigroup[Int] 40 | items.grouped(10).map{ 41 | _ 42 | .iterator 43 | .map(SpaceSaver(40, _)) 44 | .reduce(sg.plus) 45 | } 46 | .reduce(sg.plus) 47 | .topK(20) 48 | .foreach { 49 | case (item, approx, guarantee) => 50 | // println("item " + item + " : " + approx.min + " <= " + exactCounts(item) + " <= " + approx.max) 51 | assert(approx ~ exactCounts(item)) 52 | } 53 | } 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /algebird-core/src/main/scala/com/twitter/algebird/AffineFunction.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird 18 | 19 | /** 20 | * Represents functions of the kind: 21 | * f(x) = slope * x + intercept 22 | */ 23 | case class AffineFunction[R](slope: R, intercept: R) extends java.io.Serializable { 24 | def toFn(implicit ring: Ring[R]): Function1[R, R] = { x => this.apply(x)(ring) } 25 | def apply(x: R)(implicit ring: Ring[R]) = ring.plus(ring.times(slope, x), intercept) 26 | } 27 | 28 | /** 29 | * This feeds the value in on the LEFT!!! This may seem counter intuitive, but 30 | * with this approach, a stream/iterator which is summed will have the same output 31 | * as applying the function one at a time in order to the input. 32 | * If we did the "lexigraphically correct" thing, which might be (f+g)(x) = f(g(x)) 33 | * then we would wind up reversing the list in the sum. 34 | * (f1 + f2)(x) = f2(f1(x)) so that: 35 | * listOfFn.foldLeft(x) { (v, fn) => fn(v) } = (Monoid.sum(listOfFn))(x) 36 | */ 37 | class AffineFunctionMonoid[R](implicit ring: Ring[R]) extends Monoid[AffineFunction[R]] { 38 | lazy val zero = AffineFunction[R](ring.one, ring.zero) 39 | def plus(f: AffineFunction[R], g: AffineFunction[R]) = { 40 | // (f+g)(x) = g(f(x)) 41 | // g(f(x)) = g(a*x+b) = c*(a*x+b) + d = (c*a)*x + (c*b + d) 42 | val newSlope = ring.times(g.slope, f.slope) 43 | val newIntercept = ring.plus(ring.times(g.slope, f.intercept), g.intercept) 44 | AffineFunction(newSlope, newIntercept) 45 | } 46 | } 47 | 48 | object AffineFunction extends java.io.Serializable { 49 | implicit def monoid[R: Ring]: Monoid[AffineFunction[R]] = new AffineFunctionMonoid[R] 50 | } 51 | -------------------------------------------------------------------------------- /algebird-test/src/test/scala/com/twitter/algebird/DecayedVectorProperties.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird 18 | 19 | import org.scalatest.{ PropSpec, Matchers } 20 | import org.scalatest.prop.PropertyChecks 21 | import org.scalacheck.{ Gen, Arbitrary } 22 | 23 | class DecayedVectorProperties extends PropSpec with PropertyChecks with Matchers { 24 | import BaseProperties._ 25 | 26 | implicit val mpint: Arbitrary[DecayedVector[({ type x[a] = Map[Int, a] })#x]] = Arbitrary { 27 | for { 28 | t <- Gen.choose(1e-5, 200.0) // Not too high so as to avoid numerical issues 29 | m <- Arbitrary.arbitrary[Map[Int, Double]] 30 | } yield DecayedVector.forMap(m, t) 31 | } 32 | 33 | // TODO: we won't need this when we have an Equatable trait 34 | def decayedMapEqFn(a: DecayedVector[({ type x[a] = Map[Int, a] })#x], b: DecayedVector[({ type x[a] = Map[Int, a] })#x]) = { 35 | def beCloseTo(a: Double, b: Double, eps: Double = 1e-10) = 36 | a == b || (math.abs(a - b) / math.abs(a)) < eps || (a.isInfinite && b.isInfinite) || a.isNaN || b.isNaN 37 | val mapsAreClose = (a.vector.keySet ++ b.vector.keySet).forall { key => 38 | (a.vector.get(key), b.vector.get(key)) match { 39 | case (Some(aVal), Some(bVal)) => beCloseTo(aVal, bVal) 40 | case (Some(aVal), None) => beCloseTo(aVal, 0.0) 41 | case (None, Some(bVal)) => beCloseTo(bVal, 0.0) 42 | case _ => true 43 | } 44 | } 45 | val timesAreClose = beCloseTo(a.scaledTime, b.scaledTime) 46 | mapsAreClose && timesAreClose 47 | } 48 | 49 | property("DecayedVector[Map[Int, _]] is a monoid") { 50 | monoidLawsEq[DecayedVector[({ type x[a] = Map[Int, a] })#x]](decayedMapEqFn) 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /algebird-core/src/main/scala/com/twitter/algebird/Combinator.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | package com.twitter.algebird 17 | 18 | import scala.annotation.tailrec 19 | import scala.annotation.implicitNotFound 20 | import scala.math.Equiv 21 | 22 | /** 23 | * This is a combinator on semigroups, after you do the plus, you transform B with a fold function 24 | * This will not be valid for all fold functions. You need to prove that it is still associative. 25 | * 26 | * Clearly only values of (a,b) are valid if fold(a,b) == b, so keep that in mind. 27 | * 28 | * I have not yet found a sufficient condition on (A,B) => B that makes it correct 29 | * Clearly a (trivial) constant function {(l,r) => r} works. 30 | * Also, if B is List[T], and (l:A,r:List[T]) = r.sortBy(fn(l)) 31 | * this works as well (due to the associativity on A, and the fact that the list never loses data). 32 | * 33 | * For approximate lists (like top-K applications) this might work (or be close enough to associative 34 | * that for approximation algorithms it is fine), and in fact, that is the main motivation of this code: 35 | * Produce some ordering in A, and use it to do sorted-topK on the list in B. 36 | * 37 | * Seems like an open topic here.... you are obliged to think on your own about this. 38 | */ 39 | class SemigroupCombinator[A: Semigroup, B: Semigroup](fold: (A, B) => B) extends Semigroup[(A, B)] { 40 | def plus(left: (A, B), right: (A, B)) = { 41 | val first = Semigroup.plus(left._1, right._1) 42 | (first, fold(first, Semigroup.plus(left._2, right._2))) 43 | } 44 | } 45 | 46 | class MonoidCombinator[A: Monoid, B: Monoid](fold: (A, B) => B) extends SemigroupCombinator[A, B](fold) with Monoid[(A, B)] { 47 | def zero = (Monoid.zero[A], Monoid.zero[B]) 48 | } 49 | -------------------------------------------------------------------------------- /algebird-util/src/test/scala/com/twitter/algebird/util/summer/HeavyHittersCachingSummerProperties.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2013 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird.util.summer 18 | 19 | import org.scalatest.{ PropSpec, Matchers } 20 | import org.scalatest.prop.PropertyChecks 21 | 22 | class HeavyHittersCachingSummerProperties extends PropSpec with PropertyChecks with Matchers { 23 | import AsyncSummerLaws._ 24 | 25 | property("Summing with and without the summer should match") { 26 | forAll { (inputs: List[List[(Int, Long)]], 27 | flushFrequency: FlushFrequency, 28 | bufferSize: BufferSize, 29 | memoryFlushPercent: MemoryFlushPercent) => 30 | val timeOutCounter = Counter("timeOut") 31 | val sizeCounter = Counter("size") 32 | val memoryCounter = Counter("memory") 33 | val insertOp = Counter("insertOp") 34 | val insertFails = Counter("insertFails") 35 | val tuplesIn = Counter("tuplesIn") 36 | val tuplesOut = Counter("tuplesOut") 37 | 38 | val summer = new AsyncListSum[Int, Long](bufferSize, 39 | flushFrequency, 40 | memoryFlushPercent, 41 | memoryCounter, 42 | timeOutCounter, 43 | insertOp, 44 | insertFails, 45 | sizeCounter, 46 | tuplesIn, 47 | tuplesOut, 48 | workPool, 49 | Compact(false), 50 | CompactionSize(0)) 51 | val heavyHittersCachingSummer = HeavyHittersCachingSummer[Int, Long](flushFrequency, 52 | memoryFlushPercent, 53 | memoryCounter, 54 | timeOutCounter, 55 | tuplesOut, 56 | insertOp, 57 | sizeCounter, 58 | summer) 59 | assert(summingWithAndWithoutSummerShouldMatch(heavyHittersCachingSummer, inputs)) 60 | } 61 | } 62 | 63 | } 64 | -------------------------------------------------------------------------------- /algebird-util/src/main/scala/com/twitter/algebird/util/PromiseLinkMonoid.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | package com.twitter.algebird.util 17 | 18 | import com.twitter.algebird._ 19 | import com.twitter.util.Promise 20 | 21 | /** 22 | * This Monoid allows code to depend on the result of computation asynchronously. 23 | * This is a slightly less general version of the TunnelMonoid. See the documentation 24 | * for TunnelMonoid for general motivation. NOTE: the Promise will be fulfilled with 25 | * the value just before the PromiseLink is calculated. 26 | */ 27 | class PromiseLinkMonoid[V](monoid: Monoid[V]) extends Monoid[PromiseLink[V]] { //TODo(jcoveney) rename PromiseLink 28 | def zero = PromiseLink(new Promise, monoid.zero) 29 | 30 | def plus(older: PromiseLink[V], newer: PromiseLink[V]): PromiseLink[V] = { 31 | val (PromiseLink(p1, v1), PromiseLink(p2, v2)) = (older, newer) 32 | p2.foreach { futureV => 33 | Tunnel.properPromiseUpdate(p1, monoid.plus(futureV, v2)) 34 | } 35 | PromiseLink(p2, monoid.plus(v1, v2)) 36 | } 37 | 38 | override def isNonZero(v: PromiseLink[V]) = monoid.isNonZero(v.value) 39 | } 40 | 41 | /** 42 | * This class allows code to depends on the data that a value will be combined with, 43 | * fulfilling the Promise with the value just before the value is added in. 44 | */ 45 | case class PromiseLink[V](promise: Promise[V], value: V) { 46 | def completeWithStartingValue(startingV: V)(implicit monoid: Monoid[V]): V = { 47 | Tunnel.properPromiseUpdate(promise, startingV) 48 | monoid.plus(startingV, value) 49 | } 50 | } 51 | 52 | object PromiseLink { 53 | implicit def monoid[V](implicit innerMonoid: Monoid[V]): PromiseLinkMonoid[V] = 54 | new PromiseLinkMonoid[V](innerMonoid) 55 | 56 | def toPromiseLink[V](value: V) = PromiseLink(new Promise, value) 57 | } -------------------------------------------------------------------------------- /algebird-core/src/main/scala/com/twitter/algebird/SummingIterator.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird 18 | 19 | /** 20 | * @author Oscar Boykin 21 | */ 22 | 23 | import java.util.concurrent.ArrayBlockingQueue 24 | 25 | import java.util.{ LinkedHashMap => JLinkedHashMap, Map => JMap } 26 | import scala.collection.mutable.{ Map => MMap } 27 | import scala.collection.JavaConverters._ 28 | import scala.annotation.tailrec 29 | 30 | /** 31 | * Creates an Iterator that emits partial sums of an input Iterator[V]. 32 | * Generally this is useful to change from processing individiual V's to 33 | * possibly blocks of V @see SummingQueue or a cache of recent Keys in 34 | * a V=Map[K,W] case: @see SummingCache 35 | */ 36 | object SummingIterator { 37 | def apply[V](summer: StatefulSummer[V], it: Iterator[V]): SummingIterator[V] = 38 | new SummingIterator(summer, it) 39 | 40 | implicit def enrich[V](it: Iterator[V]): Enriched[V] = new Enriched(it) 41 | /** 42 | * Here to add enrichments to Iterator 43 | */ 44 | class Enriched[V](it: Iterator[V]) { 45 | def sumWith(summer: StatefulSummer[V]): SummingIterator[V] = SummingIterator(summer, it) 46 | } 47 | } 48 | 49 | class SummingIterator[V](summer: StatefulSummer[V], it: Iterator[V]) 50 | extends java.io.Serializable with Iterator[V] { 51 | 52 | // This has to be lazy because it shouldn't be touched until the val it is exhausted 53 | protected lazy val tailIter = summer.flush.iterator 54 | def hasNext: Boolean = it.hasNext || tailIter.hasNext 55 | def next = nextInternal 56 | 57 | @tailrec 58 | private def nextInternal: V = { 59 | if (it.hasNext) { 60 | summer.put(it.next) match { 61 | case None => nextInternal 62 | case Some(v) => v 63 | } 64 | } else { 65 | tailIter.next 66 | } 67 | } 68 | } 69 | -------------------------------------------------------------------------------- /algebird-core/src/main/scala/com/twitter/algebird/AdjoinedUnitRing.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | package com.twitter.algebird 17 | 18 | import scala.annotation.tailrec 19 | 20 | /** 21 | * This is for the case where your Ring[T] is a Rng (i.e. there is no unit). 22 | * @see http://en.wikipedia.org/wiki/Pseudo-ring#Adjoining_an_identity_element 23 | */ 24 | case class AdjoinedUnit[T](ones: BigInt, get: T) { 25 | def unwrap: Option[T] = if (ones == 0) Some(get) else None 26 | } 27 | 28 | object AdjoinedUnit { 29 | def apply[T](item: T): AdjoinedUnit[T] = new AdjoinedUnit[T](BigInt(0), item) 30 | implicit def ring[T](implicit ring: Ring[T]): Ring[AdjoinedUnit[T]] = new AdjoinedUnitRing[T] 31 | } 32 | 33 | class AdjoinedUnitRing[T](implicit ring: Ring[T]) extends Ring[AdjoinedUnit[T]] { 34 | val one = AdjoinedUnit[T](BigInt(1), ring.zero) 35 | val zero = AdjoinedUnit[T](ring.zero) 36 | 37 | override def isNonZero(it: AdjoinedUnit[T]) = 38 | (it.ones != 0) || ring.isNonZero(it.get) 39 | 40 | def plus(left: AdjoinedUnit[T], right: AdjoinedUnit[T]) = 41 | AdjoinedUnit(left.ones + right.ones, ring.plus(left.get, right.get)) 42 | 43 | override def negate(it: AdjoinedUnit[T]) = 44 | AdjoinedUnit(-it.ones, ring.negate(it.get)) 45 | override def minus(left: AdjoinedUnit[T], right: AdjoinedUnit[T]) = 46 | AdjoinedUnit(left.ones - right.ones, ring.minus(left.get, right.get)) 47 | 48 | def times(left: AdjoinedUnit[T], right: AdjoinedUnit[T]) = { 49 | // (n1, g1) * (n2, g2) = (n1*n2, (n1*g1) + (n2*g1) + (g1*g2)) 50 | import Group.intTimes 51 | 52 | val ones = left.ones * right.ones 53 | val part0 = intTimes(left.ones, right.get)(ring) 54 | val part1 = intTimes(right.ones, left.get)(ring) 55 | val part2 = ring.times(left.get, right.get) 56 | val nonUnit = ring.plus(part0, ring.plus(part1, part2)) 57 | 58 | AdjoinedUnit(ones, nonUnit) 59 | } 60 | 61 | } 62 | -------------------------------------------------------------------------------- /algebird-test/src/test/scala/com/twitter/algebird/MinHasherTest.scala: -------------------------------------------------------------------------------- 1 | package com.twitter.algebird 2 | 3 | import org.scalatest._ 4 | 5 | import org.scalatest.{ PropSpec, Matchers } 6 | import org.scalatest.prop.PropertyChecks 7 | import org.scalacheck.{ Gen, Arbitrary } 8 | 9 | import java.util.Arrays 10 | 11 | class MinHasherTest extends PropSpec with PropertyChecks with Matchers { 12 | import BaseProperties._ 13 | 14 | implicit val mhMonoid = new MinHasher32(0.5, 512) 15 | implicit val mhGen = Arbitrary { 16 | for ( 17 | v <- Gen.choose(0, 10000) 18 | ) yield (mhMonoid.init(v)) 19 | } 20 | 21 | property("MinHasher is a Monoid") { 22 | monoidLawsEq[MinHashSignature]{ (a, b) => a.bytes.toList == b.bytes.toList } 23 | } 24 | } 25 | 26 | class MinHasherSpec extends WordSpec with Matchers { 27 | val r = new java.util.Random 28 | 29 | def test[H](mh: MinHasher[H], similarity: Double, epsilon: Double) = { 30 | val (set1, set2) = randomSets(similarity) 31 | 32 | val exact = exactSimilarity(set1, set2) 33 | val sim = approxSimilarity(mh, set1, set2) 34 | val error: Double = math.abs(exact - sim) 35 | assert(error < epsilon) 36 | } 37 | 38 | def randomSets(similarity: Double) = { 39 | val s = 10000 40 | val uniqueFraction = if (similarity == 1.0) 0.0 else (1 - similarity) / (1 + similarity) 41 | val sharedFraction = 1 - uniqueFraction 42 | val unique1 = 1.to((s * uniqueFraction).toInt).map{ i => math.random }.toSet 43 | val unique2 = 1.to((s * uniqueFraction).toInt).map{ i => math.random }.toSet 44 | 45 | val shared = 1.to((s * sharedFraction).toInt).map{ i => math.random }.toSet 46 | (unique1 ++ shared, unique2 ++ shared) 47 | } 48 | 49 | def exactSimilarity[T](x: Set[T], y: Set[T]) = { 50 | (x & y).size.toDouble / (x ++ y).size 51 | } 52 | 53 | def approxSimilarity[T, H](mh: MinHasher[H], x: Set[T], y: Set[T]) = { 54 | val sig1 = x.map{ l => mh.init(l.toString) }.reduce{ (a, b) => mh.plus(a, b) } 55 | val sig2 = y.map{ l => mh.init(l.toString) }.reduce{ (a, b) => mh.plus(a, b) } 56 | mh.similarity(sig1, sig2) 57 | } 58 | 59 | "MinHasher32" should { 60 | "measure 0.5 similarity in 1024 bytes with < 0.1 error" in { 61 | test(new MinHasher32(0.5, 1024), 0.5, 0.1) 62 | } 63 | "measure 0.8 similarity in 1024 bytes with < 0.05 error" in { 64 | test(new MinHasher32(0.8, 1024), 0.8, 0.05) 65 | } 66 | "measure 1.0 similarity in 1024 bytes with < 0.01 error" in { 67 | test(new MinHasher32(1.0, 1024), 1.0, 0.01) 68 | } 69 | } 70 | } 71 | -------------------------------------------------------------------------------- /algebird-core/src/main/scala/com/twitter/algebird/MinPlus.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird 18 | 19 | /* 20 | * The Min-Plus algebra, or tropical semi-ring is useful for computing shortest 21 | * paths on graphs: 22 | * @see Min-plus Matrix Product" 23 | * The shortest path from i->j in k or less steps is ((G)^k)_{ij} 24 | * @see Tropical Geometry 25 | * @see Semiring definition 26 | */ 27 | 28 | // This is basically a sigil class to represent using the Min-Plus semi-ring 29 | sealed trait MinPlus[+V] extends Any with java.io.Serializable 30 | case object MinPlusZero extends MinPlus[Nothing] 31 | case class MinPlusValue[V](get: V) extends AnyVal with MinPlus[V] 32 | 33 | class MinPlusSemiring[V](implicit monoid: Monoid[V], ord: Ordering[V]) extends Ring[MinPlus[V]] { 34 | override def zero = MinPlusZero 35 | override def negate(mv: MinPlus[V]) = 36 | sys.error("MinPlus is a semi-ring, there is no additive inverse") 37 | override def one: MinPlus[V] = MinPlusValue(monoid.zero) 38 | // a+b = min(a,b) 39 | override def plus(left: MinPlus[V], right: MinPlus[V]) = 40 | // We are doing the if to avoid an allocation: 41 | (left, right) match { 42 | case (MinPlusZero, _) => right 43 | case (_, MinPlusZero) => left 44 | case (MinPlusValue(lv), MinPlusValue(rv)) => if (ord.lteq(lv, rv)) left else right 45 | } 46 | 47 | // a*b = a+b 48 | override def times(left: MinPlus[V], right: MinPlus[V]) = 49 | (left, right) match { 50 | case (MinPlusZero, _) => MinPlusZero 51 | case (_, MinPlusZero) => MinPlusZero 52 | case (MinPlusValue(lv), MinPlusValue(rv)) => MinPlusValue(monoid.plus(lv, rv)) 53 | } 54 | } 55 | 56 | object MinPlus extends java.io.Serializable { 57 | implicit def semiring[V: Monoid: Ordering]: Ring[MinPlus[V]] = new MinPlusSemiring[V] 58 | } 59 | -------------------------------------------------------------------------------- /algebird-test/src/test/scala/com/twitter/algebird/AggregatorLaws.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2013 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird 18 | 19 | import org.scalacheck.Arbitrary 20 | import org.scalatest.{ PropSpec, Matchers } 21 | import org.scalatest.prop.PropertyChecks 22 | 23 | class AggregatorLaws extends PropSpec with PropertyChecks with Matchers { 24 | import BaseProperties._ 25 | 26 | implicit def aggregator[A, B, C](implicit prepare: Arbitrary[A => B], sg: Semigroup[B], present: Arbitrary[B => C]): Arbitrary[Aggregator[A, B, C]] = Arbitrary { 27 | for { 28 | pp <- prepare.arbitrary 29 | ps <- present.arbitrary 30 | } yield new Aggregator[A, B, C] { 31 | def prepare(a: A) = pp(a) 32 | def semigroup = sg 33 | def present(b: B) = ps(b) 34 | } 35 | } 36 | 37 | property("composing before Aggregator is correct") { 38 | forAll { (in: List[Int], compose: (Int => Int), ag: Aggregator[Int, Int, Int]) => 39 | val composed = ag.composePrepare(compose) 40 | assert(in.isEmpty || composed(in) == ag(in.map(compose))) 41 | } 42 | } 43 | 44 | property("andThen after Aggregator is correct") { 45 | forAll { (in: List[Int], andt: (Int => Int), ag: Aggregator[Int, Int, Int]) => 46 | val ag1 = ag.andThenPresent(andt) 47 | assert(in.isEmpty || ag1(in) == andt(ag(in))) 48 | } 49 | } 50 | 51 | property("composing two Aggregators is correct") { 52 | forAll { (in: List[Int], ag1: Aggregator[Int, String, Int], ag2: Aggregator[Int, Int, String]) => 53 | val c = GeneratedTupleAggregator.from2(ag1, ag2) 54 | assert(in.isEmpty || c(in) == (ag1(in), ag2(in))) 55 | } 56 | } 57 | property("Applicative composing two Aggregators is correct") { 58 | forAll { (in: List[Int], ag1: Aggregator[Int, Set[Int], Int], ag2: Aggregator[Int, Unit, String]) => 59 | type AggInt[T] = Aggregator[Int, _, T] 60 | val c = Applicative.join[AggInt, Int, String](ag1, ag2) 61 | assert(in.isEmpty || c(in) == (ag1(in), ag2(in))) 62 | } 63 | } 64 | } 65 | -------------------------------------------------------------------------------- /algebird-test/src/test/scala/com/twitter/algebird/JavaBoxedTests.scala: -------------------------------------------------------------------------------- 1 | package com.twitter.algebird 2 | 3 | import org.scalatest.{ PropSpec, Matchers } 4 | import org.scalatest.prop.PropertyChecks 5 | import org.scalacheck.{ Gen, Arbitrary } 6 | 7 | import java.lang.{ Integer => JInt, Short => JShort, Long => JLong, Float => JFloat, Double => JDouble, Boolean => JBool } 8 | import java.util.{ List => JList, Map => JMap } 9 | 10 | import scala.collection.JavaConverters._ 11 | 12 | class JavaBoxedTests extends PropSpec with PropertyChecks with Matchers { 13 | import BaseProperties._ 14 | 15 | implicit val jboolArg = Arbitrary { for (v <- Gen.oneOf(JBool.TRUE, JBool.FALSE)) yield v } 16 | implicit val jintArg = Arbitrary { 17 | for (v <- Gen.choose(Int.MinValue, Int.MaxValue)) 18 | yield JInt.valueOf(v) 19 | } 20 | implicit val jshortArg = Arbitrary { 21 | for (v <- Gen.choose(Short.MinValue, Short.MaxValue)) 22 | yield Short.box(v) 23 | } 24 | implicit val jlongArg = Arbitrary { 25 | // If we put Long.Max/Min we get overflows that seem to break the ring properties, not clear why 26 | for (v <- Gen.choose(Int.MinValue, Int.MaxValue)) 27 | yield JLong.valueOf(v) 28 | } 29 | 30 | property("Boolean is a Field") { 31 | fieldLaws[Boolean] 32 | } 33 | 34 | property("JBoolean is a Field") { 35 | fieldLaws[JBool] 36 | } 37 | 38 | property("Int is a Ring") { 39 | ringLaws[Int] 40 | } 41 | 42 | property("JInt is a Ring") { 43 | ringLaws[JInt] 44 | } 45 | 46 | property("Short is a Ring") { 47 | ringLaws[Short] 48 | } 49 | 50 | property("JShort is a Ring") { 51 | ringLaws[JShort] 52 | } 53 | 54 | property("Long is a Ring") { 55 | ringLaws[Long] 56 | } 57 | 58 | property("JLong is a Ring") { 59 | ringLaws[JLong] 60 | } 61 | 62 | // TODO add testing with JFloat/JDouble but check for approximate equals, pain in the ass. 63 | 64 | implicit def jlist[T: Arbitrary] = Arbitrary { 65 | implicitly[Arbitrary[List[T]]].arbitrary.map { _.asJava } 66 | } 67 | 68 | property("JList is a Monoid") { 69 | monoidLaws[JList[Int]] 70 | } 71 | 72 | implicit def jmap[K: Arbitrary, V: Arbitrary: Semigroup] = Arbitrary { 73 | implicitly[Arbitrary[Map[K, V]]].arbitrary.map { _.filter { kv => isNonZero[V](kv._2) }.asJava } 74 | } 75 | 76 | property("JMap[String,Int] is a Monoid") { 77 | isAssociative[JMap[String, Int]] && weakZero[JMap[String, Int]] 78 | } 79 | 80 | property("JMap[String,String] is a Monoid") { 81 | isAssociative[JMap[String, String]] && weakZero[JMap[String, String]] 82 | } 83 | 84 | } 85 | -------------------------------------------------------------------------------- /algebird-test/src/test/scala/com/twitter/algebird/MomentsGroupTest.scala: -------------------------------------------------------------------------------- 1 | package com.twitter.algebird 2 | 3 | import org.scalatest._ 4 | 5 | class MomentsGroupTest extends WordSpec with Matchers { 6 | 7 | /** 8 | * Given a list of doubles, create a Moments object to hold 9 | * the list's central moments. 10 | */ 11 | def getMoments(xs: List[Double]): Moments = 12 | xs.foldLeft(MomentsGroup.zero) { (m, x) => MomentsGroup.plus(m, Moments(x)) } 13 | 14 | def testApproxEq(f1: Double, f2: Double) { 15 | if (f2 == 0) 16 | assert(f1 < 1e-10) 17 | else 18 | assert((scala.math.abs(f1 - f2) / scala.math.abs(f2)) < 1e-10) 19 | } 20 | 21 | "Moments should count" in { 22 | val m1 = getMoments(List(1, 2, 3, 4, 5)) 23 | testApproxEq(m1.count, 5) 24 | 25 | val m2 = getMoments(Nil) 26 | testApproxEq(m2.count, 0) 27 | } 28 | 29 | "Moments should calculate mean" in { 30 | val m1 = getMoments(List(1, 2, 3, 4, 5)) 31 | testApproxEq(m1.mean, 3) 32 | 33 | val m2 = getMoments(List(1, 1, 1, 2, 3)) 34 | testApproxEq(m2.mean, 1.6) 35 | } 36 | 37 | // variance = function(x) { mean((x - mean(x)) ** 2) } 38 | "Moments should calculate variance" in { 39 | val m1 = getMoments(List(1, 2, 3, 4, 5)) 40 | testApproxEq(m1.variance, 2) 41 | 42 | val m2 = getMoments(List(1, 1, 1, 2, 3)) 43 | testApproxEq(m2.variance, 0.64) 44 | } 45 | 46 | // skewness = function(x) { mean((x - mean(x)) ** 3) / (mean((x - mean(x)) ** 2)) ** 1.5 } 47 | "Moments should calculate skewness" in { 48 | val m1 = getMoments(List(1, 2, 3, 4, 5)) 49 | testApproxEq(m1.skewness, 0) 50 | 51 | val m2 = getMoments(List(1, 1, 1, 2, 3)) 52 | testApproxEq(m2.skewness, 0.84375) 53 | } 54 | 55 | // kurtosis = function(x) { -3 + mean((x - mean(x)) ** 4) / (mean((x - mean(x)) ** 2) ** 2) } 56 | "Moments should calculate kurtosis" in { 57 | val m1 = getMoments(List(1, 2, 3, 4, 5)) 58 | testApproxEq(m1.kurtosis, -1.3) 59 | 60 | val m2 = getMoments(List(1, 1, 1, 2, 3)) 61 | testApproxEq(m2.kurtosis, -0.921875) 62 | } 63 | 64 | "Moments can be aggregated" in { 65 | val m1 = MomentsAggregator(List(1, 2, 3, 4, 5)) 66 | testApproxEq(m1.count, 5) 67 | testApproxEq(m1.mean, 3) 68 | testApproxEq(m1.variance, 2) 69 | testApproxEq(m1.skewness, 0) 70 | testApproxEq(m1.kurtosis, -1.3) 71 | 72 | val m2 = MomentsAggregator(List(1, 1, 1, 2, 3)) 73 | testApproxEq(m2.count, 5) 74 | testApproxEq(m2.mean, 1.6) 75 | testApproxEq(m2.variance, 0.64) 76 | testApproxEq(m2.skewness, 0.84375) 77 | testApproxEq(m2.kurtosis, -0.921875) 78 | 79 | } 80 | } 81 | -------------------------------------------------------------------------------- /algebird-util/src/main/scala/com/twitter/algebird/util/summer/AsyncMapSum.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | package com.twitter.algebird.util.summer 17 | 18 | import com.twitter.algebird._ 19 | import com.twitter.util.{ Duration, Future, FuturePool } 20 | import java.util.concurrent.ArrayBlockingQueue 21 | import scala.collection.mutable.ArrayBuffer 22 | import scala.collection.JavaConverters._ 23 | 24 | /** 25 | * @author Ian O Connell 26 | */ 27 | 28 | class AsyncMapSum[Key, Value](bufferSize: BufferSize, 29 | override val flushFrequency: FlushFrequency, 30 | override val softMemoryFlush: MemoryFlushPercent, 31 | override val memoryIncr: Incrementor, 32 | override val timeoutIncr: Incrementor, 33 | insertOp: Incrementor, 34 | tuplesOut: Incrementor, 35 | sizeIncr: Incrementor, 36 | workPool: FuturePool)(implicit semigroup: Semigroup[Value]) 37 | extends AsyncSummer[(Key, Value), Map[Key, Value]] 38 | with WithFlushConditions[(Key, Value), Map[Key, Value]] { 39 | 40 | require(bufferSize.v > 0, "Use the Null summer for an empty async summer") 41 | 42 | protected override val emptyResult = Map.empty[Key, Value] 43 | 44 | private[this] final val queue = new ArrayBlockingQueue[Map[Key, Value]](bufferSize.v, true) 45 | override def isFlushed: Boolean = queue.size == 0 46 | 47 | override def flush: Future[Map[Key, Value]] = { 48 | val toSum = ArrayBuffer[Map[Key, Value]]() 49 | queue.drainTo(toSum.asJava) 50 | workPool { 51 | val result = Semigroup.sumOption(toSum).getOrElse(Map.empty) 52 | tuplesOut.incrBy(result.size) 53 | result 54 | } 55 | } 56 | 57 | def addAll(vals: TraversableOnce[(Key, Value)]): Future[Map[Key, Value]] = { 58 | insertOp.incr 59 | 60 | val curData = Semigroup.sumOption(vals.map(Map(_))).getOrElse(Map.empty) 61 | if (!queue.offer(curData)) { 62 | flush.map { flushRes => 63 | sizeIncr.incr //todo not sure if need to increase size 64 | Semigroup.plus(flushRes, curData) 65 | } 66 | } else { 67 | Future.value(Map.empty) 68 | } 69 | } 70 | } -------------------------------------------------------------------------------- /algebird-test/src/test/scala/com/twitter/algebird/SummingIteratorTest.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird 18 | 19 | import org.scalatest.{ PropSpec, Matchers } 20 | import org.scalatest.prop.PropertyChecks 21 | import org.scalacheck.{ Gen, Arbitrary } 22 | 23 | object SummingIteratorTest { 24 | implicit def mapEquiv[K, V: Monoid: Equiv]: Equiv[Map[K, V]] = Equiv.fromFunction { (l, r) => 25 | val zl = MapAlgebra.removeZeros(l) 26 | val zr = MapAlgebra.removeZeros(r) 27 | zl.size == zr.size && { 28 | zl.forall { 29 | case (k, v) => 30 | zr.get(k).map { rv => Equiv[V].equiv(rv, v) }.getOrElse(false) 31 | } 32 | } 33 | } 34 | } 35 | 36 | class SummingIteratorTest extends PropSpec with PropertyChecks with Matchers { 37 | import SummingIteratorTest._ 38 | def sumEquiv[V: Semigroup: Equiv](it0: Iterator[V], it1: Iterator[V]): Boolean = 39 | StatefulSummerLaws.zeroEquiv(Semigroup.sumOption(it0), Semigroup.sumOption(it1)) 40 | 41 | case class Capacity(c: Int) 42 | implicit val capArb = Arbitrary { for (c <- Gen.choose(0, 10240)) yield Capacity(c) } 43 | 44 | property("With Maps is preserved[(Short,Int)]") { 45 | forAll { (cap: Capacity, items: List[(Short, Int)]) => 46 | val mitems = items.map { Map(_) } 47 | val qit = SummingIterator[Map[Short, Int]](SummingQueue[Map[Short, Int]](cap.c), mitems.iterator) 48 | val qitc = SummingIterator[Map[Short, Int]](SummingCache[Short, Int](cap.c), mitems.iterator) 49 | assert(sumEquiv(mitems.iterator, qit) && sumEquiv(mitems.iterator, qitc)) 50 | } 51 | } 52 | 53 | property("With Maps is preserved[(Short,String)]") { 54 | forAll { (cap: Capacity, items: List[(Short, String)]) => 55 | val mitems = items.map { Map(_) } 56 | val qit = SummingIterator(SummingQueue[Map[Short, String]](cap.c), mitems.iterator) 57 | val qitc = SummingIterator(SummingCache[Short, String](cap.c), mitems.iterator) 58 | assert(sumEquiv(mitems.iterator, qit) && sumEquiv(mitems.iterator, qitc)) 59 | } 60 | } 61 | } 62 | -------------------------------------------------------------------------------- /algebird-test/src/test/scala/com/twitter/algebird/statistics/StatisticsTests.scala: -------------------------------------------------------------------------------- 1 | package com.twitter.algebird.statistics 2 | 3 | import org.scalatest.{ PropSpec, Matchers } 4 | import org.scalatest.prop.PropertyChecks 5 | import org.scalacheck.{ Arbitrary, Properties } 6 | import org.scalatest.{ PropSpec, Matchers } 7 | import org.scalatest.prop.PropertyChecks 8 | import org.scalacheck.Gen._ 9 | import org.scalatest._ 10 | 11 | import com.twitter.algebird.BaseProperties 12 | 13 | class StatisticsRingLaws extends PropSpec with PropertyChecks with Matchers { 14 | import BaseProperties._ 15 | 16 | val statsRing = new StatisticsRing[Int] 17 | val gen = for (v <- choose(0, 1 << 30)) yield v 18 | 19 | property("StatisticsRing is a Ring") { 20 | ringLaws[Int](statsRing, Arbitrary(gen)) 21 | } 22 | 23 | } 24 | 25 | class StatisticsMonoidLaws extends PropSpec with PropertyChecks with Matchers { 26 | import BaseProperties._ 27 | 28 | val statsMonoid = new StatisticsMonoid[Int] 29 | 30 | val gen = for (v <- choose(0, 1 << 14)) yield v 31 | 32 | property("StatisticsMonoid is a Monoid") { 33 | monoidLaws[Int](statsMonoid, Arbitrary(gen)) 34 | } 35 | 36 | } 37 | 38 | class StatisticsTest extends WordSpec with Matchers { 39 | 40 | // the test framework garbles the exceptions :/ 41 | lazy val statsMonoid = new StatisticsMonoid[Int] 42 | try { 43 | for (i <- 1 to 2) statsMonoid.zero 44 | for (i <- 1 to 3) statsMonoid.plus(i, i) 45 | for (i <- 1 to 3000) statsMonoid.sum(for (v <- 1 to i) yield v) 46 | for (i <- 1 to 2000) statsMonoid.sumOption(for (v <- 1 to i) yield v) 47 | } catch { 48 | case e: Exception => { 49 | e.printStackTrace() 50 | throw e 51 | } 52 | } 53 | 54 | "StatisticsMonoid" should { 55 | 56 | "count zero calls" in { 57 | assert(statsMonoid.getZeroCallCount == 2) 58 | } 59 | 60 | "count plus calls" in { 61 | assert(statsMonoid.getPlusCallCount == 3) 62 | } 63 | 64 | "count sum calls" in { 65 | assert(statsMonoid.getSumCallCount == 3000) 66 | assert(statsMonoid.getSumCallTime > 0L) 67 | statsMonoid.toString.contains("sum calls: <1: 0, <2: 1, <4: 2, <8: 4, <16: 8, <32: 16, <64: 32, <128: 64, <256: 128, <512: 256, >: 2489, avg=1500.5 count=3000") 68 | } 69 | 70 | "count sumOption calls" in { 71 | assert(statsMonoid.getSumOptionCallCount == 2000) 72 | assert(statsMonoid.getSumOptionCallTime > 0L) 73 | statsMonoid.toString.contains("sumOption calls: <1: 0, <2: 1, <4: 2, <8: 4, <16: 8, <32: 16, <64: 32, <128: 64, <256: 128, <512: 256, >: 1489, avg=1000.5 count=2000") 74 | } 75 | 76 | } 77 | 78 | } 79 | -------------------------------------------------------------------------------- /algebird-test/src/test/scala/com/twitter/algebird/MonadProperties.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird 18 | 19 | import org.scalatest.{ PropSpec, Matchers } 20 | import org.scalatest.prop.PropertyChecks 21 | import org.scalacheck.{ Gen, Arbitrary } 22 | 23 | import Monad.{ pureOp, operators } 24 | 25 | class MonadProperties extends PropSpec with PropertyChecks with Matchers { 26 | import BaseProperties._ 27 | import MonadLaws._ 28 | 29 | property("list") { 30 | monadLaws[List, Int, String, Long]() 31 | } 32 | 33 | property("option") { 34 | monadLaws[Option, Int, String, Long]() 35 | } 36 | 37 | property("indexedseq") { 38 | monadLaws[IndexedSeq, Int, String, Long]() 39 | } 40 | 41 | property("vector") { 42 | monadLaws[Vector, Int, String, Long]() 43 | } 44 | 45 | property("set") { 46 | monadLaws[Set, Int, String, Long]() 47 | } 48 | 49 | property("seq") { 50 | monadLaws[Seq, Int, String, Long]() 51 | } 52 | 53 | // Monad algebras: 54 | property("Monad Semigroup") { 55 | implicit val optSg = new MonadSemigroup[Int, Option] 56 | implicit val listSg = new MonadSemigroup[String, List] 57 | // the + here is actually a cross-product, and testing sumOption blows up 58 | semigroupLaws[Option[Int]] && isAssociative[List[String]] 59 | } 60 | 61 | property("Monad Monoid") { 62 | implicit val optSg = new MonadMonoid[Int, Option] 63 | implicit val listSg = new MonadMonoid[String, List] 64 | // the + here is actually a cross-product, and testing sumOption blows up 65 | monoidLaws[Option[Int]] && validZero[List[String]] 66 | } 67 | 68 | // These laws work for only "non-empty" monads 69 | property("Monad Group") { 70 | implicit val optSg = new MonadGroup[Int, Some] 71 | groupLaws[Some[Int]] 72 | } 73 | 74 | property("Monad Ring") { 75 | implicit val optSg = new MonadRing[Int, Some] 76 | ringLaws[Some[Int]] 77 | } 78 | 79 | property("Monad Field") { 80 | implicit val optSg = new MonadField[Boolean, Some] 81 | fieldLaws[Some[Boolean]] 82 | } 83 | } 84 | -------------------------------------------------------------------------------- /project/Unidoc.scala: -------------------------------------------------------------------------------- 1 | package algebird 2 | 3 | import sbt._ 4 | import sbt.Keys._ 5 | import sbt.Project.Initialize 6 | 7 | /** Borrowed from https://github.com/akka/akka/blob/master/project/Unidoc.scala */ 8 | object Unidoc { 9 | val unidocDirectory = SettingKey[File]("unidoc-directory") 10 | val unidocExclude = SettingKey[Seq[String]]("unidoc-exclude") 11 | val unidocAllSources = TaskKey[Seq[Seq[File]]]("unidoc-all-sources") 12 | val unidocSources = TaskKey[Seq[File]]("unidoc-sources") 13 | val unidocAllClasspaths = TaskKey[Seq[Classpath]]("unidoc-all-classpaths") 14 | val unidocClasspath = TaskKey[Seq[File]]("unidoc-classpath") 15 | val unidoc = TaskKey[File]("unidoc", "Create unified scaladoc for all aggregates") 16 | 17 | lazy val settings = Seq( 18 | unidocDirectory <<= crossTarget / "unidoc", 19 | unidocExclude := Seq.empty, 20 | unidocAllSources <<= (thisProjectRef, buildStructure, unidocExclude) flatMap allSources, 21 | unidocSources <<= unidocAllSources map { _.flatten }, 22 | unidocAllClasspaths <<= (thisProjectRef, buildStructure, unidocExclude) flatMap allClasspaths, 23 | unidocClasspath <<= unidocAllClasspaths map { _.flatten.map(_.data).distinct }, 24 | unidoc <<= unidocTask 25 | ) 26 | 27 | def allSources(projectRef: ProjectRef, structure: Load.BuildStructure, exclude: Seq[String]): Task[Seq[Seq[File]]] = { 28 | val projects = aggregated(projectRef, structure, exclude) 29 | projects flatMap { sources in Compile in LocalProject(_) get structure.data } join 30 | } 31 | 32 | def allClasspaths(projectRef: ProjectRef, structure: Load.BuildStructure, exclude: Seq[String]): Task[Seq[Classpath]] = { 33 | val projects = aggregated(projectRef, structure, exclude) 34 | projects flatMap { dependencyClasspath in Compile in LocalProject(_) get structure.data } join 35 | } 36 | 37 | def aggregated(projectRef: ProjectRef, structure: Load.BuildStructure, exclude: Seq[String]): Seq[String] = { 38 | val aggregate = Project.getProject(projectRef, structure).toSeq.flatMap(_.aggregate) 39 | aggregate flatMap { ref => 40 | if (exclude contains ref.project) Seq.empty 41 | else ref.project +: aggregated(ref, structure, exclude) 42 | } 43 | } 44 | 45 | def unidocTask: Initialize[Task[File]] = { 46 | (compilers, cacheDirectory, unidocSources, unidocClasspath, unidocDirectory, scalacOptions in doc, streams) map { 47 | (compilers, cache, sources, classpath, target, options, s) => { 48 | val scaladoc = new Scaladoc(100, compilers.scalac) 49 | scaladoc.cached(cache / "unidoc", "main", sources, classpath, target, options, s.log) 50 | target 51 | } 52 | } 53 | } 54 | } 55 | -------------------------------------------------------------------------------- /algebird-util/src/test/scala/com/twitter/algebird/util/summer/AsyncSummerLaws.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2013 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird.util.summer 18 | 19 | import org.scalatest.{ PropSpec, Matchers } 20 | import org.scalatest.prop.PropertyChecks 21 | import org.scalacheck.{ Gen, Arbitrary } 22 | import com.twitter.algebird.{ MapAlgebra, Semigroup } 23 | import com.twitter.util.{ Future, Await, Duration, FuturePool } 24 | 25 | import java.util.concurrent.Executors 26 | 27 | object AsyncSummerLaws { 28 | val executor = Executors.newFixedThreadPool(4) 29 | val workPool = FuturePool(executor) 30 | 31 | private[this] val schedulingExecutor = Executors.newFixedThreadPool(4) 32 | private[this] val schedulingWorkPool = FuturePool(schedulingExecutor) 33 | 34 | implicit def arbFlushFreq = Arbitrary { 35 | Gen.choose(1, 4000) 36 | .map { x: Int => FlushFrequency(Duration.fromMilliseconds(x)) } 37 | } 38 | 39 | implicit def arbBufferSize = Arbitrary { 40 | Gen.choose(1, 10) 41 | .map { x => BufferSize(x) } 42 | } 43 | 44 | implicit def arbMemoryFlushPercent = Arbitrary { 45 | Gen.choose(80.0f, 90.0f) 46 | .map { x => MemoryFlushPercent(x) } 47 | } 48 | 49 | implicit def arbCompactSize = Arbitrary { 50 | Gen.choose(1, 10) 51 | .map { x => CompactionSize(x) } 52 | } 53 | 54 | def sample[T: Arbitrary]: T = Arbitrary.arbitrary[T].sample.get 55 | 56 | def summingWithAndWithoutSummerShouldMatch[K, V: Semigroup](asyncSummer: AsyncSummer[(K, V), Iterable[(K, V)]], inputs: List[List[(K, V)]]) = { 57 | val reference = MapAlgebra.sumByKey(inputs.flatten) 58 | val resA = Await.result(Future.collect(inputs.map{ i => 59 | schedulingWorkPool { 60 | asyncSummer.addAll(i) 61 | }.flatMap(identity) 62 | })).map(_.toList).flatten 63 | val resB = Await.result(asyncSummer.flush) 64 | require(asyncSummer.isFlushed, "The flush should have ensured we are flushed now") 65 | 66 | val other = MapAlgebra.sumByKey(resA.toList ++ resB.toList) 67 | val res = Equiv[Map[K, V]].equiv( 68 | reference, 69 | other) 70 | res 71 | } 72 | 73 | } 74 | -------------------------------------------------------------------------------- /algebird-caliper/src/test/scala/com/twitter/algebird/caliper/HLLBenchmark.scala: -------------------------------------------------------------------------------- 1 | package com.twitter.algebird.caliper 2 | 3 | import com.twitter.algebird._ 4 | import scala.util.Random 5 | import com.twitter.bijection._ 6 | 7 | import java.util.concurrent.Executors 8 | import com.twitter.algebird.util._ 9 | import com.google.caliper.{ Param, SimpleBenchmark } 10 | import com.twitter.algebird.HyperLogLogMonoid 11 | import java.nio.ByteBuffer 12 | 13 | import scala.math._ 14 | class OldMonoid(bits: Int) extends HyperLogLogMonoid(bits) { 15 | import HyperLogLog._ 16 | 17 | override def sumOption(items: TraversableOnce[HLL]): Option[HLL] = 18 | if (items.isEmpty) None 19 | else { 20 | val buffer = new Array[Byte](size) 21 | items.foreach { _.updateInto(buffer) } 22 | Some(DenseHLL(bits, buffer.toIndexedSeq)) 23 | } 24 | } 25 | 26 | class HllBenchmark extends SimpleBenchmark { 27 | var hllMonoid: HyperLogLogMonoid = _ 28 | var oldHllMonoid: HyperLogLogMonoid = _ 29 | 30 | @Param(Array("12", "14", "24")) 31 | val numBits: Int = 0 32 | 33 | // Old sum option will not work with >=100 keys, and >= 1000 elements. 34 | @Param(Array("1", "10")) 35 | val numInputKeys: Int = 0 36 | 37 | @Param(Array("10", "1000", "10000")) 38 | val numElements: Int = 0 39 | 40 | var inputData: Seq[Seq[HLL]] = _ 41 | 42 | override def setUp { 43 | hllMonoid = new HyperLogLogMonoid(numBits) 44 | 45 | oldHllMonoid = new OldMonoid(numBits) 46 | 47 | val rng = new Random(3) 48 | 49 | val byteEncoder = implicitly[Injection[Long, Array[Byte]]] 50 | def setSize = rng.nextInt(10) + 1 // 1 -> 10 51 | def hll(elements: Set[Long]): HLL = hllMonoid.batchCreate(elements)(byteEncoder) 52 | 53 | val inputIntermediate = (0L until numElements).map { _ => 54 | val setElements = (0 until setSize).map{ _ => rng.nextInt(1000).toLong }.toSet 55 | (pow(numInputKeys, rng.nextFloat).toLong, List(hll(setElements))) 56 | } 57 | inputData = MapAlgebra.sumByKey(inputIntermediate).map(_._2).toSeq 58 | } 59 | 60 | def timeSumOption(reps: Int): Int = { 61 | var dummy = 0 62 | while (dummy < reps) { 63 | inputData.foreach(hllMonoid.sumOption(_)) 64 | dummy += 1 65 | } 66 | dummy 67 | } 68 | 69 | def timeOldSumOption(reps: Int): Int = { 70 | var dummy = 0 71 | while (dummy < reps) { 72 | inputData.foreach(oldHllMonoid.sumOption(_)) 73 | dummy += 1 74 | } 75 | dummy 76 | } 77 | 78 | def timePlus(reps: Int): Int = { 79 | var dummy = 0 80 | while (dummy < reps) { 81 | inputData.foreach { vals => 82 | vals.reduce(hllMonoid.plus(_, _)) 83 | } 84 | dummy += 1 85 | } 86 | dummy 87 | } 88 | } 89 | -------------------------------------------------------------------------------- /algebird-core/src/main/scala/com/twitter/algebird/SummingCache.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird 18 | 19 | /** 20 | * @author Oscar Boykin 21 | */ 22 | 23 | import java.util.concurrent.ArrayBlockingQueue 24 | 25 | import java.util.{ LinkedHashMap => JLinkedHashMap, Map => JMap } 26 | import scala.collection.mutable.{ Map => MMap } 27 | import scala.collection.JavaConverters._ 28 | import scala.annotation.tailrec 29 | 30 | object SummingCache { 31 | def apply[K, V: Semigroup](cap: Int): SummingCache[K, V] = new SummingCache[K, V](cap) 32 | } 33 | /** 34 | * A Stateful Summer on Map[K,V] that keeps a cache of recent keys 35 | */ 36 | class SummingCache[K, V] private (capacity: Int)(implicit sgv: Semigroup[V]) 37 | extends StatefulSummer[Map[K, V]] { 38 | 39 | require(capacity >= 0, "Cannot have negative capacity in SummingIterator") 40 | 41 | override val semigroup = new MapMonoid[K, V] 42 | protected def optNonEmpty(m: Map[K, V]) = if (m.isEmpty) None else Some(m) 43 | 44 | override def put(m: Map[K, V]): Option[Map[K, V]] = { 45 | val replaced = m.map { 46 | case (k, v) => 47 | val newV = cache.get(k) 48 | .map { oldV => sgv.plus(oldV, v) } 49 | .getOrElse { v } 50 | (k, newV) 51 | } 52 | 53 | cache ++= replaced 54 | val ret = lastEvicted 55 | // Rest this var 56 | lastEvicted = Map.empty[K, V] 57 | optNonEmpty(ret) 58 | } 59 | override def flush: Option[Map[K, V]] = { 60 | // Get a copy of the cache, since it is mutable 61 | val res = optNonEmpty(Map(cache.toSeq: _*)) 62 | cache.clear 63 | res 64 | } 65 | def isFlushed = cache.isEmpty 66 | 67 | protected var lastEvicted: Map[K, V] = Map.empty[K, V] 68 | // TODO fancier caches will give better performance: 69 | protected lazy val cache: MMap[K, V] = (new JLinkedHashMap[K, V](capacity + 1, 0.75f, true) { 70 | override protected def removeEldestEntry(eldest: JMap.Entry[K, V]) = 71 | if (super.size > capacity) { 72 | lastEvicted += (eldest.getKey -> eldest.getValue) 73 | true 74 | } else { 75 | false 76 | } 77 | }).asScala 78 | } 79 | -------------------------------------------------------------------------------- /algebird-test/src/main/scala/com/twitter/algebird/ApplicativeLaws.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2014 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird 18 | 19 | import org.scalacheck.{ Arbitrary, Prop } 20 | import org.scalacheck.Prop.forAll 21 | 22 | /** 23 | * Basic Applicative laws, useful for testing any app. 24 | */ 25 | object ApplicativeLaws { 26 | import BaseProperties.{ HigherEq, DefaultHigherEq } 27 | 28 | def applyLaw[M[_], T, U]( 29 | eq: HigherEq[M] = new DefaultHigherEq[M])( 30 | implicit app: Applicative[M], 31 | arb: Arbitrary[T], 32 | arbFn: Arbitrary[T => U]): Prop = 33 | forAll { (t: T, fn: T => U) => eq(app.map(app.apply(t))(fn), app.apply(fn(t))) } 34 | 35 | def joinLaw[M[_], T, U]( 36 | eq: HigherEq[M] = new DefaultHigherEq[M])( 37 | implicit app: Applicative[M], 38 | arb1: Arbitrary[T], 39 | arb2: Arbitrary[U]): Prop = 40 | forAll { (t: T, u: U) => 41 | eq( 42 | app.join(app.apply(t), app.apply(u)), 43 | app.apply((t, u))) 44 | } 45 | 46 | // These follow from apply and join: 47 | 48 | def sequenceLaw[M[_], T]( 49 | eq: HigherEq[M] = new DefaultHigherEq[M])( 50 | implicit app: Applicative[M], 51 | arb: Arbitrary[Seq[T]]): Prop = 52 | forAll { (ts: Seq[T]) => eq(app.sequence(ts map { app.apply(_) }), app.apply(ts)) } 53 | 54 | def joinWithLaw[M[_], T, U, V]( 55 | eq: HigherEq[M] = new DefaultHigherEq[M])( 56 | implicit app: Applicative[M], 57 | arbT: Arbitrary[T], 58 | arbU: Arbitrary[U], 59 | arbJoinFn: Arbitrary[(T, U) => V]): Prop = 60 | forAll { (t: T, u: U, fn: (T, U) => V) => eq(app.joinWith(app.apply(t), app.apply(u))(fn), app.apply(fn(t, u))) } 61 | 62 | def applicativeLaws[M[_], T, U, V]( 63 | eq: HigherEq[M] = new DefaultHigherEq[M])( 64 | implicit app: Applicative[M], 65 | arbMt: Arbitrary[T], 66 | arbMts: Arbitrary[Seq[T]], 67 | arbMu: Arbitrary[U], 68 | arbFn: Arbitrary[T => U], 69 | arbJoinFn: Arbitrary[(T, U) => V]): Prop = 70 | applyLaw[M, T, U](eq) && 71 | joinLaw[M, T, U](eq) && 72 | sequenceLaw[M, T](eq) && 73 | joinWithLaw[M, T, U, V](eq) 74 | } 75 | 76 | -------------------------------------------------------------------------------- /algebird-test/src/main/scala/com/twitter/algebird/SucPredLaws.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2014 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird 18 | 19 | import org.scalacheck.{ Arbitrary, Prop } 20 | import org.scalacheck.Prop.forAll 21 | 22 | object SuccessibleLaws { 23 | // Should always be true: 24 | def law[T: Successible](t: T): Boolean = 25 | Successible.next(t) match { 26 | case None => true // t is the max 27 | case Some(n) => 28 | val pord = implicitly[Successible[T]].partialOrdering 29 | pord.lt(t, n) 30 | } 31 | 32 | def iterateNextIncreases[T: Successible](t: T, size: Short): Boolean = 33 | Successible.iterateNext(t).take(size.toInt).sliding(2).forall { 34 | case a :: b :: Nil => implicitly[Successible[T]].partialOrdering.lt(a, b) 35 | case a :: Nil => true 36 | case s => sys.error("should never happen: " + s) 37 | } 38 | 39 | /** 40 | * Use this to test your implementations: 41 | * property("My succ") { 42 | * successibleLaws[MyType] 43 | * } 44 | * 45 | */ 46 | def successibleLaws[T: Successible: Arbitrary]: Prop = forAll { (t: T, size: Short) => 47 | law(t) && iterateNextIncreases(t, size) 48 | } 49 | } 50 | 51 | object PredecessibleLaws { 52 | // Should always be true: 53 | def law[T: Predecessible](t: T): Boolean = 54 | Predecessible.prev(t) match { 55 | case None => true // t is the max 56 | case Some(p) => 57 | val pord = implicitly[Predecessible[T]].partialOrdering 58 | pord.lt(p, t) 59 | } 60 | 61 | def iteratePrevDecreases[T: Predecessible](t: T, size: Short): Boolean = 62 | Predecessible.iteratePrev(t).take(size.toInt).sliding(2).forall { 63 | case a :: b :: Nil => implicitly[Predecessible[T]].partialOrdering.lt(b, a) 64 | case a :: Nil => true 65 | case s => sys.error("should never happen: " + s) 66 | } 67 | 68 | /** 69 | * Use this to test your implementations: 70 | * property("My succ") { 71 | * predessibleLaws[MyType] 72 | * } 73 | * 74 | */ 75 | def predessibleLaws[T: Predecessible: Arbitrary]: Prop = forAll { (t: T, size: Short) => 76 | law(t) && iteratePrevDecreases(t, size) 77 | } 78 | } 79 | -------------------------------------------------------------------------------- /algebird-core/src/main/scala/com/twitter/algebird/matrix/SparseColumnMatrix.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird.matrix 18 | import scala.collection.mutable.{ ArrayBuffer, Map => MMap } 19 | import com.twitter.algebird.{ Monoid, AdaptiveVector } 20 | 21 | object SparseColumnMatrix { 22 | def fromSeqMap[V: Monoid](cols: Int, data: IndexedSeq[MMap[Int, V]]) = { 23 | val monoidZero = implicitly[Monoid[V]].zero 24 | SparseColumnMatrix(data.map { mm => 25 | AdaptiveVector.fromMap(mm.toMap, monoidZero, cols) 26 | }.toIndexedSeq) 27 | } 28 | } 29 | 30 | case class SparseColumnMatrix[V: Monoid](rowsByColumns: IndexedSeq[AdaptiveVector[V]]) extends AdaptiveMatrix[V] { 31 | /** Row is the outer Seq, the columns are the inner vectors. */ 32 | 33 | val valueMonoid = implicitly[Monoid[V]] 34 | 35 | override def rows: Int = rowsByColumns.size 36 | 37 | override def cols: Int = rowsByColumns(0).size 38 | 39 | def getValue(position: (Int, Int)): V = rowsByColumns(position._1)(position._2) 40 | 41 | def updated(position: (Int, Int), value: V): SparseColumnMatrix[V] = { 42 | val (row, col) = position 43 | SparseColumnMatrix[V](rowsByColumns.updated(row, rowsByColumns(row).updated(col, value))) 44 | } 45 | 46 | override def updateInto(buffer: ArrayBuffer[V]) { 47 | val lcols = cols 48 | var row = 0 49 | while (row < rows) { 50 | val iter = rowsByColumns(row).denseIterator 51 | while (iter.hasNext) { 52 | val (col, value) = iter.next 53 | val indx = row * lcols + col 54 | buffer(indx) = valueMonoid.plus(buffer(indx), value) 55 | } 56 | row += 1 57 | } 58 | } 59 | 60 | def toDense: DenseMatrix[V] = { 61 | val buf = ArrayBuffer.fill(size)(valueMonoid.zero) 62 | updateInto(buf) 63 | DenseMatrix(rows, cols, buf) 64 | } 65 | 66 | override def toString: String = { 67 | val builder = new scala.collection.mutable.StringBuilder() 68 | builder ++= "Row: %d, Cols: %d. Dense elements:\n".format(rows, cols) 69 | rowsByColumns.foreach { v => 70 | builder ++= v.denseIterator.toList.toString 71 | builder ++= "\n" 72 | } 73 | builder.toString 74 | } 75 | } -------------------------------------------------------------------------------- /algebird-core/src/main/scala/com/twitter/algebird/IndexedSeq.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird 18 | 19 | /** 20 | * Note that this works similar to Semigroup[Map[Int,T]] not like Semigroup[List[T]] 21 | * This does element-wise operations, like standard vector math, not concatenation, 22 | * like Semigroup[String] or Semigroup[List[T]] 23 | * 24 | * If l.size != r.size, then only sums the elements up to the index min(l.size, r.size); appends 25 | * the remainder to the result. 26 | */ 27 | class IndexedSeqSemigroup[T](implicit semi: Semigroup[T]) extends Semigroup[IndexedSeq[T]] { 28 | 29 | def plus(left: IndexedSeq[T], right: IndexedSeq[T]): IndexedSeq[T] = { 30 | // We need summands to be the same length 31 | val (leftSummand, rightSummand, remainder) = if (left.size > right.size) { 32 | (left.view(0, right.size), 33 | right, 34 | left.view(right.size, left.size)) 35 | } else { 36 | (left, 37 | right.view(0, left.size), 38 | right.view(left.size, right.size)) 39 | } 40 | 41 | val sum = leftSummand 42 | .zip(rightSummand) 43 | .map { tup => semi.plus(tup._1, tup._2) } 44 | 45 | (sum ++ remainder).toIndexedSeq 46 | } 47 | } 48 | 49 | class IndexedSeqMonoid[T](implicit mont: Monoid[T]) extends IndexedSeqSemigroup[T] with Monoid[IndexedSeq[T]] { 50 | def zero = IndexedSeq.empty[T] 51 | override def isNonZero(v: IndexedSeq[T]) = 52 | v.exists { t => mont.isNonZero(t) } 53 | } 54 | 55 | class IndexedSeqGroup[T](implicit grp: Group[T]) extends IndexedSeqMonoid[T]()(grp) 56 | with Group[IndexedSeq[T]] { 57 | override def negate(g: IndexedSeq[T]): IndexedSeq[T] = g.map { grp.negate(_) } 58 | } 59 | 60 | class IndexedSeqRing[T](implicit rng: Ring[T]) extends IndexedSeqGroup[T]()(rng) 61 | with Ring[IndexedSeq[T]] { 62 | 63 | // TODO 64 | def one = sys.error("IndexedSeqRing.one is unimplemented. It's a lot of work, and almost never used") 65 | 66 | def times(left: IndexedSeq[T], right: IndexedSeq[T]): IndexedSeq[T] = 67 | // We don't need to pad, because 0 * x = 0 68 | left.view 69 | .zip(right) 70 | .map { tup => rng.times(tup._1, tup._2) } 71 | .toIndexedSeq 72 | } 73 | -------------------------------------------------------------------------------- /algebird-core/src/main/scala/com/twitter/algebird/monad/Trampoline.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2013 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird.monad 18 | 19 | import com.twitter.algebird.Monad 20 | 21 | // A simple trampoline implementation which we copied for the State monad 22 | sealed trait Trampoline[+A] { 23 | def map[B](fn: A => B): Trampoline[B] 24 | def flatMap[B](fn: A => Trampoline[B]): Trampoline[B] = FlatMapped(this, fn) 25 | /** 26 | * get triggers the computation which is run exactly once 27 | */ 28 | def get: A 29 | } 30 | 31 | final case class Done[A](override val get: A) extends Trampoline[A] { 32 | def map[B](fn: A => B) = Done(fn(get)) 33 | } 34 | 35 | final case class FlatMapped[C, A](start: Trampoline[C], fn: C => Trampoline[A]) extends Trampoline[A] { 36 | def map[B](fn: A => B) = FlatMapped(this, { (a: A) => Done(fn(a)) }) 37 | lazy val get = Trampoline.run(this) 38 | } 39 | 40 | object Trampoline { 41 | val unit: Trampoline[Unit] = Done(()) 42 | def apply[A](a: A): Trampoline[A] = Done(a) 43 | def lazyVal[A](a: => A): Trampoline[A] = FlatMapped(unit, { (u: Unit) => Done(a) }) 44 | /** 45 | * Use this to call to another trampoline returning function 46 | * you break the effect of this if you directly recursively call a Trampoline 47 | * returning function 48 | */ 49 | def call[A](layzee: => Trampoline[A]): Trampoline[A] = FlatMapped(unit, { (u: Unit) => layzee }) 50 | implicit val Monad: Monad[Trampoline] = new Monad[Trampoline] { 51 | def apply[A](a: A) = Done(a) 52 | def flatMap[A, B](start: Trampoline[A])(fn: A => Trampoline[B]) = start.flatMap(fn) 53 | } 54 | // This triggers evaluation. Will reevaluate every time. Prefer .get 55 | def run[A](tramp: Trampoline[A]): A = { 56 | @annotation.tailrec 57 | def loop(start: Trampoline[Any], stack: List[(Any) => Trampoline[Any]]): Any = { 58 | start match { 59 | case Done(a) => stack match { 60 | case next :: tail => loop(next(a), tail) 61 | case Nil => a 62 | } 63 | case FlatMapped(item, fn) => loop(item, fn :: stack) 64 | } 65 | } 66 | // Sorry for the cast, but it is tough to get the types right without a lot of wrapping 67 | loop(tramp, Nil).asInstanceOf[A] 68 | } 69 | } 70 | -------------------------------------------------------------------------------- /algebird-core/src/main/scala/com/twitter/algebird/mutable/PriorityQueueMonoid.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | package com.twitter.algebird.mutable 17 | 18 | import com.twitter.algebird.Monoid 19 | 20 | import scala.annotation.tailrec 21 | import scala.collection.JavaConverters._ 22 | 23 | import java.util.PriorityQueue 24 | 25 | /** 26 | * for sort-with take and better performance over large values 27 | * The priority queues should be MAX queues, i.e. the ones we want least 28 | * should be in the .peek position 29 | * This is MUCH Faster for Top-K algorithms 30 | * Note this is MUTABLE. When you put something in plus, it is changed! 31 | */ 32 | class PriorityQueueMonoid[K](max: Int)(implicit ord: Ordering[K]) extends Monoid[PriorityQueue[K]] { 33 | 34 | require(max > 0, "PriorityQueueMonoid requires keeping at least 1 item") 35 | // Java throws if you try to make a queue size 0 36 | protected val MINQUEUESIZE = 1 37 | def build(k: K): PriorityQueue[K] = { 38 | val q = new PriorityQueue[K](1, ord.reverse); 39 | q.add(k) 40 | q 41 | } 42 | def build(items: Iterable[K]): PriorityQueue[K] = { 43 | val q = new PriorityQueue(items.size max MINQUEUESIZE, ord.reverse); 44 | items.foreach { item => 45 | if (q.size < max || ord.lteq(item, q.peek)) { 46 | q.add(item) 47 | } 48 | } 49 | limit(q) 50 | q 51 | } 52 | protected def limit(q: PriorityQueue[K]) { 53 | while (q.size > max) { q.poll() } 54 | } 55 | 56 | override def zero = new PriorityQueue[K](MINQUEUESIZE, ord.reverse) 57 | override def isNonZero(q: PriorityQueue[K]) = q.size > 0 58 | 59 | override def plus(left: PriorityQueue[K], right: PriorityQueue[K]): PriorityQueue[K] = { 60 | val (bigger, smaller) = if (left.size >= right.size) (left, right) else (right, left) 61 | var biggest = bigger.peek 62 | 63 | var next = smaller.poll 64 | while (next != null) { 65 | if (bigger.size < max) { 66 | // we may have increased the biggest value: 67 | biggest = ord.max(biggest, next) 68 | bigger.add(next) 69 | } else if (ord.lteq(next, biggest)) { 70 | // this cannot increase the biggest 71 | bigger.add(next) 72 | } 73 | next = smaller.poll 74 | } 75 | limit(bigger) 76 | bigger 77 | } 78 | } 79 | -------------------------------------------------------------------------------- /algebird-test/src/main/scala/com/twitter/algebird/BaseVectorSpaceProperties.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird 18 | 19 | import org.scalacheck.Arbitrary 20 | import org.scalacheck.Prop.forAll 21 | 22 | /** 23 | * Base properties for VectorSpace tests. 24 | */ 25 | 26 | object BaseVectorSpaceProperties { 27 | def isEqualIfZero[F, C[_]](eqfn: (C[F], C[F]) => Boolean)(implicit vs: VectorSpace[F, C], arb: Arbitrary[C[F]]) = 28 | forAll { (a: C[F]) => 29 | eqfn(VectorSpace.scale(vs.field.zero, a), vs.group.zero) 30 | } 31 | 32 | def distributesWithPlus[F, C[_]](eqfn: (C[F], C[F]) => Boolean)(implicit vs: VectorSpace[F, C], arbC: Arbitrary[C[F]], arbF: Arbitrary[F]) = 33 | forAll { (a: C[F], b: C[F], c: F) => 34 | val v1 = VectorSpace.scale(c, vs.group.plus(a, b)) 35 | val v2 = vs.group.plus(VectorSpace.scale(c, a), VectorSpace.scale(c, b)) 36 | eqfn(v1, v2) 37 | } 38 | 39 | def isAssociative[F, C[_]](eqfn: (C[F], C[F]) => Boolean)(implicit vs: VectorSpace[F, C], arbC: Arbitrary[C[F]], arbF: Arbitrary[F]) = 40 | forAll { (a: C[F], b: F, c: F) => 41 | val v1 = VectorSpace.scale(c, VectorSpace.scale(b, a)) 42 | val v2 = VectorSpace.scale(vs.field.times(c, b), a) 43 | eqfn(v1, v2) 44 | } 45 | 46 | def identityOne[F, C[_]](eqfn: (C[F], C[F]) => Boolean)(implicit vs: VectorSpace[F, C], arb: Arbitrary[C[F]]) = 47 | forAll { (a: C[F]) => 48 | eqfn(VectorSpace.scale(vs.field.one, a), a) 49 | } 50 | 51 | def distributesOverScalarPlus[F, C[_]](eqfn: (C[F], C[F]) => Boolean)(implicit vs: VectorSpace[F, C], arbC: Arbitrary[C[F]], arbF: Arbitrary[F]) = 52 | forAll { (a: C[F], b: F, c: F) => 53 | val v1 = VectorSpace.scale(vs.field.plus(b, c), a) 54 | val v2 = vs.group.plus(VectorSpace.scale(b, a), VectorSpace.scale(c, a)) 55 | eqfn(v1, v2) 56 | } 57 | 58 | def vectorSpaceLaws[F, C[_]](eqfn: (C[F], C[F]) => Boolean)(implicit vs: VectorSpace[F, C], arbC: Arbitrary[C[F]], arbF: Arbitrary[F]) = 59 | isEqualIfZero(eqfn) && distributesWithPlus(eqfn) && isAssociative(eqfn) && identityOne(eqfn) && distributesOverScalarPlus(eqfn) 60 | 61 | def beCloseTo(a: Double, b: Double) = 62 | a == b || (math.abs(a - b) / math.abs(a)) < 1e-10 || (a.isInfinite && b.isInfinite) || a.isNaN || b.isNaN 63 | } 64 | -------------------------------------------------------------------------------- /algebird-test/src/test/scala/com/twitter/algebird/Generators.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2013 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird 18 | 19 | import org.scalacheck.Arbitrary 20 | import org.scalacheck.Gen 21 | import org.scalacheck.Gen._ 22 | 23 | /** 24 | * Generators useful in testing Interval 25 | */ 26 | object Generators { 27 | // The new scalacheck oneOf results in diverging implicits here 28 | // should follow up an investigation? 29 | def oneOf[T](g1: Gen[T], g2: Gen[T], gs: Gen[T]*) = for { 30 | i <- choose(0, gs.length + 1) 31 | x <- if (i == 0) g1 else if (i == 1) g2 else gs(i - 2) 32 | } yield x 33 | 34 | implicit def intervalArb[T](implicit arb: Arbitrary[T], ord: Ordering[T]): Arbitrary[Interval[T]] = 35 | Arbitrary(oneOf(genUniverse, genEmpty, genInclusiveLower, genExclusiveLower, genInclusiveUpper, genExclusiveUpper, genIntersection)) 36 | 37 | implicit def lowerIntArb[T: Arbitrary: Ordering]: Arbitrary[Lower[T]] = 38 | Arbitrary(oneOf(genInclusiveLower, genExclusiveLower)) 39 | 40 | implicit def upperIntArb[T: Arbitrary: Ordering]: Arbitrary[Upper[T]] = 41 | Arbitrary(oneOf(genInclusiveUpper, genExclusiveUpper)) 42 | 43 | implicit def intersectionArb[T: Arbitrary: Ordering]: Arbitrary[Interval.GenIntersection[T]] = 44 | Arbitrary(genIntersection) 45 | 46 | def genUniverse[T: Arbitrary: Ordering] = 47 | for { 48 | u <- Unit 49 | } yield Universe[T]() 50 | 51 | def genEmpty[T: Arbitrary: Ordering] = 52 | for { 53 | u <- Unit 54 | } yield Empty[T]() 55 | 56 | def genInclusiveLower[T: Arbitrary: Ordering] = 57 | for { 58 | l <- Arbitrary.arbitrary[T] 59 | } yield InclusiveLower(l) 60 | 61 | def genExclusiveLower[T: Arbitrary: Ordering] = 62 | for { 63 | l <- Arbitrary.arbitrary[T] 64 | } yield ExclusiveLower(l) 65 | 66 | def genInclusiveUpper[T: Arbitrary: Ordering] = 67 | for { 68 | u <- Arbitrary.arbitrary[T] 69 | } yield InclusiveUpper(u) 70 | 71 | def genExclusiveUpper[T: Arbitrary: Ordering] = 72 | for { 73 | u <- Arbitrary.arbitrary[T] 74 | } yield ExclusiveUpper(u) 75 | 76 | def genIntersection[T: Arbitrary: Ordering] = 77 | for { 78 | l <- Arbitrary.arbitrary[Lower[T]] 79 | u <- Arbitrary.arbitrary[Upper[T]] if l.intersects(u) 80 | } yield Intersection(l, u) 81 | } 82 | -------------------------------------------------------------------------------- /algebird-test/src/test/scala/com/twitter/algebird/TopKTests.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird 18 | 19 | import com.twitter.algebird.mutable.PriorityQueueMonoid 20 | import java.util.PriorityQueue 21 | 22 | import scala.collection.JavaConverters._ 23 | 24 | import org.scalatest._ 25 | 26 | import org.scalatest.prop.PropertyChecks 27 | import org.scalacheck.{ Gen, Arbitrary } 28 | 29 | class TopKTests extends PropSpec with PropertyChecks with Matchers { 30 | import BaseProperties._ 31 | val SIZE = 10 32 | 33 | implicit def qmonoid = new PriorityQueueMonoid[Int](SIZE) 34 | implicit def queueArb = Arbitrary { 35 | implicitly[Arbitrary[List[Int]]].arbitrary.map { qmonoid.build(_) } 36 | } 37 | 38 | def q2l(q1: PriorityQueue[Int]): List[Int] = q1.iterator.asScala.toList.sorted 39 | 40 | def eqFn(q1: PriorityQueue[Int], q2: PriorityQueue[Int]): Boolean = { 41 | q2l(q1) == q2l(q2) 42 | } 43 | 44 | def pqIsCorrect(items: List[List[Int]]): Boolean = { 45 | val correct = items.flatten.sorted.take(SIZE) 46 | // Have to do this last since this monoid is mutating inputs 47 | q2l(Monoid.sum(items.map { l => qmonoid.build(l) })) == correct 48 | } 49 | 50 | property("PriorityQueueMonoid works") { 51 | forAll { (items: List[List[Int]]) => 52 | assert(pqIsCorrect(items)) 53 | } 54 | } 55 | /** 56 | * The following were specific bugs that we failed some prior 57 | * scalacheck (yay for randomized testing) 58 | */ 59 | val pqPriorBugs = Seq(List(List(1, 1, 1, 2), List(0, 0, 0, 0, 0, 0, 0))) 60 | property("Specific regressions are handled") { 61 | pqPriorBugs.forall(pqIsCorrect(_)) 62 | } 63 | 64 | property("PriorityQueueMonoid is a Monoid") { 65 | monoidLawsEq[PriorityQueue[Int]](eqFn) 66 | } 67 | 68 | implicit def tkmonoid = new TopKMonoid[Int](SIZE) 69 | 70 | implicit def topkArb = Arbitrary { 71 | implicitly[Arbitrary[List[Int]]].arbitrary.map { tkmonoid.build(_) } 72 | } 73 | 74 | property("TopKMonoid works") { 75 | forAll { (its: List[List[Int]]) => 76 | val correct = its.flatten.sorted.take(SIZE) 77 | assert(Equiv[List[Int]].equiv(Monoid.sum(its.map { l => tkmonoid.build(l) }).items, correct)) 78 | } 79 | } 80 | 81 | property("TopKMonoid is a Monoid") { 82 | monoidLawsEq[PriorityQueue[Int]](eqFn) 83 | } 84 | 85 | } 86 | -------------------------------------------------------------------------------- /algebird-test/src/test/scala/com/twitter/algebird/SGDTest.scala: -------------------------------------------------------------------------------- 1 | package com.twitter.algebird 2 | 3 | import org.scalatest._ 4 | 5 | import org.scalatest.{ PropSpec, Matchers } 6 | import org.scalatest.prop.PropertyChecks 7 | import org.scalacheck.{ Gen, Arbitrary } 8 | import java.lang.AssertionError 9 | import java.util.Arrays 10 | 11 | class SGDLaws extends PropSpec with PropertyChecks with Matchers { 12 | import BaseProperties._ 13 | 14 | implicit val sgdMonoid = new SGDMonoid(SGD.constantStep(0.001), SGD.linearGradient) 15 | val zeroStepMonoid = new SGDMonoid(SGD.constantStep(0.0), SGD.linearGradient) 16 | 17 | val (m, b) = (2.0, 4.0) 18 | val eps = 1e-3 19 | 20 | val sgdPosGen = for ( 21 | x <- Gen.choose(0.0, 1.0); 22 | n <- Gen.choose(0.0, 0.001) 23 | ) yield SGDPos((m * x + b + n, IndexedSeq(x))) 24 | 25 | val sgdWGen = for ( 26 | cnt <- Gen.choose(0L, 100000L); 27 | m <- Gen.choose(-10.0, 10.0); 28 | b <- Gen.choose(-10.0, 10.0) 29 | ) yield SGDWeights(cnt, Vector(m, b)) 30 | 31 | val zeroGen = Gen.const(SGDZero) 32 | 33 | implicit val sgdPos = Arbitrary(sgdPosGen) 34 | implicit val sgdWArb = Arbitrary(sgdWGen) 35 | implicit val sgdArb: Arbitrary[SGD[(Double, IndexedSeq[Double])]] = Arbitrary { 36 | Gen.oneOf(sgdWGen, sgdPosGen, zeroGen) 37 | } 38 | 39 | property("is a Monoid") { 40 | monoidLaws[SGD[(Double, IndexedSeq[Double])]] 41 | } 42 | 43 | property("Gradient is zero on the line") { 44 | forAll { (w: SGDWeights, x: Double) => 45 | val m = w.weights(0) 46 | val b = w.weights(1) 47 | val y = m * x + b 48 | 49 | assert(y.isInfinity || { 50 | val pos = (y, IndexedSeq(x)) 51 | val grad = SGD.linearGradient(w.weights, pos) 52 | (scala.math.abs(grad(0)) < eps) && (scala.math.abs(grad(1)) < eps) 53 | }) 54 | } 55 | } 56 | 57 | property("Gradient at x=0 has zero first component") { 58 | forAll { (w: SGDWeights, y: Double) => 59 | assert(SGD.linearGradient(w.weights, (y, IndexedSeq(0.0)))(0) == 0.0) 60 | } 61 | } 62 | 63 | property("Zero-step leaves Weights unchanged") { 64 | forAll { 65 | (w: SGDWeights, pos: SGDPos[(Double, IndexedSeq[Double])]) => 66 | val next = zeroStepMonoid.newWeights(w, pos.pos.head) 67 | assert(next.weights == w.weights && next.count == (w.count + 1L)) 68 | } 69 | } 70 | 71 | def minus(x: IndexedSeq[Double], y: IndexedSeq[Double]): IndexedSeq[Double] = { 72 | x.zip(y).map { case (x: Double, y: Double) => x - y } 73 | } 74 | 75 | val oneStepMonoid = new SGDMonoid(SGD.constantStep(1.0), SGD.linearGradient) 76 | 77 | property("unit step can be undone by adding gradient") { 78 | forAll { 79 | (w: SGDWeights, pos: SGDPos[(Double, IndexedSeq[Double])]) => 80 | val next = oneStepMonoid.newWeights(w, pos.pos.head) 81 | assert(next.weights == minus(w.weights, SGD.linearGradient(w.weights, pos.pos.head))) 82 | } 83 | } 84 | } 85 | -------------------------------------------------------------------------------- /algebird-core/src/main/scala/com/twitter/algebird/SummingQueue.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird 18 | 19 | /** 20 | * A useful utility for aggregation systems: you buffer up some number of items 21 | * in a thread-safe way, and when you have at most K of them, you sum them all 22 | * together. A good use-case of this is doing a limited preaggregation before 23 | * sending on to a next phase (from mappers to reducers on Hadoop, or between 24 | * storm bolts). 25 | * 26 | * Without this finite buffer history, an aggregated item could build up infinite 27 | * history, and therefore it is unbounded in the error you could introduce by 28 | * losing the buffer. 29 | * 30 | * @author Ashu Singhal 31 | * @author Oscar Boykin 32 | */ 33 | 34 | import java.io.Serializable 35 | import java.util.concurrent.ArrayBlockingQueue 36 | 37 | import scala.collection.JavaConverters._ 38 | import scala.collection.mutable.ListBuffer 39 | 40 | object SummingQueue { 41 | def apply[V: Semigroup](cap: Int): SummingQueue[V] = new SummingQueue(cap) 42 | } 43 | 44 | class SummingQueue[V] private (capacity: Int)(override implicit val semigroup: Semigroup[V]) 45 | extends StatefulSummer[V] { 46 | 47 | private val queueOption: Option[ArrayBlockingQueue[V]] = 48 | if (capacity > 0) Some(new ArrayBlockingQueue[V](capacity, true)) else None 49 | 50 | /** 51 | * puts an item to the queue, optionally sums up the queue and returns value 52 | * This never blocks interally. It uses offer. If the queue is full, we drain, 53 | * sum the queue. 54 | */ 55 | final def put(item: V): Option[V] = 56 | if (queueOption.isDefined) { 57 | queueOption.flatMap { queue => 58 | if (!queue.offer(item)) { 59 | // Queue is full, do the work: 60 | Monoid.plus(flush, Some(item)) 61 | } else { 62 | // We are in the queue 63 | None 64 | } 65 | } 66 | } else { Some(item) } 67 | 68 | def apply(v: V): Option[V] = put(v) 69 | 70 | /** 71 | * drain the queue and return the sum. If empty, return None 72 | */ 73 | def flush: Option[V] = { 74 | queueOption.flatMap { queue => 75 | val toSum = ListBuffer[V]() 76 | queue.drainTo(toSum.asJava) 77 | Semigroup.sumOption(toSum) 78 | } 79 | } 80 | def isFlushed: Boolean = queueOption.map { _.size == 0 }.getOrElse(true) 81 | } 82 | -------------------------------------------------------------------------------- /algebird-util/src/test/scala/com/twitter/algebird/util/TunnelMonoidProperties.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | package com.twitter.algebird.util 17 | 18 | import com.twitter.algebird._ 19 | import com.twitter.util.{ Await, Future } 20 | import scala.util.Random 21 | import org.scalatest.{ PropSpec, Matchers } 22 | import org.scalatest.prop.PropertyChecks 23 | import org.scalacheck.Arbitrary 24 | 25 | object TunnelMonoidProperties { 26 | def testTunnelMonoid[I, V](makeRandomInput: Int => I, 27 | makeTunnel: I => V, 28 | collapseFinalValues: (V, Seq[V], I) => Seq[Future[I]])(implicit monoid: Monoid[I], 29 | superMonoid: Monoid[V]) = { 30 | val r = new Random 31 | val numbers = (1 to 40).map { _ => makeRandomInput(r.nextInt) } 32 | def helper(seeds: Seq[I], toFeed: I) = { 33 | val tunnels = seeds.map(makeTunnel) 34 | @annotation.tailrec 35 | def process(tunnels: Seq[V]): V = { 36 | val size = tunnels.size 37 | if (size > 2) { 38 | val (tun1, tun2) = tunnels.splitAt(r.nextInt(size - 2)) 39 | val (of2, rest) = tun2.splitAt(2) 40 | process(tun1 ++ (Monoid.plus(of2.head, of2.tail.head) +: rest)) 41 | } else if (size == 2) { 42 | Monoid.plus(tunnels.head, tunnels.tail.head) 43 | } else { 44 | tunnels.head 45 | } 46 | } 47 | collapseFinalValues(process(tunnels), tunnels, toFeed) 48 | } 49 | numbers.forall { _ => 50 | val toFeed = makeRandomInput(r.nextInt) 51 | val finalResults = helper(numbers, toFeed) zip helper(numbers, toFeed) map { 52 | case (f1, f2) => for { 53 | b1 <- f1 54 | b2 <- f2 55 | } yield b1 == b2 56 | } 57 | Await.result(Future.collect(finalResults).map { _.forall(identity) }) 58 | } 59 | } 60 | } 61 | 62 | class TunnelMonoidProperties extends PropSpec with PropertyChecks with Matchers { 63 | import TunnelMonoidProperties._ 64 | 65 | implicit val monoid = new Monoid[Int] { 66 | val zero = 0 67 | def plus(older: Int, newer: Int): Int = older + newer 68 | } 69 | 70 | property("associative") { 71 | def makeTunnel(seed: Int) = Tunnel.toIncrement(seed) 72 | def collapseFinalValues(finalTunnel: Tunnel[Int], tunnels: Seq[Tunnel[Int]], toFeed: Int) = 73 | finalTunnel(toFeed) +: tunnels.map { _.future } 74 | 75 | testTunnelMonoid[Int, Tunnel[Int]](identity, makeTunnel, collapseFinalValues) 76 | } 77 | } -------------------------------------------------------------------------------- /algebird-util/src/main/scala/com/twitter/algebird/util/summer/AsyncSummer.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | package com.twitter.algebird.util.summer 17 | 18 | import com.twitter.algebird._ 19 | import com.twitter.util.{ Duration, Future } 20 | 21 | /** 22 | * @author Ian O Connell 23 | */ 24 | 25 | trait AsyncSummer[T, +M <: Iterable[T]] { self => 26 | def flush: Future[M] 27 | def tick: Future[M] 28 | def add(t: T) = addAll(Iterator(t)) 29 | def addAll(vals: TraversableOnce[T]): Future[M] 30 | 31 | def isFlushed: Boolean 32 | def cleanup: Future[Unit] = Future.Unit 33 | def withCleanup(cleanup: () => Future[Unit]): AsyncSummer[T, M] = { 34 | val oldSelf = self 35 | new AsyncSummerProxy[T, M] { 36 | override val self = oldSelf 37 | override def cleanup = { 38 | oldSelf.cleanup.flatMap { _ => cleanup } 39 | } 40 | } 41 | } 42 | } 43 | 44 | trait AsyncSummerProxy[T, +M <: Iterable[T]] extends AsyncSummer[T, M] { 45 | def self: AsyncSummer[T, M] 46 | def flush = self.flush 47 | def tick = self.tick 48 | override def add(t: T) = self.add(t) 49 | def addAll(vals: TraversableOnce[T]) = self.addAll(vals) 50 | def isFlushed = self.isFlushed 51 | override def cleanup: Future[Unit] = self.cleanup 52 | } 53 | 54 | trait WithFlushConditions[T, M <: Iterable[T]] extends AsyncSummer[T, M] { 55 | private[this] val className = getClass.getName 56 | protected var lastDump: Long = System.currentTimeMillis 57 | protected def softMemoryFlush: MemoryFlushPercent 58 | protected def flushFrequency: FlushFrequency 59 | protected def emptyResult: M 60 | 61 | protected def memoryIncr: Incrementor 62 | protected def timeoutIncr: Incrementor 63 | 64 | protected def timedOut = (System.currentTimeMillis - lastDump) >= flushFrequency.v.inMilliseconds 65 | protected lazy val runtime = Runtime.getRuntime 66 | 67 | protected def memoryWaterMark = { 68 | val used = ((runtime.totalMemory - runtime.freeMemory).toDouble * 100) / runtime.maxMemory 69 | used > softMemoryFlush.v 70 | } 71 | 72 | def tick: Future[M] = { 73 | if (timedOut) { 74 | timeoutIncr.incr 75 | lastDump = System.currentTimeMillis // reset the timeout condition 76 | flush 77 | } else if (memoryWaterMark) { 78 | memoryIncr.incr 79 | lastDump = System.currentTimeMillis // reset the timeout condition 80 | flush 81 | } else { 82 | Future.value(emptyResult) 83 | } 84 | } 85 | } 86 | -------------------------------------------------------------------------------- /algebird-core/src/main/scala/com/twitter/algebird/monad/Reader.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2013 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird.monad 18 | 19 | import com.twitter.algebird.Monad 20 | 21 | // TODO this is general, move somewhere better 22 | 23 | // Reader Monad, represents a series of operations that mutate some environment 24 | // type (the input to the function) 25 | 26 | sealed trait Reader[-Env, +T] { 27 | def apply(env: Env): T 28 | def flatMap[E1 <: Env, U](next: T => Reader[E1, U]): Reader[E1, U] = 29 | FlatMappedReader[E1, T, U](this, next) 30 | def map[U](thatFn: T => U): Reader[Env, U] = 31 | FlatMappedReader(this, { (t: T) => ConstantReader(thatFn(t)) }) 32 | } 33 | 34 | final case class ConstantReader[+T](get: T) extends Reader[Any, T] { 35 | override def apply(env: Any) = get 36 | override def map[U](fn: T => U) = ConstantReader(fn(get)) 37 | override def flatMap[E1 <: Any, U](next: T => Reader[E1, U]): Reader[E1, U] = next(get) 38 | } 39 | final case class ReaderFn[E, +T](fn: E => T) extends Reader[E, T] { 40 | override def apply(env: E) = fn(env) 41 | } 42 | final case class FlatMappedReader[E, U, +T](first: Reader[E, U], fn: U => Reader[E, T]) extends Reader[E, T] { 43 | def apply(env: E): T = { 44 | @annotation.tailrec 45 | def loop(r: Reader[E, Any], stack: List[(Any) => Reader[E, Any]]): Any = 46 | r match { 47 | case ConstantReader(get) => stack match { 48 | case head :: tail => loop(head(get), tail) 49 | case Nil => get 50 | } 51 | case ReaderFn(fn) => stack match { 52 | case head :: tail => loop(head(fn(env)), tail) 53 | case Nil => fn(env) 54 | } 55 | case FlatMappedReader(first, nextFn) => loop(first, nextFn :: stack) 56 | } 57 | loop(first, List(fn.asInstanceOf[(Any) => Reader[E, Any]])).asInstanceOf[T] 58 | } 59 | } 60 | 61 | object Reader { 62 | def const[T](t: T): Reader[Any, T] = ConstantReader(t) 63 | implicit def apply[E, T](fn: (E) => T): Reader[E, T] = ReaderFn(fn) 64 | 65 | class ReaderM[Env] extends Monad[({ type Result[T] = Reader[Env, T] })#Result] { 66 | def apply[T](t: T) = ConstantReader(t) 67 | def flatMap[T, U](self: Reader[Env, T])(next: T => Reader[Env, U]) = self.flatMap(next) 68 | override def map[T, U](self: Reader[Env, T])(fn: T => U) = self.map(fn) 69 | } 70 | implicit def monad[Env]: Monad[({ type Result[T] = Reader[Env, T] })#Result] = new ReaderM[Env] 71 | } 72 | -------------------------------------------------------------------------------- /algebird-caliper/src/test/scala/com/twitter/algebird/caliper/CMSBenchmark.scala: -------------------------------------------------------------------------------- 1 | package com.twitter.algebird.caliper 2 | 3 | import com.google.caliper.{ Param, SimpleBenchmark } 4 | import com.twitter.algebird.{ TopPctCMS, TopCMS, CMSHasherImplicits, TopPctCMSMonoid } 5 | 6 | /** 7 | * Benchmarks the Count-Min sketch implementation in Algebird. 8 | * 9 | * We benchmark different `K` types as well as different input data streams. 10 | */ 11 | // Once we can convince cappi (https://github.com/softprops/capp) -- the sbt plugin we use to run 12 | // caliper benchmarks -- to work with the latest caliper 1.0-beta-1, we would: 13 | // - Let `CMSBenchmark` extend `Benchmark` (instead of `SimpleBenchmark`) 14 | // - Annotate `timePlus` with `@MacroBenchmark`. 15 | class CMSBenchmark extends SimpleBenchmark { 16 | 17 | @Param(Array("0.1", "0.005")) 18 | val eps: Double = 0.0 19 | 20 | @Param(Array("0.0000001" /* 1E-8 */ )) 21 | val delta: Double = 0.0 22 | 23 | @Param(Array("0.2")) 24 | val heavyHittersPct: Double = 0.0 25 | 26 | @Param(Array("100")) 27 | val operations: Int = 0 // Number of operations per benchmark repetition (cf. `reps`) 28 | 29 | @Param(Array("2048")) 30 | val maxBits: Int = 0 31 | 32 | var random: scala.util.Random = _ 33 | var cmsLongMonoid: TopPctCMSMonoid[Long] = _ 34 | var cmsBigIntMonoid: TopPctCMSMonoid[BigInt] = _ 35 | 36 | override def setUp { 37 | // Required import of implicit values (e.g. for BigInt- or Long-backed CMS instances) 38 | import CMSHasherImplicits._ 39 | 40 | cmsLongMonoid = { 41 | val seed = 1 42 | TopPctCMS.monoid[Long](eps, delta, seed, heavyHittersPct) 43 | } 44 | 45 | cmsBigIntMonoid = { 46 | val seed = 1 47 | TopPctCMS.monoid[BigInt](eps, delta, seed, heavyHittersPct) 48 | } 49 | 50 | random = new scala.util.Random 51 | } 52 | 53 | // Case A (K=Long): We count the first hundred integers, i.e. [1, 100] 54 | def timePlusOfFirstHundredIntegersWithLongCms(reps: Int): Int = { 55 | var dummy = 0 56 | while (dummy < reps) { 57 | (1 to operations).view.foldLeft(cmsLongMonoid.zero)((l, r) => { l ++ cmsLongMonoid.create(r) }) 58 | dummy += 1 59 | } 60 | dummy 61 | } 62 | 63 | // Case B.1 (K=BigInt): We count the first hundred integers, i.e. [1, 100] 64 | def timePlusOfFirstHundredIntegersWithBigIntCms(reps: Int): Int = { 65 | var dummy = 0 66 | while (dummy < reps) { 67 | (1 to operations).view.foldLeft(cmsBigIntMonoid.zero)((l, r) => { l ++ cmsBigIntMonoid.create(r) }) 68 | dummy += 1 69 | } 70 | dummy 71 | } 72 | 73 | // Case B.2 (K=BigInt): We draw numbers randomly from a 2^maxBits address space 74 | def timePlusOfRandom2048BitNumbersWithBigIntCms(reps: Int): Int = { 75 | var dummy = 0 76 | while (dummy < reps) { 77 | (1 to operations).view.foldLeft(cmsBigIntMonoid.zero)((l, r) => { 78 | val n = scala.math.BigInt(maxBits, random) 79 | l ++ cmsBigIntMonoid.create(n) 80 | }) 81 | dummy += 1 82 | } 83 | dummy 84 | } 85 | 86 | } -------------------------------------------------------------------------------- /algebird-core/src/main/scala/com/twitter/algebird/AveragedValue.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird 18 | 19 | object AveragedValue { 20 | implicit val group = AveragedGroup 21 | def aggregator: Aggregator[Double, AveragedValue, Double] = Averager 22 | def numericAggregator[N](implicit num: Numeric[N]): MonoidAggregator[N, AveragedValue, Double] = 23 | Aggregator.prepareMonoid { n: N => AveragedValue(num.toDouble(n)) } 24 | .andThenPresent(_.value) 25 | 26 | def apply[V <% Double](v: V) = new AveragedValue(1L, v) 27 | def apply[V <% Double](c: Long, v: V) = new AveragedValue(c, v) 28 | } 29 | 30 | case class AveragedValue(count: Long, value: Double) 31 | 32 | object AveragedGroup extends Group[AveragedValue] { 33 | // When combining averages, if the counts sizes are too close we should use a different 34 | // algorithm. This constant defines how close the ratio of the smaller to the total count 35 | // can be: 36 | private val STABILITY_CONSTANT = 0.1 37 | /** 38 | * Uses a more stable online algorithm which should 39 | * be suitable for large numbers of records 40 | * similar to: 41 | * http://en.wikipedia.org/wiki/Algorithms_for_calculating_variance#Parallel_algorithm 42 | */ 43 | val zero = AveragedValue(0L, 0.0) 44 | 45 | override def isNonZero(av: AveragedValue) = (av.count != 0L) 46 | 47 | override def negate(av: AveragedValue) = AveragedValue(-av.count, av.value) 48 | 49 | def plus(cntAve1: AveragedValue, cntAve2: AveragedValue): AveragedValue = { 50 | val (big, small) = if (cntAve1.count >= cntAve2.count) 51 | (cntAve1, cntAve2) 52 | else 53 | (cntAve2, cntAve1) 54 | val n = big.count 55 | val k = small.count 56 | val newCnt = n + k 57 | if (newCnt == n) { 58 | // Handle zero without allocation 59 | big 60 | } else if (newCnt == 0L) { 61 | zero 62 | } else { 63 | val an = big.value 64 | val ak = small.value 65 | val scaling = k.toDouble / newCnt 66 | // a_n + (a_k - a_n)*(k/(n+k)) is only stable if n is not approximately k 67 | val newAve = if (scaling < STABILITY_CONSTANT) (an + (ak - an) * scaling) else (n * an + k * ak) / newCnt 68 | new AveragedValue(newCnt, newAve) 69 | } 70 | } 71 | } 72 | 73 | object Averager extends MonoidAggregator[Double, AveragedValue, Double] { 74 | val monoid = AveragedGroup 75 | def prepare(value: Double) = AveragedValue(value) 76 | def present(average: AveragedValue) = average.value 77 | } 78 | -------------------------------------------------------------------------------- /algebird-test/src/main/scala/com/twitter/algebird/MonadLaws.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird 18 | 19 | import org.scalacheck.Arbitrary 20 | import org.scalacheck.Prop.forAll 21 | 22 | import Monad.{ pureOp, operators } 23 | 24 | /** 25 | * Basic Monad laws, useful for testing any monad. 26 | */ 27 | 28 | object MonadLaws { 29 | import BaseProperties._ 30 | 31 | def defaultEq[T] = { (t0: T, t1: T) => (t0 == t1) } 32 | 33 | def leftIdentity[M[_], T, U](eq: (M[U], M[U]) => Boolean = defaultEq[M[U]])(implicit monad: Monad[M], arb: Arbitrary[T], arbfn: Arbitrary[(T) => M[U]]) = 34 | forAll { (t: T, fn: T => M[U]) => eq(t.pure[M].flatMap(fn), fn(t)) } 35 | 36 | def rightIdentity[M[_], T](eq: (M[T], M[T]) => Boolean = defaultEq[M[T]])(implicit monad: Monad[M], arb: Arbitrary[M[T]]) = 37 | forAll { (mt: M[T]) => eq(mt.flatMap { _.pure[M] }, mt) } 38 | 39 | def associative[M[_], T, U, V](eq: (M[V], M[V]) => Boolean)(implicit monad: Monad[M], arb: Arbitrary[M[T]], fn1: Arbitrary[(T) => M[U]], 40 | fn2: Arbitrary[U => M[V]]) = forAll { (mt: M[T], f1: T => M[U], f2: U => M[V]) => 41 | eq(mt.flatMap(f1).flatMap(f2), mt.flatMap { t => f1(t).flatMap(f2) }) 42 | } 43 | 44 | // Just generate a map and use that as a function: 45 | implicit def fnArb[M[_], T, U](implicit map: Arbitrary[Map[T, M[U]]], 46 | arbu: Arbitrary[M[U]]): Arbitrary[T => M[U]] = Arbitrary { 47 | for ( 48 | m <- implicitly[Arbitrary[Map[T, M[U]]]].arbitrary; 49 | defu <- implicitly[Arbitrary[M[U]]].arbitrary 50 | ) yield ({ (t: T) => m.getOrElse(t, defu) }) 51 | } 52 | 53 | def monadLaws[M[_], T, U, R](eq: (M[R], M[R]) => Boolean = defaultEq[M[R]])(implicit monad: Monad[M], arb: Arbitrary[M[T]], fn1: Arbitrary[(T) => M[U]], 54 | arbr: Arbitrary[M[R]], fn2: Arbitrary[U => M[R]], arbu: Arbitrary[U]) = 55 | associative[M, T, U, R](eq) && rightIdentity[M, R](eq) && leftIdentity[M, U, R](eq) 56 | 57 | implicit def indexedSeqA[T](implicit arbl: Arbitrary[List[T]]): Arbitrary[IndexedSeq[T]] = 58 | Arbitrary { arbl.arbitrary.map { _.toIndexedSeq } } 59 | 60 | implicit def vectorA[T](implicit arbl: Arbitrary[List[T]]): Arbitrary[Vector[T]] = 61 | Arbitrary { arbl.arbitrary.map { l => Vector(l: _*) } } 62 | 63 | implicit def seqA[T](implicit arbl: Arbitrary[List[T]]): Arbitrary[Seq[T]] = 64 | Arbitrary { arbl.arbitrary.map { l => Seq(l: _*) } } 65 | 66 | implicit def someA[T](implicit arbl: Arbitrary[T]): Arbitrary[Some[T]] = 67 | Arbitrary { arbl.arbitrary.map { l => Some(l) } } 68 | } 69 | -------------------------------------------------------------------------------- /algebird-core/src/main/scala/com/twitter/algebird/DecayedValue.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird 18 | 19 | // Represents a decayed value that is decayed of the form: 20 | // \sum_i e^{-(t_i - t)} v_i 21 | // 2^{-(t/th)} = exp(ln(2)(-t/th)) = exp(-t * (ln(2)/th)) 22 | // So time is measured in units of (half-life/ln(2)), so. 23 | // t in seconds, 1 day half life means: t => t * ln(2)/(86400.0) 24 | 25 | object DecayedValue extends java.io.Serializable { 26 | def build[V <% Double](value: V, time: Double, halfLife: Double) = { 27 | DecayedValue(value, time * scala.math.log(2.0) / halfLife) 28 | } 29 | val zero = DecayedValue(0.0, Double.NegativeInfinity) 30 | 31 | def scale(newv: DecayedValue, oldv: DecayedValue, eps: Double) = { 32 | val newValue = newv.value + 33 | scala.math.exp(oldv.scaledTime - newv.scaledTime) * oldv.value 34 | if (scala.math.abs(newValue) > eps) { 35 | DecayedValue(newValue, newv.scaledTime) 36 | } else { 37 | zero 38 | } 39 | } 40 | 41 | def monoidWithEpsilon(eps: Double): Monoid[DecayedValue] = new DecayedValueMonoid(eps) 42 | } 43 | 44 | case class DecayedValueMonoid(eps: Double) extends Monoid[DecayedValue] { 45 | override val zero = DecayedValue(0.0, Double.NegativeInfinity) 46 | override def plus(left: DecayedValue, right: DecayedValue) = 47 | if (left < right) { 48 | // left is older: 49 | DecayedValue.scale(right, left, eps) 50 | } else { 51 | // right is older 52 | DecayedValue.scale(left, right, eps) 53 | } 54 | 55 | // Returns value if timestamp is less than value's timestamp 56 | def valueAsOf(value: DecayedValue, halfLife: Double, timestamp: Double): Double = { 57 | plus(DecayedValue.build(0, timestamp, halfLife), value).value 58 | } 59 | } 60 | 61 | case class DecayedValue(value: Double, scaledTime: Double) extends Ordered[DecayedValue] { 62 | def compare(that: DecayedValue): Int = { 63 | scaledTime.compareTo(that.scaledTime) 64 | } 65 | 66 | // A DecayedValue can be translated to a moving average with the window size of its half-life. 67 | // It is EXACTLY a sample of the Laplace transform of the signal of values. 68 | // Therefore, we can get the moving average by normalizing a decayed value with halflife/ln(2), 69 | // which is the integral of exp(-t(ln(2))/halflife) from 0 to infinity. 70 | // 71 | // See: https://github.com/twitter/algebird/wiki/Using-DecayedValue-as-moving-average 72 | def average(halfLife: Double) = { 73 | val normalization = halfLife / math.log(2) 74 | value / normalization 75 | } 76 | } 77 | -------------------------------------------------------------------------------- /algebird-core/src/main/scala/com/twitter/algebird/DecayedVector.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird 18 | 19 | /** 20 | * Represents a container class together with time. 21 | * Its monoid consists of exponentially scaling the older value and summing with 22 | * the newer one. 23 | */ 24 | object DecayedVector { 25 | def buildWithHalflife[C[_]](vector: C[Double], time: Double, halfLife: Double) = { 26 | DecayedVector(vector, time * scala.math.log(2.0) / halfLife) 27 | } 28 | 29 | def monoidWithEpsilon[C[_]](eps: Double)(implicit vs: VectorSpace[Double, C], metric: Metric[C[Double]]) = new Monoid[DecayedVector[C]] { 30 | override val zero = DecayedVector(vs.group.zero, Double.NegativeInfinity) 31 | override def plus(left: DecayedVector[C], right: DecayedVector[C]) = { 32 | if (left.scaledTime <= right.scaledTime) { 33 | scaledPlus(right, left, eps) 34 | } else { 35 | scaledPlus(left, right, eps) 36 | } 37 | } 38 | } 39 | 40 | def forMap[K](m: Map[K, Double], scaledTime: Double) = DecayedVector[({ type x[a] = Map[K, a] })#x](m, scaledTime) 41 | def forMapWithHalflife[K](m: Map[K, Double], time: Double, halfLife: Double) = 42 | forMap(m, time * scala.math.log(2.0) / halfLife) 43 | 44 | def mapMonoidWithEpsilon[K](eps: Double)(implicit vs: VectorSpace[Double, ({ type x[a] = Map[K, a] })#x], metric: Metric[Map[K, Double]]) = 45 | monoidWithEpsilon[({ type x[a] = Map[K, a] })#x](eps) 46 | 47 | // This is the default monoid that never thresholds. 48 | // If you want to set a specific accuracy you need to implicitly override this 49 | implicit def monoid[F, C[_]](implicit vs: VectorSpace[F, C], metric: Metric[C[F]], ord: Ordering[F]) = monoidWithEpsilon(-1.0) 50 | implicit def mapMonoid[K](implicit vs: VectorSpace[Double, ({ type x[a] = Map[K, a] })#x], metric: Metric[Map[K, Double]]) = 51 | mapMonoidWithEpsilon(-1.0) 52 | 53 | def scaledPlus[C[_]](newVal: DecayedVector[C], oldVal: DecayedVector[C], eps: Double)(implicit vs: VectorSpace[Double, C], metric: Metric[C[Double]]): DecayedVector[C] = { 54 | implicit val mon: Monoid[C[Double]] = vs.group 55 | val expFactor = scala.math.exp(oldVal.scaledTime - newVal.scaledTime) 56 | val newVector = Monoid.plus(newVal.vector, vs.scale(expFactor, oldVal.vector)) 57 | if (eps < 0.0 || Metric.norm(newVector) > eps) { 58 | DecayedVector(newVector, newVal.scaledTime) 59 | } else { 60 | DecayedVector(mon.zero, Double.NegativeInfinity) 61 | } 62 | } 63 | } 64 | 65 | case class DecayedVector[C[_]](vector: C[Double], scaledTime: Double) 66 | -------------------------------------------------------------------------------- /algebird-core/src/main/scala/com/twitter/algebird/statistics/IterCallStatistics.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2014 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | package com.twitter.algebird.statistics 17 | 18 | /** 19 | * used to keep track of stats and time spent processing iterators passed to the methods 20 | * @author Julien Le Dem 21 | */ 22 | private class IterCallStatistics(threadSafe: Boolean) { 23 | 24 | /** 25 | * internal collection of a distribution of values on a log scale 26 | */ 27 | private class Statistics(threadSafe: Boolean) { 28 | import scala.math.min 29 | import java.lang.Long.numberOfLeadingZeros 30 | val maxBucket = 10 31 | val distribution = IndexedSeq.fill(maxBucket + 1) { Counter(threadSafe) } 32 | val total = Counter(threadSafe) 33 | 34 | def put(v: Long) { 35 | total.add(v) 36 | // log2(v + 1) for v up to 2^maxBucket 37 | val bucket = min(64 - numberOfLeadingZeros(v), maxBucket) 38 | distribution(bucket).increment 39 | } 40 | 41 | def count = distribution.foldLeft(0L) { _ + _.get } // sum 42 | 43 | def pow2(i: Int): Int = 1 << i 44 | 45 | override def toString = 46 | distribution.zipWithIndex.map { 47 | case (v, i) => 48 | (if (i == maxBucket) ">" else "<" + pow2(i)) + ": " + v 49 | }.mkString(", ") + ", avg=" + total.toDouble / count + " count=" + count 50 | 51 | } 52 | 53 | private[this] final val countStats = new Statistics(threadSafe) 54 | private[this] final val totalCallTime = Counter(threadSafe) 55 | 56 | /** used to count how many values are pulled from the Iterator without iterating twice */ 57 | private class CountingIterator[T](val i: Iterator[T]) extends Iterator[T] { 58 | private[this] final var nextCount: Long = 0 59 | override def hasNext = i.hasNext 60 | override def next = { 61 | val n = i.next 62 | nextCount += 1 63 | n 64 | } 65 | def getNextCount = nextCount 66 | } 67 | 68 | /** measures the time spent calling f on iter and the size of iter */ 69 | def measure[T, O](iter: TraversableOnce[T])(f: (TraversableOnce[T]) => O): O = { 70 | val ci = new CountingIterator(iter.toIterator) 71 | val t0 = System.currentTimeMillis() 72 | val r = f(ci) 73 | val t1 = System.currentTimeMillis() 74 | countStats.put(ci.getNextCount) 75 | totalCallTime.add(t1 - t0) 76 | r 77 | } 78 | 79 | def getCallCount = countStats.count 80 | def getTotalCallTime = totalCallTime.get 81 | 82 | override def toString = 83 | countStats.toString + ", " + 84 | "total time: " + totalCallTime + "ms, " + 85 | "avg time: " + (totalCallTime.toDouble / countStats.count) 86 | } -------------------------------------------------------------------------------- /algebird-util/src/test/scala/com/twitter/algebird/util/summer/AsyncListSumProperties.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2013 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird.util.summer 18 | 19 | import org.scalatest.prop.PropertyChecks 20 | import org.scalatest.{ Matchers, PropSpec } 21 | 22 | class AsyncListSumProperties extends PropSpec with PropertyChecks with Matchers { 23 | 24 | import com.twitter.algebird.util.summer.AsyncSummerLaws._ 25 | 26 | property("NonCompactingList Summing with and without the summer should match") { 27 | forAll { (inputs: List[List[(Int, Long)]], 28 | flushFrequency: FlushFrequency, 29 | bufferSize: BufferSize, 30 | memoryFlushPercent: MemoryFlushPercent) => 31 | val timeOutCounter = Counter("timeOut") 32 | val sizeCounter = Counter("size") 33 | val memoryCounter = Counter("memory") 34 | val insertOp = Counter("insertOp") 35 | val insertFails = Counter("insertFails") 36 | val tuplesIn = Counter("tuplesIn") 37 | val tuplesOut = Counter("tuplesOut") 38 | val summer = new AsyncListSum[Int, Long](bufferSize, 39 | flushFrequency, 40 | memoryFlushPercent, 41 | memoryCounter, 42 | timeOutCounter, 43 | insertOp, 44 | insertFails, 45 | sizeCounter, 46 | tuplesIn, 47 | tuplesOut, 48 | workPool, 49 | Compact(false), 50 | CompactionSize(0)) 51 | assert(summingWithAndWithoutSummerShouldMatch(summer, inputs)) 52 | } 53 | } 54 | 55 | property("CompactingList Summing with and without the summer should match") { 56 | forAll { (inputs: List[List[(Int, Long)]], 57 | flushFrequency: FlushFrequency, 58 | bufferSize: BufferSize, 59 | memoryFlushPercent: MemoryFlushPercent, 60 | compactionSize: CompactionSize) => 61 | val timeOutCounter = Counter("timeOut") 62 | val sizeCounter = Counter("size") 63 | val memoryCounter = Counter("memory") 64 | val insertOp = Counter("insertOp") 65 | val insertFails = Counter("insertFails") 66 | val tuplesIn = Counter("tuplesIn") 67 | val tuplesOut = Counter("tuplesOut") 68 | val summer = new AsyncListSum[Int, Long](bufferSize, 69 | flushFrequency, 70 | memoryFlushPercent, 71 | memoryCounter, 72 | timeOutCounter, 73 | insertOp, 74 | insertFails, 75 | sizeCounter, 76 | tuplesIn, 77 | tuplesOut, 78 | workPool, 79 | Compact(true), 80 | compactionSize) 81 | assert(summingWithAndWithoutSummerShouldMatch(summer, inputs)) 82 | } 83 | } 84 | } 85 | -------------------------------------------------------------------------------- /algebird-test/src/test/scala/com/twitter/algebird/RightFolded2Test.scala: -------------------------------------------------------------------------------- 1 | package com.twitter.algebird 2 | 3 | import org.scalatest.{ PropSpec, Matchers } 4 | import org.scalatest.prop.PropertyChecks 5 | import org.scalacheck.{ Gen, Arbitrary } 6 | 7 | import scala.annotation.tailrec 8 | 9 | class RightFolded2Test extends PropSpec with PropertyChecks with Matchers { 10 | import BaseProperties._ 11 | 12 | def monFold(i: Int, l: Long) = l + i.toLong 13 | def mapFn(l: Long) = l / 2 14 | 15 | implicit val rightFoldedMonoid = RightFolded2.monoid[Int, Long, Long](mapFn)(monFold) 16 | 17 | def rightFolded2Value[In, Out, Acc](implicit arbout: Arbitrary[Out], mon: RightFolded2Monoid[In, Out, Acc]): Gen[RightFoldedValue2[In, Out, Acc]] = 18 | for (v <- arbout.arbitrary) yield mon.init(v) 19 | 20 | def rightFolded2ToFold[In, Out, Acc](implicit arbin: Arbitrary[In], mon: RightFolded2Monoid[In, Out, Acc]): Gen[RightFoldedToFold2[In]] = 21 | for (v <- arbin.arbitrary) yield mon.toFold(v) 22 | 23 | implicit def rightFolded2[In, Out, Acc](implicit arbin: Arbitrary[In], arbout: Arbitrary[Out], mon: RightFolded2Monoid[In, Out, Acc]): Arbitrary[RightFolded2[In, Out, Acc]] = 24 | Arbitrary { Gen.oneOf(rightFolded2Value[In, Out, Acc], rightFolded2ToFold[In, Out, Acc]) } 25 | 26 | property("RightFolded2 is a monoid") { 27 | monoidLaws[RightFolded2[Int, Long, Long]] 28 | } 29 | 30 | // Make a list of lists such that the all but the last element satisfies the predicate 31 | // and joining the lists returns the original list 32 | @tailrec 33 | private[this] def chunk[T](items: List[T], acc: List[List[T]] = Nil)(pred: T => Boolean): List[List[T]] = { 34 | val (headL, tailL) = items.span(pred) 35 | if (tailL.isEmpty) { 36 | if (!headL.isEmpty) (headL :: acc).reverse else acc.reverse 37 | } else { 38 | val newAcc = (headL :+ (tailL.head)) :: acc 39 | chunk(tailL.tail, newAcc)(pred) 40 | } 41 | } 42 | // The last element in this list must be a rightFoldedValue 43 | def fold[In, Out, Acc](l: List[RightFolded2[In, Out, Acc]])(foldfn: (In, Out) => Out): Option[Out] = { 44 | l.last match { 45 | case RightFoldedValue2(v, _, _) => { 46 | Some(l.dropRight(1) 47 | .flatMap { _.asInstanceOf[RightFoldedToFold2[In]].in } 48 | .foldRight(v)(foldfn)) 49 | } 50 | case _ => None 51 | } 52 | } 53 | 54 | def sum[In, Out, Acc: Group](l: List[RightFolded2[In, Out, Acc]])(foldfn: (In, Out) => Out)(mapfn: (Out) => Acc): Acc = { 55 | def notIsVal(rf: RightFolded2[In, Out, Acc]) = rf match { 56 | case RightFoldedValue2(_, _, _) => false 57 | case _ => true 58 | } 59 | val chunks = chunk(l)(notIsVal) 60 | 61 | val grp = implicitly[Group[Acc]] 62 | val vals = chunks.map { fold(_)(foldfn).map(mapfn).getOrElse(grp.zero) } 63 | grp.sum(vals) 64 | } 65 | 66 | def accOf[In, Out, Acc](rfv: RightFolded2[In, Out, Acc]): Option[Acc] = { 67 | rfv match { 68 | case RightFoldedValue2(_, acc, _) => Some(acc) 69 | case _ => None 70 | } 71 | } 72 | 73 | property("RightFolded2 sum works as expected") { 74 | forAll { (ls: List[RightFolded2[Int, Long, Long]]) => 75 | val accSum = accOf(rightFoldedMonoid.sum(ls)).getOrElse(0L) 76 | assert(sum(ls)(monFold)(mapFn) == accSum) 77 | } 78 | } 79 | } 80 | -------------------------------------------------------------------------------- /algebird-util/src/main/scala/com/twitter/algebird/util/summer/AsyncListMMapSum.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2012 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | package com.twitter.algebird.util.summer 17 | 18 | import com.twitter.algebird._ 19 | import com.twitter.util.{ Duration, Future, FuturePool } 20 | import java.util.concurrent.ConcurrentHashMap 21 | import java.util.concurrent.atomic.AtomicInteger 22 | import scala.collection.JavaConverters._ 23 | import scala.collection.mutable.{ Set => MSet, ListBuffer, Map => MMap } 24 | import scala.collection.breakOut 25 | 26 | /** 27 | * @author Ian O Connell 28 | * 29 | * This is a simple asyncronous summer, where a shared mutable map is used between all readers/writers. 30 | * When flushing it acquires the lock, drains the mutable map but does the compaction without holding the lock. 31 | */ 32 | 33 | class AsyncListMMapSum[Key, Value](bufferSize: BufferSize, 34 | override val flushFrequency: FlushFrequency, 35 | override val softMemoryFlush: MemoryFlushPercent, 36 | override val memoryIncr: Incrementor, 37 | override val timeoutIncr: Incrementor, 38 | tuplesOut: Incrementor, 39 | insertOp: Incrementor, 40 | sizeIncr: Incrementor, 41 | workPool: FuturePool)(implicit sg: Semigroup[Value]) 42 | extends AsyncSummer[(Key, Value), Map[Key, Value]] 43 | with WithFlushConditions[(Key, Value), Map[Key, Value]] { 44 | require(bufferSize.v > 0, "Use the Null summer for an empty async summer") 45 | 46 | private[this] final val queueMap = MMap[Key, ListBuffer[Value]]() 47 | private[this] final val mutex = new Object() 48 | @volatile private[this] var presentTuples = 0 49 | 50 | protected override val emptyResult = Map.empty[Key, Value] 51 | 52 | override def isFlushed: Boolean = mutex.synchronized { presentTuples == 0 } 53 | 54 | override def flush: Future[Map[Key, Value]] = 55 | workPool { 56 | val curData = mutex.synchronized { 57 | presentTuples = 0 58 | val l = queueMap.toList 59 | queueMap.clear 60 | l 61 | } 62 | val result: Map[Key, Value] = curData.flatMap { 63 | case (k, listV) => 64 | sg.sumOption(listV).map(v => (k, v)) 65 | }(breakOut) 66 | 67 | tuplesOut.incrBy(result.size) 68 | result 69 | } 70 | 71 | def addAll(vals: TraversableOnce[(Key, Value)]): Future[Map[Key, Value]] = { 72 | insertOp.incr 73 | var newlyAddedTuples = 0 74 | 75 | mutex.synchronized { 76 | vals.foreach { 77 | case (k, v) => 78 | val existingV = queueMap.getOrElseUpdate(k, ListBuffer[Value]()) 79 | existingV += v 80 | newlyAddedTuples += 1 81 | } 82 | presentTuples += newlyAddedTuples 83 | } 84 | 85 | if (presentTuples >= bufferSize.v) { 86 | sizeIncr.incr 87 | flush 88 | } else 89 | Future.value(emptyResult) 90 | } 91 | } -------------------------------------------------------------------------------- /algebird-caliper/README.md: -------------------------------------------------------------------------------- 1 | [Caliper](https://code.google.com/p/caliper/)-based Benchmarks for Algebird data structures. 2 | 3 | # Usage 4 | 5 | Run the following commands from the top-level Algebird directory: 6 | 7 | $ ./sbt # <<< enter sbt REPL 8 | > project algebird-caliper 9 | 10 | Now you can run the following commands from within the sbt REPL: 11 | 12 | # List available benchmarks 13 | > show cappi::benchmarks 14 | 15 | # Run a particular benchmark 16 | > cappi::benchmarkOnly com.twitter.algebird.caliper.HLLBenchmark 17 | 18 | # Debug a particular benchmark (shows e.g. number of repetitions that will be run) 19 | > cappi::benchmarkOnly --debug com.twitter.algebird.caliper.HLLBenchmark 20 | 21 | # Run all benchmarks (apparently this is broken, see https://github.com/softprops/cappi/issues/1) 22 | > cappi::benchmarks 23 | 24 | You can find further details in the [cappi](https://github.com/softprops/cappi) documentation, which is the sbt plugin 25 | we use to run the caliper benchmarks. 26 | 27 | Example output for [CMSBenchmark](src/test/scala/com/twitter/algebird/caliper/CMSBenchmark.scala): 28 | 29 | > cappi::benchmarkOnly com.twitter.algebird.caliper.CMSBenchmark 30 | [info] Running com.google.caliper.Runner com.twitter.algebird.caliper.CMSBenchmark 31 | [info] 0% Scenario{vm=java, trial=0, benchmark=PlusOfFirstHundredIntegersWithLongCms, delta=0.0000001, eps=0.1, heavyHittersPct=0.2, maxBits=2048, operations=100} 292576.31 ns; σ=1271.12 ns @ 3 trials 32 | [info] 17% Scenario{vm=java, trial=0, benchmark=PlusOfFirstHundredIntegersWithBigIntCms, delta=0.0000001, eps=0.1, heavyHittersPct=0.2, maxBits=2048, operations=100} 830195.29 ns; σ=7349.10 ns @ 3 trials 33 | [info] 33% Scenario{vm=java, trial=0, benchmark=PlusOfRandom2048BitNumbersWithBigIntCms, delta=0.0000001, eps=0.1, heavyHittersPct=0.2, maxBits=2048, operations=100} 3362751.81 ns; σ=104683.16 ns @ 10 trials 34 | [info] 50% Scenario{vm=java, trial=0, benchmark=PlusOfFirstHundredIntegersWithLongCms, delta=0.0000001, eps=0.005, heavyHittersPct=0.2, maxBits=2048, operations=100} 384133.61 ns; σ=41211.47 ns @ 10 trials 35 | [info] 67% Scenario{vm=java, trial=0, benchmark=PlusOfFirstHundredIntegersWithBigIntCms, delta=0.0000001, eps=0.005, heavyHittersPct=0.2, maxBits=2048, operations=100} 1018308.55 ns; σ=43285.12 ns @ 10 trials 36 | [info] 83% Scenario{vm=java, trial=0, benchmark=PlusOfRandom2048BitNumbersWithBigIntCms, delta=0.0000001, eps=0.005, heavyHittersPct=0.2, maxBits=2048, operations=100} 3610991.09 ns; σ=195033.95 ns @ 10 trials 37 | [info] 38 | [info] benchmark eps us linear runtime 39 | [info] PlusOfFirstHundredIntegersWithLongCms 0.1 293 == 40 | [info] PlusOfFirstHundredIntegersWithLongCms 0.005 384 === 41 | [info] PlusOfFirstHundredIntegersWithBigIntCms 0.1 830 ====== 42 | [info] PlusOfFirstHundredIntegersWithBigIntCms 0.005 1018 ======== 43 | [info] PlusOfRandom2048BitNumbersWithBigIntCms 0.1 3363 =========================== 44 | [info] PlusOfRandom2048BitNumbersWithBigIntCms 0.005 3611 ============================== 45 | [info] 46 | [info] vm: java 47 | [info] trial: 0 48 | [info] delta: 0.0000001 49 | [info] heavyHittersPct: 0.2 50 | [info] maxBits: 2048 51 | [info] operations: 100 52 | [success] Total time: 74 s, completed Oct 12, 2014 2:36:04 PM 53 | -------------------------------------------------------------------------------- /algebird-core/src/main/java/com/twitter/algebird/javaapi/Monoids.java: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2014 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | package com.twitter.algebird.javaapi; 17 | 18 | import com.twitter.algebird.Monoid; 19 | import com.twitter.algebird.Monoid$; 20 | import com.twitter.algebird.Semigroup; 21 | 22 | /** 23 | * 24 | * help use the Monoids from Java 25 | * 26 | * @author Julien Le Dem 27 | * 28 | */ 29 | public class Monoids { 30 | 31 | private Monoids() {} 32 | 33 | private static final Monoid$ MODULE = Monoid$.MODULE$; 34 | 35 | private static final MethodRegistry registry = new MethodRegistry(MODULE, Monoid.class); 36 | 37 | /** 38 | * @param c the type of the elements 39 | * @return a monoid of c 40 | */ 41 | public static Monoid monoid(Class c) { 42 | @SuppressWarnings("unchecked") 43 | Monoid m = (Monoid)registry.resolveAndInvoke(c); 44 | return m; 45 | } 46 | 47 | public static Monoid boolMonoid() { return MODULE.jboolMonoid(); } 48 | public static Monoid intMonoid() { return MODULE.jintMonoid(); } 49 | public static Monoid shortMonoid() { return MODULE.jshortMonoid(); } 50 | public static Monoid bigIntMonoid() { return MODULE.bigIntMonoid(); } 51 | public static Monoid longMonoid() { return MODULE.jlongMonoid(); } 52 | public static Monoid floatMonoid() { return MODULE.jfloatMonoid(); } 53 | public static Monoid doubleMonoid() { return MODULE.jdoubleMonoid(); } 54 | public static Monoid stringMonoid() { return MODULE.stringMonoid(); } 55 | public static Monoid> optionMonoid(Semigroup componentSemigroup) { return MODULE.optionMonoid(componentSemigroup); } 56 | public static Monoid> listMonoid() { return MODULE.listMonoid(); } 57 | public static Monoid> seqMonoid() { return MODULE.seqMonoid(); } 58 | public static Monoid> indexedSeqMonoid(Monoid componentMonoid) { return MODULE.indexedSeqMonoid(componentMonoid); } 59 | public static Monoid> jlistMonoid() { return MODULE.jlistMonoid(); } 60 | public static Monoid> setMonoid() { return MODULE.setMonoid(); } 61 | public static Monoid> mapMonoid(Semigroup componentSemigroup) { return MODULE.mapMonoid(componentSemigroup); } 62 | public static Monoid> scMapMonoid(Semigroup componentSemigroup) { return MODULE.scMapMonoid(componentSemigroup); } 63 | public static Monoid> jmapMonoid(Semigroup componentSemigroup) { return MODULE.jmapMonoid(componentSemigroup); } 64 | public static Monoid> function1Monoid() { return MODULE.function1Monoid(); } 65 | } 66 | -------------------------------------------------------------------------------- /algebird-test/src/test/scala/com/twitter/algebird/FoldTest.scala: -------------------------------------------------------------------------------- 1 | package com.twitter.algebird 2 | 3 | import org.scalatest._ 4 | 5 | class FoldTest extends WordSpec { 6 | 7 | sealed trait Case[I, O] { 8 | def expected: O 9 | def runCase(fold: Fold[I, O]): O 10 | } 11 | case class Zero[I, O](expected: O) extends Case[I, O] { 12 | override def runCase(fold: Fold[I, O]) = fold.overEmpty 13 | } 14 | case class One[I, O](in: I, expected: O) extends Case[I, O] { 15 | override def runCase(fold: Fold[I, O]) = fold.overSingleton(in) 16 | } 17 | case class Many[I, O](in: Traversable[I], expected: O) extends Case[I, O] { 18 | override def runCase(fold: Fold[I, O]) = fold.overTraversable(in) 19 | } 20 | 21 | def run[I, O](fold: Fold[I, O], cases: Case[I, O]*): Unit = 22 | cases.foreach { c => assert(c.runCase(fold) === c.expected) } 23 | 24 | "Fold" should { 25 | 26 | "foldLeft" in { 27 | run[String, String]( 28 | Fold.foldLeft("") { (a, b) => a ++ b }, 29 | Zero(""), 30 | One("1", "1"), 31 | Many(Seq("1", "2", "3"), "123")) 32 | } 33 | 34 | "seq" in { 35 | run[Int, Seq[Int]]( 36 | Fold.seq, 37 | Zero(Seq.empty), 38 | One(1, Seq(1)), 39 | Many(Seq(1, 2, 3), Seq(1, 2, 3)), 40 | Many(Seq(2, 1, 3), Seq(2, 1, 3))) 41 | } 42 | 43 | "const" in { 44 | run[Int, String]( 45 | Fold.const("42"), 46 | Zero("42"), 47 | One(1, "42"), 48 | Many(Seq(1, 2, 3), "42")) 49 | } 50 | 51 | "first" in { 52 | run[String, Option[String]]( 53 | Fold.first, 54 | Zero(None), 55 | One("1", Some("1")), 56 | Many(Seq("1", "2", "3"), Some("1"))) 57 | } 58 | 59 | "last" in { 60 | run[String, Option[String]]( 61 | Fold.last, 62 | Zero(None), 63 | One("1", Some("1")), 64 | Many(Seq("1", "2", "3"), Some("3"))) 65 | } 66 | 67 | "max" in { 68 | run[Int, Option[Int]]( 69 | Fold.max, 70 | Zero(None), 71 | One(1, Some(1)), 72 | Many(Seq(1, 2, 3), Some(3)), 73 | Many(Seq(1, 3, 2), Some(3))) 74 | } 75 | 76 | "min" in { 77 | run[Int, Option[Int]]( 78 | Fold.min, 79 | Zero(None), 80 | One(1, Some(1)), 81 | Many(Seq(1, 2, 3), Some(1)), 82 | Many(Seq(2, 1, 3), Some(1))) 83 | } 84 | 85 | "sum" in { 86 | run[Int, Int]( 87 | Fold.sum, 88 | Zero(0), 89 | One(1, 1), 90 | Many(Seq(1, 2, 3), 6), 91 | Many(Seq(2, 1, 3), 6)) 92 | } 93 | 94 | "size" in { 95 | run[String, Long]( 96 | Fold.size, 97 | Zero(0), 98 | One("1", 1), 99 | Many(Seq("1", "2", "3"), 3)) 100 | } 101 | 102 | "average" in { 103 | run[Int, Double]( 104 | Fold.sum[Int].joinWith(Fold.size) { (s, c) => s.toDouble / c }, 105 | One(1, 1.0), 106 | Many(Seq(1, 2, 3), 2.0), 107 | Many(Seq(2, 1, 3), 2.0)) 108 | } 109 | 110 | "sequence" in { 111 | run[Int, Seq[Long]]( 112 | Fold.sequence(Seq(Fold.count { _ < 0 }, Fold.count { _ >= 0 })), 113 | Zero(Seq(0, 0)), 114 | One(1, Seq(0, 1)), 115 | Many(Seq(-2, -1, 0, 1, 2), Seq(2, 3))) 116 | } 117 | 118 | } 119 | 120 | } 121 | -------------------------------------------------------------------------------- /algebird-core/src/main/scala/com/twitter/algebird/Predecessible.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2014 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | package com.twitter.algebird 17 | 18 | /** 19 | * This is a typeclass to represent things which are countable down. Note that it is important 20 | * that a value prev(t) is always less than t. Note 21 | * that prev returns Option because this class comes with the notion that some items may reach a minimum 22 | * key, which is None. 23 | */ 24 | trait Predecessible[T] extends java.io.Serializable { 25 | def prev(old: T): Option[T] 26 | def prev(old: Option[T]): Option[T] = old.flatMap(prev) 27 | def iteratePrev(old: T): Iterable[T] = { 28 | val self = this 29 | // TODO in scala 2.11, there is an AbstractIterable which should be used here 30 | // to reduce generated class size due to all the methods in Iterable. 31 | // https://github.com/twitter/algebird/issues/263 32 | new AbstractIterable[T] { 33 | def iterator = 34 | Iterator.iterate[Option[T]](Some(old)) { self.prev(_) } 35 | .takeWhile(_.isDefined) 36 | .collect { case Some(t) => t } 37 | } 38 | } 39 | /** 40 | * The law is: 41 | * prev(t) 42 | * .map { n => partialOrdering.lteq(n, t) && (!partialOrdering.equiv(t, n)) } 43 | * .getOrElse(true) 44 | * 45 | * Note Ordering extends PartialOrdering, so we are taking a weak constraint 46 | * that some items can be ordered, and namely, the sequence of items returned 47 | * by prev is strictly decreasing 48 | */ 49 | def partialOrdering: PartialOrdering[T] 50 | } 51 | 52 | object Predecessible extends java.io.Serializable { 53 | /** 54 | * This makes it easy to construct from a function when T has an ordering, which is common 55 | * Note, your function must respect the ordering 56 | */ 57 | def fromPrevOrd[T](prevFn: T => Option[T])(implicit ord: Ordering[T]): Predecessible[T] = new Predecessible[T] { 58 | def prev(t: T) = prevFn(t) 59 | def partialOrdering = ord 60 | } 61 | // enables: Predecessible.prev(2) == Some(1) 62 | def prev[T](t: T)(implicit p: Predecessible[T]): Option[T] = p.prev(t) 63 | def prev[T](t: Option[T])(implicit p: Predecessible[T]): Option[T] = p.prev(t) 64 | 65 | def iteratePrev[T](first: T)(implicit p: Predecessible[T]): Iterable[T] = 66 | p.iteratePrev(first) 67 | 68 | implicit def integralPrev[N: Integral]: Predecessible[N] = new IntegralPredecessible[N] 69 | } 70 | 71 | class IntegralPredecessible[T: Integral] extends Predecessible[T] { 72 | private[this] val integral = implicitly[Integral[T]] 73 | def prev(old: T) = { 74 | val newV = integral.minus(old, integral.one) 75 | if (integral.compare(newV, old) >= 0) { 76 | // We wrapped around 77 | None 78 | } else { 79 | Some(newV) 80 | } 81 | } 82 | 83 | def partialOrdering: PartialOrdering[T] = integral 84 | } 85 | -------------------------------------------------------------------------------- /algebird-test/src/test/scala/com/twitter/algebird/MonadInstanceLaws.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2013 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird.monad 18 | 19 | import com.twitter.algebird.Monad 20 | import org.scalatest.{ PropSpec, Matchers } 21 | import org.scalatest.prop.PropertyChecks 22 | import org.scalacheck.{ Gen, Arbitrary } 23 | 24 | class MonadInstanceLaws extends PropSpec with PropertyChecks with Matchers { 25 | 26 | // Mutually recursive functions 27 | def ping(todo: Int, acc: Int): Trampoline[Int] = 28 | if (todo <= 0) Trampoline(acc) else Trampoline.call(pong(todo - 1, acc + 1)) 29 | 30 | def pong(todo: Int, acc: Int): Trampoline[Int] = 31 | if (todo <= 0) Trampoline(acc) else Trampoline.call(ping(todo - 1, acc + 1)) 32 | 33 | property("Trampoline should run without stackoverflow") { 34 | forAll { (b: Int) => 35 | val bsmall = b % 1000000 36 | assert(ping(bsmall, 0).get == (bsmall max 0)) 37 | } 38 | } 39 | 40 | property("get/swap") { 41 | forAll { (i: Int) => 42 | val fn = for { 43 | start <- StateWithError.getState[Int] 44 | oldState <- StateWithError.swapState[Int](start * 2) 45 | } yield oldState 46 | 47 | assert(fn(i) == Right((2 * i, i))) 48 | } 49 | } 50 | 51 | property("State behaves correctly") { 52 | forAll { (in: Int, head: Long, fns: List[(Int) => Either[String, (Int, Long)]]) => 53 | val mons = fns.map { StateWithError(_) } 54 | val init = StateWithError.const[Int, Long](head): StateWithError[Int, String, Long] 55 | val comp = mons.foldLeft(init) { (old, fn) => 56 | old.flatMap { x => fn } // just bind 57 | } 58 | assert(comp(in) == (fns.foldLeft(Right((in, head)): Either[String, (Int, Long)]) { (oldState, fn) => 59 | oldState.right.flatMap { case (s, v) => fn(s) } 60 | })) 61 | } 62 | } 63 | 64 | class MutableBox(var item: Int) { 65 | def inc(v: Int) = { item += v } 66 | } 67 | 68 | property("Reader behaves correctly") { 69 | forAll { (initialEnv: Int, fns: List[(Int) => Int]) => 70 | // Set up the mutable variable to feed to the readers: 71 | val m1 = new MutableBox(initialEnv) 72 | val m2 = new MutableBox(initialEnv) 73 | 74 | val readers = fns.map { fn => 75 | Reader { (m: MutableBox) => 76 | val toInc = fn(m.item) 77 | m.inc(toInc) 78 | } 79 | } 80 | // Now apply them all: 81 | val bigReader = readers.foldLeft(Reader.const(()): Reader[MutableBox, Unit]) { (oldr, thisR) => 82 | oldr.flatMap { x => thisR } // just sequence them 83 | } 84 | // apply: 85 | val result = bigReader(m1) 86 | 87 | // This should be the same as this loop: 88 | fns.foreach { fn => 89 | m2.inc(fn(m2.item)) 90 | } 91 | assert(m1.item == m2.item) 92 | } 93 | } 94 | 95 | } 96 | -------------------------------------------------------------------------------- /algebird-test/src/test/scala/com/twitter/algebird/QTreeTest.scala: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2013 Twitter, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | package com.twitter.algebird 18 | 19 | import org.scalatest._ 20 | 21 | import org.scalatest.{ PropSpec, Matchers } 22 | import org.scalatest.prop.PropertyChecks 23 | import org.scalacheck.Arbitrary 24 | import org.scalatest.{ PropSpec, Matchers } 25 | import org.scalatest.prop.PropertyChecks 26 | import org.scalacheck.Arbitrary.arbitrary 27 | import org.scalatest.{ PropSpec, Matchers } 28 | import org.scalatest.prop.PropertyChecks 29 | import org.scalacheck.Properties 30 | import org.scalatest.{ PropSpec, Matchers } 31 | import org.scalatest.prop.PropertyChecks 32 | import org.scalacheck.Gen.choose 33 | 34 | import java.util.Arrays 35 | 36 | class QTreeLaws extends PropSpec with PropertyChecks with Matchers { 37 | import BaseProperties._ 38 | 39 | implicit val qtSemigroup = new QTreeSemigroup[Long](6) 40 | implicit val qtGen = Arbitrary { 41 | for ( 42 | v <- choose(0L, 10000L) 43 | ) yield (QTree(v)) 44 | } 45 | 46 | property("QTree is associative") { 47 | isAssociative[QTree[Long]] 48 | } 49 | 50 | } 51 | 52 | class QTreeTest extends WordSpec with Matchers { 53 | def randomList(n: Long) = { 54 | (1L to n).map{ i => math.random } 55 | } 56 | 57 | def buildQTree(k: Int, list: Seq[Double]) = { 58 | val qtSemigroup = new QTreeSemigroup[Double](k) 59 | list.map{ QTree(_) }.reduce{ qtSemigroup.plus(_, _) } 60 | } 61 | 62 | def trueQuantile(list: Seq[Double], q: Double) = { 63 | val rank = math.floor(q * list.size).toInt 64 | val sorted = list.toList.sorted 65 | sorted(rank) 66 | } 67 | 68 | def trueRangeSum(list: Seq[Double], from: Double, to: Double) = { 69 | list.filter{ _ >= from }.filter{ _ < to }.sum 70 | } 71 | 72 | for (k <- (1 to 6)) 73 | ("QTree with sizeHint 2^" + k) should { 74 | "always contain the true quantile within its bounds" in { 75 | val list = randomList(10000) 76 | val qt = buildQTree(k, list) 77 | val quantile = math.random 78 | val (lower, upper) = qt.quantileBounds(quantile) 79 | val truth = trueQuantile(list, quantile) 80 | assert(truth >= lower) 81 | assert(truth <= upper) 82 | } 83 | "always contain the true range sum within its bounds" in { 84 | val list = randomList(10000) 85 | val qt = buildQTree(k, list) 86 | val from = math.random 87 | val to = math.random 88 | val (lower, upper) = qt.rangeSumBounds(from, to) 89 | val truth = trueRangeSum(list, from, to) 90 | assert(truth >= lower) 91 | assert(truth <= upper) 92 | } 93 | "have size bounded by 2^(k+2)" in { 94 | val list = randomList(100000) 95 | val qt = buildQTree(k, list) 96 | assert(qt.size <= (1 << (k + 2))) 97 | } 98 | } 99 | } 100 | --------------------------------------------------------------------------------