mirror of
https://github.com/TheAnachronism/docspell.git
synced 2025-06-22 02:18:26 +00:00
Move job queue to scheduler-api and fix notification of periodic tasks
This commit is contained in:
@ -0,0 +1,97 @@
|
||||
/*
|
||||
* Copyright 2020 Eike K. & Contributors
|
||||
*
|
||||
* SPDX-License-Identifier: AGPL-3.0-or-later
|
||||
*/
|
||||
|
||||
package docspell.scheduler
|
||||
|
||||
import cats.effect._
|
||||
import cats.implicits._
|
||||
import docspell.common._
|
||||
import docspell.store.Store
|
||||
import docspell.store.queries.QJob
|
||||
import docspell.store.records.RJob
|
||||
|
||||
trait JobQueue[F[_]] {
|
||||
|
||||
/** Inserts the job into the queue to get picked up as soon as possible. The job must
|
||||
* have a new unique id.
|
||||
*/
|
||||
def insert(job: RJob): F[Unit]
|
||||
|
||||
/** Inserts the job into the queue only, if there is no job with the same tracker-id
|
||||
* running at the moment. The job id must be a new unique id.
|
||||
*
|
||||
* If the job has no tracker defined, it is simply inserted.
|
||||
*/
|
||||
def insertIfNew(job: RJob): F[Boolean]
|
||||
|
||||
def insertAll(jobs: Seq[RJob]): F[List[Boolean]]
|
||||
|
||||
def insertAllIfNew(jobs: Seq[RJob]): F[List[Boolean]]
|
||||
|
||||
def nextJob(
|
||||
prio: Ident => F[Priority],
|
||||
worker: Ident,
|
||||
retryPause: Duration
|
||||
): F[Option[RJob]]
|
||||
}
|
||||
|
||||
object JobQueue {
|
||||
private[scheduler] def create[F[_]: Async](store: Store[F]): Resource[F, JobQueue[F]] =
|
||||
Resource.pure[F, JobQueue[F]](new JobQueue[F] {
|
||||
private[this] val logger = docspell.logging.getLogger[F]
|
||||
|
||||
def nextJob(
|
||||
prio: Ident => F[Priority],
|
||||
worker: Ident,
|
||||
retryPause: Duration
|
||||
): F[Option[RJob]] =
|
||||
logger
|
||||
.trace("Select next job") *> QJob.takeNextJob(store)(prio, worker, retryPause)
|
||||
|
||||
def insert(job: RJob): F[Unit] =
|
||||
store
|
||||
.transact(RJob.insert(job))
|
||||
.flatMap { n =>
|
||||
if (n != 1)
|
||||
Async[F]
|
||||
.raiseError(new Exception(s"Inserting job failed. Update count: $n"))
|
||||
else ().pure[F]
|
||||
}
|
||||
|
||||
def insertIfNew(job: RJob): F[Boolean] =
|
||||
for {
|
||||
rj <- job.tracker match {
|
||||
case Some(tid) =>
|
||||
store.transact(RJob.findNonFinalByTracker(tid))
|
||||
case None =>
|
||||
None.pure[F]
|
||||
}
|
||||
ret <-
|
||||
if (rj.isDefined) false.pure[F]
|
||||
else insert(job).as(true)
|
||||
} yield ret
|
||||
|
||||
def insertAll(jobs: Seq[RJob]): F[List[Boolean]] =
|
||||
jobs.toList
|
||||
.traverse(j => insert(j).attempt)
|
||||
.flatMap(_.traverse {
|
||||
case Right(()) => true.pure[F]
|
||||
case Left(ex) =>
|
||||
logger.error(ex)("Could not insert job. Skipping it.").as(false)
|
||||
|
||||
})
|
||||
|
||||
def insertAllIfNew(jobs: Seq[RJob]): F[List[Boolean]] =
|
||||
jobs.toList
|
||||
.traverse(j => insertIfNew(j).attempt)
|
||||
.flatMap(_.traverse {
|
||||
case Right(true) => true.pure[F]
|
||||
case Right(false) => false.pure[F]
|
||||
case Left(ex) =>
|
||||
logger.error(ex)("Could not insert job. Skipping it.").as(false)
|
||||
})
|
||||
})
|
||||
}
|
@ -27,6 +27,7 @@ trait PeriodicScheduler[F[_]] {
|
||||
def periodicAwake: F[Fiber[F, Throwable, Unit]]
|
||||
|
||||
def notifyChange: F[Unit]
|
||||
}
|
||||
|
||||
object PeriodicScheduler {}
|
||||
/** Starts listening for notify messages in the background. */
|
||||
def startSubscriptions: F[Unit]
|
||||
}
|
||||
|
@ -22,6 +22,10 @@ trait Scheduler[F[_]] {
|
||||
|
||||
def notifyChange: F[Unit]
|
||||
|
||||
/** Starts reacting on notify and cancel messages. */
|
||||
def startSubscriptions: F[Unit]
|
||||
|
||||
/** Starts the schedulers main loop. */
|
||||
def start: Stream[F, Nothing]
|
||||
|
||||
/** Requests to shutdown the scheduler.
|
||||
|
@ -0,0 +1,26 @@
|
||||
/*
|
||||
* Copyright 2020 Eike K. & Contributors
|
||||
*
|
||||
* SPDX-License-Identifier: AGPL-3.0-or-later
|
||||
*/
|
||||
|
||||
package docspell.scheduler.msg
|
||||
|
||||
import docspell.common._
|
||||
import docspell.pubsub.api.{Topic, TypedTopic}
|
||||
import io.circe.generic.semiauto.{deriveDecoder, deriveEncoder}
|
||||
import io.circe.{Decoder, Encoder}
|
||||
|
||||
/** Message to request to cancel a job. */
|
||||
final case class CancelJob(jobId: Ident, nodeId: Ident)
|
||||
|
||||
object CancelJob {
|
||||
implicit val jsonDecoder: Decoder[CancelJob] =
|
||||
deriveDecoder[CancelJob]
|
||||
|
||||
implicit val jsonEncoder: Encoder[CancelJob] =
|
||||
deriveEncoder[CancelJob]
|
||||
|
||||
val topic: TypedTopic[CancelJob] =
|
||||
TypedTopic(Topic("job-cancel-request"))
|
||||
}
|
@ -0,0 +1,78 @@
|
||||
/*
|
||||
* Copyright 2020 Eike K. & Contributors
|
||||
*
|
||||
* SPDX-License-Identifier: AGPL-3.0-or-later
|
||||
*/
|
||||
|
||||
package docspell.scheduler.msg
|
||||
|
||||
import cats.effect._
|
||||
import cats.implicits._
|
||||
import docspell.common.{Duration, Ident, Priority}
|
||||
import docspell.notification.api.{Event, EventSink}
|
||||
import docspell.pubsub.api.PubSubT
|
||||
import docspell.scheduler.JobQueue
|
||||
import docspell.store.Store
|
||||
import docspell.store.records.RJob
|
||||
|
||||
final class JobQueuePublish[F[_]: Sync](
|
||||
delegate: JobQueue[F],
|
||||
pubsub: PubSubT[F],
|
||||
eventSink: EventSink[F]
|
||||
) extends JobQueue[F] {
|
||||
|
||||
private def msg(job: RJob): JobSubmitted =
|
||||
JobSubmitted(job.id, job.group, job.task, job.args)
|
||||
|
||||
private def event(job: RJob): Event.JobSubmitted =
|
||||
Event.JobSubmitted(
|
||||
job.id,
|
||||
job.group,
|
||||
job.task,
|
||||
job.args,
|
||||
job.state,
|
||||
job.subject,
|
||||
job.submitter
|
||||
)
|
||||
|
||||
private def publish(job: RJob): F[Unit] =
|
||||
pubsub.publish1(JobSubmitted.topic, msg(job)).as(()) *>
|
||||
eventSink.offer(event(job))
|
||||
|
||||
def insert(job: RJob) =
|
||||
delegate.insert(job).flatTap(_ => publish(job))
|
||||
|
||||
def insertIfNew(job: RJob) =
|
||||
delegate.insertIfNew(job).flatTap {
|
||||
case true => publish(job)
|
||||
case false => ().pure[F]
|
||||
}
|
||||
|
||||
def insertAll(jobs: Seq[RJob]) =
|
||||
delegate.insertAll(jobs).flatTap { results =>
|
||||
results.zip(jobs).traverse { case (res, job) =>
|
||||
if (res) publish(job)
|
||||
else ().pure[F]
|
||||
}
|
||||
}
|
||||
|
||||
def insertAllIfNew(jobs: Seq[RJob]) =
|
||||
delegate.insertAllIfNew(jobs).flatTap { results =>
|
||||
results.zip(jobs).traverse { case (res, job) =>
|
||||
if (res) publish(job)
|
||||
else ().pure[F]
|
||||
}
|
||||
}
|
||||
|
||||
def nextJob(prio: Ident => F[Priority], worker: Ident, retryPause: Duration) =
|
||||
delegate.nextJob(prio, worker, retryPause)
|
||||
}
|
||||
|
||||
object JobQueuePublish {
|
||||
def apply[F[_]: Async](
|
||||
store: Store[F],
|
||||
pubSub: PubSubT[F],
|
||||
eventSink: EventSink[F]
|
||||
): Resource[F, JobQueue[F]] =
|
||||
JobQueue.create(store).map(q => new JobQueuePublish[F](q, pubSub, eventSink))
|
||||
}
|
@ -0,0 +1,26 @@
|
||||
/*
|
||||
* Copyright 2020 Eike K. & Contributors
|
||||
*
|
||||
* SPDX-License-Identifier: AGPL-3.0-or-later
|
||||
*/
|
||||
|
||||
package docspell.scheduler.msg
|
||||
|
||||
import docspell.common._
|
||||
import docspell.pubsub.api.{Topic, TypedTopic}
|
||||
import io.circe.generic.semiauto.{deriveDecoder, deriveEncoder}
|
||||
import io.circe.{Decoder, Encoder}
|
||||
|
||||
final case class JobSubmitted(jobId: Ident, group: Ident, task: Ident, args: String)
|
||||
|
||||
object JobSubmitted {
|
||||
|
||||
implicit val jsonDecoder: Decoder[JobSubmitted] =
|
||||
deriveDecoder
|
||||
|
||||
implicit val jsonEncoder: Encoder[JobSubmitted] =
|
||||
deriveEncoder
|
||||
|
||||
val topic: TypedTopic[JobSubmitted] =
|
||||
TypedTopic(Topic("job-submitted"))
|
||||
}
|
@ -0,0 +1,9 @@
|
||||
package docspell.scheduler.msg
|
||||
|
||||
import docspell.pubsub.api.{Topic, TypedTopic}
|
||||
|
||||
/** A generic notification to the job executors to look for new work. */
|
||||
object JobsNotify {
|
||||
def apply(): TypedTopic[Unit] =
|
||||
TypedTopic[Unit](Topic("jobs-notify"))
|
||||
}
|
@ -0,0 +1,9 @@
|
||||
package docspell.scheduler.msg
|
||||
|
||||
import docspell.pubsub.api.{Topic, TypedTopic}
|
||||
|
||||
/** A generic notification to the periodic task scheduler to look for new work. */
|
||||
object PeriodicTaskNotify {
|
||||
def apply(): TypedTopic[Unit] =
|
||||
TypedTopic[Unit](Topic("periodic-task-notify"))
|
||||
}
|
Reference in New Issue
Block a user