Skip to content

Commit

Permalink
Remove build deprecation warnings (#1107)
Browse files Browse the repository at this point in the history
  • Loading branch information
regadas authored Mar 5, 2023
1 parent 53d7d1c commit 65a95ce
Show file tree
Hide file tree
Showing 51 changed files with 1,818 additions and 1,939 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@ package com.twitter.algebird.bijection
import com.twitter.algebird.{Group, InvariantSemigroup, Monoid, Ring, Semigroup}
import com.twitter.bijection.{AbstractBijection, Bijection, ImplicitBijection, Reverse}

import scala.collection.compat._

/**
* Bijections on Algebird's abstract algebra datatypes.
*
Expand Down Expand Up @@ -53,7 +55,7 @@ class BijectedRing[T, U](implicit val ring: Ring[T], bij: ImplicitBijection[T, U
override def times(l: U, r: U): U =
bijection.invert(ring.times(bijection(l), bijection(r)))
override def product(iter: TraversableOnce[U]): U =
bijection.invert(ring.product(iter.map(bijection.toFunction)))
bijection.invert(ring.product(iter.iterator.map(bijection.toFunction)))
}

trait AlgebirdBijections {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ private[algebird] trait CompatFold {
Fold.foldMutable[Builder[I, C[I]], I, C[I]](
{ case (b, i) => b += i },
_ => cbf.newBuilder,
_.result
_.result()
)
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,9 +33,9 @@ class SentinelCache[K, V](implicit sgv: Semigroup[V]) {

def size: Int = map.get.map(_.size).getOrElse(0)

def clear(): Unit = map.get.foreach(_.clear)
def clear(): Unit = map.get.foreach(_.clear())

def stopGrowing(): Unit = map.clear
def stopGrowing(): Unit = map.clear()

def put(in: Map[K, V]): Unit =
if (map.get.isDefined) {
Expand Down Expand Up @@ -87,9 +87,9 @@ class AdaptiveCache[K, V: Semigroup](maxCapacity: Int, growthMargin: Double = 3.
summingCache = new SummingWithHitsCache(currentCapacity)

if (currentCapacity == maxCapacity)
sentinelCache.stopGrowing
sentinelCache.stopGrowing()
else
sentinelCache.clear
sentinelCache.clear()
}
ret
}
Expand All @@ -101,7 +101,7 @@ class AdaptiveCache[K, V: Semigroup](maxCapacity: Int, growthMargin: Double = 3.

override def flush: Option[Map[K, V]] = {
val ret = summingCache.flush
sentinelCache.clear
sentinelCache.clear()
ret
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -145,7 +145,7 @@ object AdaptiveVector {
def iteq: Boolean =
(lit.hasNext, rit.hasNext) match {
case (true, true) =>
val (lnext, rnext) = (lit.next, rit.next)
val (lnext, rnext) = (lit.next(), rit.next())
if (lnext._1 == rnext._1 && Equiv[V].equiv(lnext._2, rnext._2))
iteq
else
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -114,7 +114,7 @@ object Aggregator extends java.io.Serializable {
if (inputs.iterator.isEmpty) None
else {
val itr = inputs.iterator
val t = prepare(itr.next)
val t = prepare(itr.next())
Some(itr.foldLeft(t)(appnd))
}
}
Expand Down Expand Up @@ -155,7 +155,7 @@ object Aggregator extends java.io.Serializable {
override def apply(inputs: TraversableOnce[F]): P = present(agg(inputs))

override def applyOption(inputs: TraversableOnce[F]): Option[P] =
if (inputs.isEmpty) None else Some(apply(inputs))
if (inputs.iterator.isEmpty) None else Some(apply(inputs))

override def append(l: T, r: F): T = appnd(l, r)

Expand All @@ -165,7 +165,7 @@ object Aggregator extends java.io.Serializable {
override def appendAll(items: TraversableOnce[F]): T = agg(items)

private def agg(inputs: TraversableOnce[F]): T =
inputs.foldLeft(m.zero)(append)
inputs.iterator.foldLeft(m.zero)(append)
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ trait Applicative[M[_]] extends Functor[M] {
case _ =>
val mb =
ms.foldLeft(apply(Seq.newBuilder[T]))((mb, mt) => joinWith(mb, mt)((b, t) => b += t))
map(mb)(_.result)
map(mb)(_.result())
}
def joinWith[T, U, V](mt: M[T], mu: M[U])(fn: (T, U) => V): M[V] =
map(join(mt, mu)) { case (t, u) => fn(t, u) }
Expand Down Expand Up @@ -102,7 +102,7 @@ object Applicative {
)(implicit app: Applicative[M], cbf: Factory[T, R[T]]): M[R[T]] = {
val bldr = cbf.newBuilder
val mbldr = ms.iterator.foldLeft(app.apply(bldr))((mb, mt) => app.joinWith(mb, mt)(_ += _))
app.map(mbldr)(_.result)
app.map(mbldr)(_.result())
}

def joinWith[M[_], T, U, V](mt: M[T], mu: M[U])(fn: (T, U) => V)(implicit app: Applicative[M]): M[V] =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ import scala.collection.compat._
* As long as your count doesn't overflow a Long, the mean calculation won't overflow.
*
* @see
* [[MomentsGroup.getCombinedMean]] for implementation of [[+]]
* [[Moments.getCombinedMeanDouble]] for implementation of [[+]]
* @param count
* the number of aggregated items
* @param value
Expand Down Expand Up @@ -77,7 +77,7 @@ case class AveragedValue(count: Long, value: Double) {
* an instance representing the mean of this instance and `that`.
*/
def +(that: Double): AveragedValue =
AveragedValue(count + 1L, MomentsGroup.getCombinedMean(count, value, 1L, that))
AveragedValue(count + 1L, Moments.getCombinedMeanDouble(count.toDouble, value, 1L, that))

/**
* Returns a new instance that averages `that` into this instance.
Expand Down Expand Up @@ -140,7 +140,6 @@ object AveragedValue {
* `AveragedValue`
*/
object AveragedGroup extends Group[AveragedValue] with CommutativeGroup[AveragedValue] {
import MomentsGroup.getCombinedMean

override val zero: AveragedValue = AveragedValue(0L, 0.0)

Expand All @@ -160,7 +159,7 @@ object AveragedGroup extends Group[AveragedValue] with CommutativeGroup[Averaged
val it = iter.toIterator
while (it.hasNext) {
val av = it.next()
average = getCombinedMean(count, average, av.count, av.value)
average = Moments.getCombinedMeanDouble(count.toDouble, average, av.count.toDouble, av.value)
count += av.count
}
Some(AveragedValue(count, average))
Expand All @@ -172,7 +171,7 @@ object AveragedGroup extends Group[AveragedValue] with CommutativeGroup[Averaged
override def plus(l: AveragedValue, r: AveragedValue): AveragedValue = {
val n = l.count
val k = r.count
val newAve = getCombinedMean(n, l.value, k, r.value)
val newAve = Moments.getCombinedMeanDouble(n.toDouble, l.value, k.toDouble, r.value)
AveragedValue(n + k, newAve)
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,7 @@ sealed abstract class Batched[T] extends Serializable {
* This method will grow the tree to the left.
*/
def append(that: TraversableOnce[T]): Batched[T] =
that.foldLeft(this)((b, t) => b.combine(Batched(t)))
that.iterator.foldLeft(this)((b, t) => b.combine(Batched(t)))

/**
* Provide an iterator over the underlying tree structure.
Expand Down Expand Up @@ -104,7 +104,7 @@ object Batched {
if (ts.iterator.isEmpty) None
else {
val it = ts.iterator
val t0 = it.next
val t0 = it.next()
Some(Item(t0).append(it))
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ object RichCBitSet {
def fromBitSet(bs: BitSet): CBitSet = {
val nbs = new CBitSet
val it = bs.iterator
while (it.hasNext) { nbs.set(it.next) }
while (it.hasNext) { nbs.set(it.next()) }
nbs
}
implicit def cb2rcb(cb: CBitSet): RichCBitSet = new RichCBitSet(cb)
Expand All @@ -60,7 +60,7 @@ class RichCBitSet(val cb: CBitSet) extends AnyVal {
val a = LongBitSet.empty(width)
val iter = cb.intIterator
while (iter.hasNext) {
val i = iter.next
val i = iter.next()
a.set(i)
}
a.toBitSetNoCopy
Expand Down Expand Up @@ -235,7 +235,7 @@ case class BloomFilterMonoid[A](numHashes: Int, width: Int)(implicit hash: Hash1
case BFInstance(_, bitset, _) =>
// these Ints are boxed so, that's a minor bummer
val iter = bitset.iterator
while (iter.hasNext) { set(iter.next) }
while (iter.hasNext) { set(iter.next()) }
}
if (sets == 0) Some(zero)
else if (sets == numHashes && (oneItem != null)) Some(oneItem)
Expand Down Expand Up @@ -307,7 +307,7 @@ object BF {
new IntIterator {
val boxedIter: Iterator[Int] = bitset.iterator
override def hasNext: Boolean = boxedIter.hasNext
override def next: Int = boxedIter.next
override def next: Int = boxedIter.next()
}
case BFZero(_, _) =>
new IntIterator {
Expand Down Expand Up @@ -484,7 +484,7 @@ case class BFSparse[A](hashes: BFHash[A], bits: CBitSet, override val width: Int
var count = 0
while (it.hasNext) {
count += 1
it.next
it.next()
}
count
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,7 @@ abstract class ArrayBufferedOperation[I, O](size: Int) extends Buffered[I, O] {
if (buffer.isEmpty) None
else {
val res = operate(buffer.toSeq)
buffer.clear
buffer.clear()
Some(res)
}

Expand Down
Original file line number Diff line number Diff line change
@@ -1,5 +1,7 @@
package com.twitter.algebird

import scala.collection.compat._

object Correlation {
def apply(x: (Double, Double), weight: Double): Correlation =
Correlation(c2 = 0, m2x = 0, m2y = 0, m1x = x._1, m1y = x._2, weight)
Expand Down Expand Up @@ -113,9 +115,9 @@ object CorrelationMonoid extends Monoid[Correlation] {
override val zero: Correlation = Correlation(0, 0, 0, 0, 0, 0)

override def sumOption(cs: TraversableOnce[Correlation]): Option[Correlation] =
if (cs.isEmpty) None
if (cs.iterator.isEmpty) None
else {
val iter = cs.toIterator
val iter = cs.iterator
val item = iter.next()

var m0 = item.m0
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -185,9 +185,9 @@ class CMSSummation[K](params: CMSParams[K]) {
val rit = matrix.iterator
while (rit.hasNext) {
var col = 0
val cit = rit.next.iterator
val cit = rit.next().iterator
while (cit.hasNext) {
cells(offset + col) += cit.next
cells(offset + col) += cit.next()
col += 1
}
offset += width
Expand All @@ -206,7 +206,7 @@ class CMSSummation[K](params: CMSParams[K]) {
b += cells(offset + col)
col += 1
}
b.result
b.result()
}

val b = Vector.newBuilder[Vector[Long]]
Expand All @@ -215,7 +215,7 @@ class CMSSummation[K](params: CMSParams[K]) {
b += vectorize(row)
row += 1
}
CMSInstance(CMSInstance.CountsTable(b.result), totalCount, params)
CMSInstance(CMSInstance.CountsTable(b.result()), totalCount, params)
}
}

Expand Down Expand Up @@ -724,7 +724,7 @@ case class CMSInstance[K](
val it = countsTable.counts.iterator
var i = 0
while (it.hasNext) {
val row = it.next
val row = it.next()
val count = row(hs(i)(item))
if (count < freq) freq = count
i += 1
Expand Down Expand Up @@ -817,13 +817,13 @@ object CMSInstance {
val yss = other.counts.iterator
val rows = Vector.newBuilder[Vector[Long]]
while (xss.hasNext) {
val xs = xss.next.iterator
val ys = yss.next.iterator
val xs = xss.next().iterator
val ys = yss.next().iterator
val row = Vector.newBuilder[Long]
while (xs.hasNext) row += (xs.next + ys.next)
rows += row.result
while (xs.hasNext) row += (xs.next() + ys.next())
rows += row.result()
}
CountsTable[K](rows.result)
CountsTable[K](rows.result())
}
}

Expand Down
Loading

0 comments on commit 65a95ce

Please sign in to comment.