├── 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 |
--------------------------------------------------------------------------------