use async db coll in puzzle module

This commit is contained in:
Thibault Duplessis 2019-12-07 10:43:48 -06:00
parent 26573d5b18
commit 4134b38e6b
9 changed files with 173 additions and 135 deletions

View file

@ -11,6 +11,8 @@ final class AsyncColl(resolve: () => Fu[Coll]) {
def apply[A](f: Coll => Fu[A]) = get flatMap f
def map[A](f: Coll => A) = get map f
def find(selector: Bdoc) = get.map(_.find(selector, none))
def find(selector: Bdoc, proj: Bdoc) = get.map(_.find(selector, proj.some))

View file

@ -6,11 +6,12 @@ import akka.actor.Scheduler
import akka.pattern.ask
import org.joda.time.DateTime
import lila.db.AsyncColl
import lila.db.dsl._
import Puzzle.{ BSONFields => F }
private[puzzle] final class Daily(
coll: Coll,
coll: AsyncColl,
renderer: lila.hub.actors.Renderer,
asyncCache: lila.memo.AsyncCache.Builder,
scheduler: Scheduler
@ -47,19 +48,23 @@ private[puzzle] final class Daily(
none
}
private def findCurrent = coll.ext.find(
$doc(F.day $gt DateTime.now.minusMinutes(24 * 60 - 15))
).uno[Puzzle]
private def findCurrent = coll {
_.ext.find(
$doc(F.day $gt DateTime.now.minusMinutes(24 * 60 - 15))
).uno[Puzzle]
}
private def findNew = coll.ext.find(
$doc(F.day $exists false, F.voteNb $gte 200)
).sort($doc(F.voteRatio -> -1)).uno[Puzzle] flatMap {
case Some(puzzle) => coll.update.one(
$id(puzzle.id),
$set(F.day -> DateTime.now)
) inject puzzle.some
case None => fuccess(none)
}
private def findNew = coll { c =>
c.ext.find(
$doc(F.day $exists false, F.voteNb $gte 200)
).sort($doc(F.voteRatio -> -1)).uno[Puzzle] flatMap {
case Some(puzzle) => c.update.one(
$id(puzzle.id),
$set(F.day -> DateTime.now)
) inject puzzle.some
case None => fuccess(none)
}
}
}
object Daily {

View file

@ -35,7 +35,11 @@ final class Env(
private val config = appConfig.get[PuzzleConfig]("puzzle")(AutoConfig.loader)
private lazy val db = mongo.blockingDb("puzzle", config.mongoUri)
private lazy val db = mongo.asyncDb("puzzle", config.mongoUri)
private def puzzleColl = db(config.puzzleColl)
private def roundColl = db(config.roundColl)
private def voteColl = db(config.voteColl)
private def headColl = db(config.headColl)
private lazy val gameJson = wire[GameJson]
@ -97,9 +101,4 @@ final class Env(
}
}
}
private lazy val puzzleColl = db(config.puzzleColl)
private lazy val roundColl = db(config.roundColl)
private lazy val voteColl = db(config.voteColl)
private lazy val headColl = db(config.headColl)
}

View file

@ -5,6 +5,7 @@ import org.joda.time.DateTime
import chess.Mode
import lila.common.Bus
import lila.db.AsyncColl
import lila.db.dsl._
import lila.rating.{ Glicko, PerfType }
import lila.user.{ User, UserRepo }
@ -13,7 +14,7 @@ private[puzzle] final class Finisher(
api: PuzzleApi,
userRepo: UserRepo,
historyApi: lila.history.HistoryApi,
puzzleColl: Coll
puzzleColl: AsyncColl
) {
def apply(puzzle: Puzzle, user: User, result: Result, mobile: Boolean): Fu[(Round, Mode)] = {
@ -36,26 +37,25 @@ private[puzzle] final class Finisher(
)
historyApi.addPuzzle(user = user, completedAt = date, perf = userPerf)
(api.round upsert round) >> {
puzzleColl.update.one(
$id(puzzle.id),
$inc(Puzzle.BSONFields.attempts -> $int(1)) ++
$set(Puzzle.BSONFields.perf -> PuzzlePerf.puzzlePerfBSONHandler.write(puzzlePerf))
) zip userRepo.setPerf(user.id, PerfType.Puzzle, userPerf)
puzzleColl {
_.update.one(
$id(puzzle.id),
$inc(Puzzle.BSONFields.attempts -> $int(1)) ++
$set(Puzzle.BSONFields.perf -> PuzzlePerf.puzzlePerfBSONHandler.write(puzzlePerf))
)
} zip userRepo.setPerf(user.id, PerfType.Puzzle, userPerf)
} inject {
Bus.publish(Puzzle.UserResult(puzzle.id, user.id, result, formerUserRating -> userPerf.intRating), "finishPuzzle")
round -> Mode.Rated
}
}
case _ => fuccess {
incPuzzleAttempts(puzzle)
new Round(
id = Round.Id(user.id, puzzle.id),
date = DateTime.now,
result = result,
rating = formerUserRating,
ratingDiff = 0
) -> Mode.Casual
}
case _ => incPuzzleAttempts(puzzle) inject new Round(
id = Round.Id(user.id, puzzle.id),
date = DateTime.now,
result = result,
rating = formerUserRating,
ratingDiff = 0
) -> Mode.Casual
}
}
@ -93,8 +93,8 @@ private[puzzle] final class Finisher(
private val TAU = 0.75d
private val system = new RatingCalculator(VOLATILITY, TAU)
def incPuzzleAttempts(puzzle: Puzzle) =
puzzleColl.incFieldUnchecked($id(puzzle.id), Puzzle.BSONFields.attempts)
def incPuzzleAttempts(puzzle: Puzzle): Funit =
puzzleColl.map(_.incFieldUnchecked($id(puzzle.id), Puzzle.BSONFields.attempts))
private def updateRatings(u1: Rating, u2: Rating, result: Glicko.Result): Unit = {
val results = new RatingPeriodResults()

View file

@ -9,32 +9,36 @@ import reactivemongo.api.ReadPreference
import scala.concurrent.duration._
import lila.common.config.MaxPerSecond
import lila.db.AsyncColl
import lila.db.dsl._
import lila.user.User
final class PuzzleActivity(
puzzleColl: Coll,
roundColl: Coll
puzzleColl: AsyncColl,
roundColl: AsyncColl
)(implicit system: akka.actor.ActorSystem) {
import PuzzleActivity._
import Round.RoundBSONHandler
def stream(config: Config): Source[String, _] = roundColl
.ext.find($doc("_id" $startsWith config.user.id))
.sort($sort desc "_id")
.batchSize(config.perSecond.value)
.cursor[Round](ReadPreference.secondaryPreferred)
.documentSource()
.take(config.max | Int.MaxValue)
.grouped(config.perSecond.value)
.delay(1 second)
.mapAsync(1)(enrich)
.mapConcat(identity)
.map { json => s"${Json.stringify(json)}\n" }
def stream(config: Config): Source[String, _] = Source futureSource {
roundColl.map {
_.ext.find($doc("_id" $startsWith config.user.id))
.sort($sort desc "_id")
.batchSize(config.perSecond.value)
.cursor[Round](ReadPreference.secondaryPreferred)
.documentSource()
.take(config.max | Int.MaxValue)
.grouped(config.perSecond.value)
.delay(1 second)
.mapAsync(1)(enrich)
.mapConcat(identity)
.map { json => s"${Json.stringify(json)}\n" }
}
}
private def enrich(rounds: Seq[Round]): Fu[Seq[JsObject]] =
puzzleColl.primitiveMap[Int, Double](
private def enrich(rounds: Seq[Round]): Fu[Seq[JsObject]] = puzzleColl {
_.primitiveMap[Int, Double](
ids = rounds.map(_.id.puzzleId).toSeq,
field = "perf.gl.r",
fieldExtractor = obj => for {
@ -55,6 +59,7 @@ final class PuzzleActivity(
}
}
}
}
}
object PuzzleActivity {

View file

@ -5,15 +5,16 @@ import scala.concurrent.duration._
import play.api.libs.json.JsValue
import lila.common.config.Secret
import lila.db.AsyncColl
import lila.db.dsl._
import lila.user.User
import Puzzle.{ BSONFields => F }
private[puzzle] final class PuzzleApi(
puzzleColl: Coll,
roundColl: Coll,
voteColl: Coll,
headColl: Coll,
puzzleColl: AsyncColl,
roundColl: AsyncColl,
voteColl: AsyncColl,
headColl: AsyncColl,
puzzleIdMin: PuzzleId,
asyncCache: lila.memo.AsyncCache.Builder,
apiToken: Secret
@ -24,86 +25,97 @@ private[puzzle] final class PuzzleApi(
object puzzle {
def find(id: PuzzleId): Fu[Option[Puzzle]] =
puzzleColl.ext.find($doc(F.id -> id)).uno[Puzzle]
puzzleColl(_.ext.find($doc(F.id -> id)).uno[Puzzle])
def findMany(ids: List[PuzzleId]): Fu[List[Option[Puzzle]]] =
puzzleColl.optionsByOrderedIds[Puzzle, PuzzleId](ids)(_.id)
puzzleColl(_.optionsByOrderedIds[Puzzle, PuzzleId](ids)(_.id))
def latest(nb: Int): Fu[List[Puzzle]] =
puzzleColl.ext.find($empty)
def latest(nb: Int): Fu[List[Puzzle]] = puzzleColl {
_.ext.find($empty)
.sort($doc(F.date -> -1))
.cursor[Puzzle]()
.gather[List](nb)
.list[Puzzle](nb)
}
val cachedLastId = asyncCache.single(
name = "puzzle.lastId",
f = lila.db.Util findNextId puzzleColl map (_ - 1),
f = puzzleColl(lila.db.Util.findNextId) dmap (_ - 1),
expireAfter = _.ExpireAfterWrite(1 day)
)
def export(nb: Int): Fu[List[Puzzle]] = List(true, false).map { mate =>
puzzleColl.ext.find($doc(F.mate -> mate))
.sort($doc(F.voteRatio -> -1))
.cursor[Puzzle]().gather[List](nb / 2)
}.sequenceFu.map(_.flatten)
// def export(nb: Int): Fu[List[Puzzle]] = List(true, false).map { mate =>
// puzzleColl {
// _.ext.find($doc(F.mate -> mate))
// .sort($doc(F.voteRatio -> -1))
// .list[Puzzle](nb / 2)
// }
// }.sequenceFu.map(_.flatten)
def disable(id: PuzzleId): Funit =
puzzleColl.update.one(
def disable(id: PuzzleId): Funit = puzzleColl {
_.update.one(
$id(id),
$doc("$set" -> $doc(F.vote -> AggregateVote.disable))
).void
}
}
object round {
def add(a: Round) = roundColl.insert.one(a)
def add(a: Round) = roundColl(_.insert.one(a))
def upsert(a: Round) = roundColl.update.one($id(a.id), a, upsert = true)
def upsert(a: Round) = roundColl(_.update.one($id(a.id), a, upsert = true))
def reset(user: User) = roundColl.delete.one($doc(
Round.BSONFields.id $startsWith s"${user.id}:"
))
def reset(user: User) = roundColl {
_.delete.one($doc(
Round.BSONFields.id $startsWith s"${user.id}:"
))
}
}
object vote {
def value(id: PuzzleId, user: User): Fu[Option[Boolean]] =
voteColl.primitiveOne[Boolean]($id(Vote.makeId(id, user.id)), "v")
def find(id: PuzzleId, user: User): Fu[Option[Vote]] = voteColl.byId[Vote](Vote.makeId(id, user.id))
def update(id: PuzzleId, user: User, v1: Option[Vote], v: Boolean): Fu[(Puzzle, Vote)] = puzzle find id flatMap {
case None => fufail(s"Can't vote for non existing puzzle ${id}")
case Some(p1) =>
val (p2, v2) = v1 match {
case Some(from) => (
(p1 withVote (_.change(from.value, v))),
from.copy(v = v)
)
case None => (
(p1 withVote (_ add v)),
Vote(Vote.makeId(id, user.id), v)
)
}
voteColl.update.one(
$id(v2.id),
$set("v" -> v),
upsert = true
) zip
puzzleColl.update.one(
$id(p2.id),
$set(F.vote -> p2.vote)
) map {
case _ => p2 -> v2
}
def value(id: PuzzleId, user: User): Fu[Option[Boolean]] = voteColl {
_.primitiveOne[Boolean]($id(Vote.makeId(id, user.id)), "v")
}
def find(id: PuzzleId, user: User): Fu[Option[Vote]] = voteColl {
_.byId[Vote](Vote.makeId(id, user.id))
}
def update(id: PuzzleId, user: User, v1: Option[Vote], v: Boolean): Fu[(Puzzle, Vote)] =
puzzle find id flatMap {
case None => fufail(s"Can't vote for non existing puzzle ${id}")
case Some(p1) =>
val (p2, v2) = v1 match {
case Some(from) => (
(p1 withVote (_.change(from.value, v))),
from.copy(v = v)
)
case None => (
(p1 withVote (_ add v)),
Vote(Vote.makeId(id, user.id), v)
)
}
voteColl {
_.update.one(
$id(v2.id),
$set("v" -> v),
upsert = true
)
} zip
puzzleColl {
_.update.one(
$id(p2.id),
$set(F.vote -> p2.vote)
)
} inject (p2 -> v2)
}
}
object head {
def find(user: User): Fu[Option[PuzzleHead]] = headColl.byId[PuzzleHead](user.id)
def find(user: User): Fu[Option[PuzzleHead]] = headColl(_.byId[PuzzleHead](user.id))
def set(h: PuzzleHead) = headColl.update.one($id(h.id), h, upsert = true) void
def set(h: PuzzleHead) = headColl(_.update.one($id(h.id), h, upsert = true).void)
def addNew(user: User, puzzleId: PuzzleId) = set(PuzzleHead(user.id, puzzleId.some, puzzleId))

View file

@ -1,11 +1,12 @@
package lila.puzzle
import lila.db.AsyncColl
import lila.db.dsl._
import lila.user.User
import Puzzle.{ BSONFields => F }
private[puzzle] final class PuzzleBatch(
puzzleColl: Coll,
puzzleColl: AsyncColl,
api: PuzzleApi,
finisher: Finisher,
puzzleIdMin: PuzzleId
@ -47,29 +48,34 @@ private[puzzle] final class PuzzleBatch(
case Some(PuzzleHead(_, _, l)) if l < maxId - 500 => after.fold(l)(_ atLeast l)
case _ => puzzleIdMin
}
tryRange(
rating = rating,
tolerance = step,
step = step,
idRange = Range(fromId, fromId + nb * 50),
nb = nb
)
puzzleColl { coll =>
tryRange(
coll = coll,
rating = rating,
tolerance = step,
step = step,
idRange = Range(fromId, fromId + nb * 50),
nb = nb
)
}
}
}
private def tryRange(
coll: Coll,
rating: Int,
tolerance: Int,
step: Int,
idRange: Range,
nb: Int
): Fu[List[Puzzle]] = puzzleColl.ext.find(rangeSelector(
): Fu[List[Puzzle]] = coll.ext.find(rangeSelector(
rating = rating,
tolerance = tolerance,
idRange = idRange
)).list[Puzzle](nb) flatMap {
case res if res.size < nb && (tolerance + step) <= toleranceMax =>
tryRange(
coll = coll,
rating = rating,
tolerance = tolerance + step,
step = step,

View file

@ -2,13 +2,14 @@ package lila.puzzle
import scala.util.Random
import lila.db.AsyncColl
import lila.db.dsl._
import lila.rating.Perf
import lila.user.User
import Puzzle.{ BSONFields => F }
private[puzzle] final class Selector(
puzzleColl: Coll,
puzzleColl: AsyncColl,
api: PuzzleApi,
puzzleIdMin: Int
) {
@ -19,11 +20,12 @@ private[puzzle] final class Selector(
lila.mon.puzzle.selector.count()
me match {
// anon
case None => puzzleColl // this query precisely matches a mongodb partial index
.ext.find($doc(F.voteNb $gte 50))
.sort($sort desc F.voteRatio)
.skip(Random nextInt anonSkipMax)
.uno[Puzzle]
case None => puzzleColl { // this query precisely matches a mongodb partial index
_.ext.find($doc(F.voteNb $gte 50))
.sort($sort desc F.voteRatio)
.skip(Random nextInt anonSkipMax)
.uno[Puzzle]
}
// user
case Some(user) => api.head find user flatMap {
// new player
@ -56,26 +58,30 @@ private[puzzle] final class Selector(
private def newPuzzleForUser(user: User, lastPlayed: PuzzleId): Fu[Option[Puzzle]] = {
val rating = user.perfs.puzzle.intRating atMost 2300 atLeast 900
val step = toleranceStepFor(rating, user.perfs.puzzle.nb)
tryRange(
rating = rating,
tolerance = step,
step = step,
idRange = Range(lastPlayed, lastPlayed + 200)
)
puzzleColl { coll =>
tryRange(
coll = coll,
rating = rating,
tolerance = step,
step = step,
idRange = Range(lastPlayed, lastPlayed + 200)
)
}
}
private def tryRange(
coll: Coll,
rating: Int,
tolerance: Int,
step: Int,
idRange: Range
): Fu[Option[Puzzle]] = puzzleColl.ext.find(rangeSelector(
): Fu[Option[Puzzle]] = coll.ext.find(rangeSelector(
rating = rating,
tolerance = tolerance,
idRange = idRange
)).sort($sort asc F.id).uno[Puzzle] flatMap {
case None if (tolerance + step) <= toleranceMax =>
tryRange(rating, tolerance + step, step, Range(idRange.min, idRange.max + 100))
tryRange(coll, rating, tolerance + step, step, Range(idRange.min, idRange.max + 100))
case res => fuccess(res)
}
}

View file

@ -2,12 +2,13 @@ package lila.puzzle
import reactivemongo.api.bson._
import lila.db.AsyncColl
import lila.db.dsl._
import lila.user.User
case class UserInfos(user: User, history: List[Round])
final class UserInfosApi(roundColl: Coll, currentPuzzleId: User => Fu[Option[PuzzleId]]) {
final class UserInfosApi(roundColl: AsyncColl, currentPuzzleId: User => Fu[Option[PuzzleId]]) {
private val historySize = 15
private val chartSize = 15
@ -22,9 +23,11 @@ final class UserInfosApi(roundColl: Coll, currentPuzzleId: User => Fu[Option[Puz
private def fetchRounds(userId: User.ID, currentPuzzleId: Option[PuzzleId]): Fu[List[Round]] = {
val idSelector = $doc("$regex" -> BSONRegex(s"^$userId:", "")) ++
currentPuzzleId.?? { id => $doc("$lte" -> s"$userId:${Round encode id}") }
roundColl.ext.find($doc(Round.BSONFields.id -> idSelector))
.sort($sort desc Round.BSONFields.id)
.list[Round](historySize atLeast chartSize)
.map(_.reverse)
roundColl {
_.ext.find($doc(Round.BSONFields.id -> idSelector))
.sort($sort desc Round.BSONFields.id)
.list[Round](historySize atLeast chartSize)
.dmap(_.reverse)
}
}
}