Merge pull request #163 from eikek/job-cancelling

Fix cancelling jobs
This commit is contained in:
mergify[bot] 2020-06-26 21:27:11 +00:00 committed by GitHub
commit 9d1717f0e0
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
11 changed files with 191 additions and 79 deletions

View File

@ -307,6 +307,7 @@ val joexapi = project.in(file("modules/joexapi")).
name := "docspell-joexapi", name := "docspell-joexapi",
libraryDependencies ++= libraryDependencies ++=
Dependencies.circe ++ Dependencies.circe ++
Dependencies.http4sCirce ++
Dependencies.http4sClient, Dependencies.http4sClient,
openapiTargetLanguage := Language.Scala, openapiTargetLanguage := Language.Scala,
openapiPackage := Pkg("docspell.joexapi.model"), openapiPackage := Pkg("docspell.joexapi.model"),

View File

@ -2,6 +2,7 @@ package docspell.backend.ops
import cats.implicits._ import cats.implicits._
import cats.effect._ import cats.effect._
import cats.data.OptionT
import docspell.backend.ops.OJob.{CollectiveQueueState, JobCancelResult} import docspell.backend.ops.OJob.{CollectiveQueueState, JobCancelResult}
import docspell.common.{Ident, JobState} import docspell.common.{Ident, JobState}
import docspell.store.Store import docspell.store.Store
@ -22,6 +23,10 @@ object OJob {
case object Removed extends JobCancelResult case object Removed extends JobCancelResult
case object CancelRequested extends JobCancelResult case object CancelRequested extends JobCancelResult
case object JobNotFound extends JobCancelResult case object JobNotFound extends JobCancelResult
def removed: JobCancelResult = Removed
def cancelRequested: JobCancelResult = CancelRequested
def jobNotFound: JobCancelResult = JobNotFound
} }
case class JobDetail(job: RJob, logs: Vector[RJobLog]) case class JobDetail(job: RJob, logs: Vector[RJobLog])
@ -49,43 +54,30 @@ object OJob {
.map(CollectiveQueueState) .map(CollectiveQueueState)
def cancelJob(id: Ident, collective: Ident): F[JobCancelResult] = { def cancelJob(id: Ident, collective: Ident): F[JobCancelResult] = {
def mustCancel(job: Option[RJob]): Option[(RJob, Ident)] = def remove(job: RJob): F[JobCancelResult] =
store.transact(RJob.delete(job.id)) *> JobCancelResult.removed.pure[F]
def tryCancel(job: RJob): F[JobCancelResult] =
job.worker match {
case Some(worker) =>
for { for {
worker <- job.flatMap(_.worker) flag <- joex.cancelJob(job.id, worker)
job <- job.filter(j => res <-
j.state == JobState.Scheduled || j.state == JobState.Running if (flag) JobCancelResult.cancelRequested.pure[F]
) else remove(job)
} yield (job, worker) } yield res
case None =>
def canDelete(j: RJob): Boolean = remove(job)
mustCancel(j.some).isEmpty
val tryDelete = for {
job <- RJob.findByIdAndGroup(id, collective)
jobm = job.filter(canDelete)
del <- jobm.traverse(j => RJob.delete(j.id))
} yield del match {
case Some(_) => Right(JobCancelResult.Removed: JobCancelResult)
case None => Left(mustCancel(job))
} }
def tryCancel(job: RJob, worker: Ident): F[JobCancelResult] = (for {
joex job <- OptionT(store.transact(RJob.findByIdAndGroup(id, collective)))
.cancelJob(job.id, worker) result <- OptionT.liftF(
.map(flag => if (job.isInProgress) tryCancel(job)
if (flag) JobCancelResult.CancelRequested else JobCancelResult.JobNotFound else remove(job)
) )
} yield result)
for { .getOrElse(JobCancelResult.jobNotFound)
tryDel <- store.transact(tryDelete)
result <- tryDel match {
case Right(r) => r.pure[F]
case Left(Some((job, worker))) =>
tryCancel(job, worker)
case Left(None) =>
(JobCancelResult.JobNotFound: OJob.JobCancelResult).pure[F]
}
} yield result
} }
}) })
} }

View File

@ -2,6 +2,7 @@ package docspell.backend.ops
import cats.implicits._ import cats.implicits._
import cats.effect._ import cats.effect._
import cats.data.OptionT
import docspell.common.{Ident, NodeType} import docspell.common.{Ident, NodeType}
import docspell.joexapi.client.JoexClient import docspell.joexapi.client.JoexClient
import docspell.store.Store import docspell.store.Store
@ -28,10 +29,10 @@ object OJoex {
} yield () } yield ()
def cancelJob(job: Ident, worker: Ident): F[Boolean] = def cancelJob(job: Ident, worker: Ident): F[Boolean] =
for { (for {
node <- store.transact(RNode.findById(worker)) node <- OptionT(store.transact(RNode.findById(worker)))
cancel <- node.traverse(n => client.cancelJob(n.url, job)) cancel <- OptionT.liftF(client.cancelJob(node.url, job))
} yield cancel.isDefined } yield cancel.success).getOrElse(false)
}) })
def create[F[_]: ConcurrentEffect]( def create[F[_]: ConcurrentEffect](

View File

@ -20,7 +20,7 @@ object JobState {
/** Is currently executing */ /** Is currently executing */
case object Running extends JobState {} case object Running extends JobState {}
/** Finished with failure and is being retried. */ /** Task completed with failure and is being retried. */
case object Stuck extends JobState {} case object Stuck extends JobState {}
/** Finished finally with a failure */ /** Finished finally with a failure */
@ -36,6 +36,7 @@ object JobState {
Set(Waiting, Scheduled, Running, Stuck, Failed, Cancelled, Success) Set(Waiting, Scheduled, Running, Stuck, Failed, Cancelled, Success)
val queued: Set[JobState] = Set(Waiting, Scheduled, Stuck) val queued: Set[JobState] = Set(Waiting, Scheduled, Stuck)
val done: Set[JobState] = Set(Failed, Cancelled, Success) val done: Set[JobState] = Set(Failed, Cancelled, Success)
val inProgress: Set[JobState] = Set(Scheduled, Running, Stuck)
def parse(str: String): Either[String, JobState] = def parse(str: String): Either[String, JobState] =
str.toLowerCase match { str.toLowerCase match {

View File

@ -2,6 +2,7 @@ package docspell.joex.process
import cats.implicits._ import cats.implicits._
import cats.effect._ import cats.effect._
import cats.data.OptionT
import fs2.Stream import fs2.Stream
import docspell.common.{ItemState, ProcessItemArgs} import docspell.common.{ItemState, ProcessItemArgs}
import docspell.joex.Config import docspell.joex.Config
@ -11,15 +12,22 @@ import docspell.store.records.RItem
import docspell.ftsclient.FtsClient import docspell.ftsclient.FtsClient
object ItemHandler { object ItemHandler {
def onCancel[F[_]: Sync: ContextShift]: Task[F, ProcessItemArgs, Unit] = type Args = ProcessItemArgs
logWarn("Now cancelling. Deleting potentially created data.").flatMap(_ =>
deleteByFileIds.flatMap(_ => deleteFiles) def onCancel[F[_]: Sync: ContextShift]: Task[F, Args, Unit] =
logWarn("Now cancelling.").flatMap(_ =>
markItemCreated.flatMap {
case true =>
Task.pure(())
case false =>
deleteByFileIds[F].flatMap(_ => deleteFiles)
}
) )
def newItem[F[_]: ConcurrentEffect: ContextShift]( def newItem[F[_]: ConcurrentEffect: ContextShift](
cfg: Config, cfg: Config,
fts: FtsClient[F] fts: FtsClient[F]
): Task[F, ProcessItemArgs, Unit] = ): Task[F, Args, Unit] =
CreateItem[F] CreateItem[F]
.flatMap(itemStateTask(ItemState.Processing)) .flatMap(itemStateTask(ItemState.Processing))
.flatMap(safeProcess[F](cfg, fts)) .flatMap(safeProcess[F](cfg, fts))
@ -34,13 +42,13 @@ object ItemHandler {
.map(_ => data) .map(_ => data)
) )
def isLastRetry[F[_]: Sync]: Task[F, ProcessItemArgs, Boolean] = def isLastRetry[F[_]: Sync]: Task[F, Args, Boolean] =
Task(_.isLastRetry) Task(_.isLastRetry)
def safeProcess[F[_]: ConcurrentEffect: ContextShift]( def safeProcess[F[_]: ConcurrentEffect: ContextShift](
cfg: Config, cfg: Config,
fts: FtsClient[F] fts: FtsClient[F]
)(data: ItemData): Task[F, ProcessItemArgs, ItemData] = )(data: ItemData): Task[F, Args, ItemData] =
isLastRetry[F].flatMap { isLastRetry[F].flatMap {
case true => case true =>
ProcessItem[F](cfg, fts)(data).attempt.flatMap({ ProcessItem[F](cfg, fts)(data).attempt.flatMap({
@ -56,17 +64,43 @@ object ItemHandler {
ProcessItem[F](cfg, fts)(data).flatMap(itemStateTask(ItemState.Created)) ProcessItem[F](cfg, fts)(data).flatMap(itemStateTask(ItemState.Created))
} }
def deleteByFileIds[F[_]: Sync: ContextShift]: Task[F, ProcessItemArgs, Unit] = private def markItemCreated[F[_]: Sync]: Task[F, Args, Boolean] =
Task { ctx =>
val fileMetaIds = ctx.args.files.map(_.fileMetaId).toSet
(for {
item <- OptionT(ctx.store.transact(QItem.findOneByFileIds(fileMetaIds.toSeq)))
_ <- OptionT.liftF(
ctx.logger.info("Processing cancelled. Marking item as created anyways.")
)
_ <- OptionT.liftF(
ctx.store
.transact(
RItem.updateState(item.id, ItemState.Created, ItemState.invalidStates)
)
)
} yield true)
.getOrElseF(
ctx.logger.warn("Processing cancelled. No item created").map(_ => false)
)
}
def deleteByFileIds[F[_]: Sync: ContextShift]: Task[F, Args, Unit] =
Task { ctx => Task { ctx =>
for { for {
items <- ctx.store.transact(QItem.findByFileIds(ctx.args.files.map(_.fileMetaId))) items <- ctx.store.transact(QItem.findByFileIds(ctx.args.files.map(_.fileMetaId)))
_ <- ctx.logger.info(s"Deleting items ${items.map(_.id.id)}") _ <-
if (items.nonEmpty) ctx.logger.info(s"Deleting items ${items.map(_.id.id)}")
else
ctx.logger.info(
s"No items found for file ids ${ctx.args.files.map(_.fileMetaId)}"
)
_ <- items.traverse(i => QItem.delete(ctx.store)(i.id, ctx.args.meta.collective)) _ <- items.traverse(i => QItem.delete(ctx.store)(i.id, ctx.args.meta.collective))
} yield () } yield ()
} }
private def deleteFiles[F[_]: Sync]: Task[F, ProcessItemArgs, Unit] = private def deleteFiles[F[_]: Sync]: Task[F, Args, Unit] =
Task(ctx => Task(ctx =>
ctx.logger.info("Deleting input files …") *>
Stream Stream
.emits(ctx.args.files.map(_.fileMetaId.id)) .emits(ctx.args.files.map(_.fileMetaId.id))
.flatMap(id => ctx.store.bitpeace.delete(id).attempt.drain) .flatMap(id => ctx.store.bitpeace.delete(id).attempt.drain)
@ -74,6 +108,6 @@ object ItemHandler {
.drain .drain
) )
private def logWarn[F[_]](msg: => String): Task[F, ProcessItemArgs, Unit] = private def logWarn[F[_]](msg: => String): Task[F, Args, Unit] =
Task(_.logger.warn(msg)) Task(_.logger.warn(msg))
} }

View File

@ -30,5 +30,4 @@ object ProcessItem {
.flatMap(FindProposal[F](cfg.processing)) .flatMap(FindProposal[F](cfg.processing))
.flatMap(EvalProposals[F]) .flatMap(EvalProposals[F])
.flatMap(SaveProposals[F]) .flatMap(SaveProposals[F])
} }

View File

@ -3,6 +3,7 @@ package docspell.joex.scheduler
import fs2.Stream import fs2.Stream
import cats.implicits._ import cats.implicits._
import cats.effect.concurrent.Semaphore import cats.effect.concurrent.Semaphore
import cats.data.OptionT
import docspell.common._ import docspell.common._
import docspell.common.syntax.all._ import docspell.common.syntax.all._
import docspell.store.queue.JobQueue import docspell.store.queue.JobQueue
@ -51,7 +52,16 @@ final class SchedulerImpl[F[_]: ConcurrentEffect: ContextShift](
state.get.flatMap(_.cancelRequest(jobId) match { state.get.flatMap(_.cancelRequest(jobId) match {
case Some(ct) => ct.map(_ => true) case Some(ct) => ct.map(_ => true)
case None => case None =>
(for {
job <- OptionT(store.transact(RJob.findByIdAndWorker(jobId, config.name)))
_ <- OptionT.liftF(
if (job.isInProgress) executeCancel(job)
else ().pure[F]
)
} yield true)
.getOrElseF(
logger.fwarn(s"Job ${jobId.id} not found, cannot cancel.").map(_ => false) logger.fwarn(s"Job ${jobId.id} not found, cannot cancel.").map(_ => false)
)
}) })
def notifyChange: F[Unit] = def notifyChange: F[Unit] =
@ -127,6 +137,31 @@ final class SchedulerImpl[F[_]: ConcurrentEffect: ContextShift](
}) })
} }
private def executeCancel(job: RJob): F[Unit] = {
val task = for {
jobtask <-
tasks
.find(job.task)
.toRight(s"This executor cannot run tasks with name: ${job.task}")
} yield jobtask
task match {
case Left(err) =>
logger.ferror(s"Unable to run cancellation task for job ${job.info}: $err")
case Right(t) =>
for {
_ <-
logger.fdebug(s"Creating context for job ${job.info} to run cancellation $t")
ctx <- Context[F, String](job, job.args, config, logSink, blocker, store)
_ <- t.onCancel.run(ctx)
_ <- state.modify(_.markCancelled(job))
_ <- onFinish(job, JobState.Cancelled)
_ <- ctx.logger.warn("Job has been cancelled.")
_ <- logger.fdebug(s"Job ${job.info} has been cancelled.")
} yield ()
}
}
def execute(job: RJob): F[Unit] = { def execute(job: RJob): F[Unit] = {
val task = for { val task = for {
jobtask <- jobtask <-

View File

@ -4,9 +4,11 @@ import cats.implicits._
import cats.effect._ import cats.effect._
import docspell.common.{Ident, LenientUri} import docspell.common.{Ident, LenientUri}
import docspell.common.syntax.all._ import docspell.common.syntax.all._
import docspell.joexapi.model.BasicResult
import org.http4s.{Method, Request, Uri} import org.http4s.{Method, Request, Uri}
import org.http4s.client.Client import org.http4s.client.Client
import org.http4s.client.blaze.BlazeClientBuilder import org.http4s.client.blaze.BlazeClientBuilder
import org.http4s.circe.CirceEntityDecoder._
import scala.concurrent.ExecutionContext import scala.concurrent.ExecutionContext
import org.log4s.getLogger import org.log4s.getLogger
@ -17,7 +19,7 @@ trait JoexClient[F[_]] {
def notifyJoexIgnoreErrors(base: LenientUri): F[Unit] def notifyJoexIgnoreErrors(base: LenientUri): F[Unit]
def cancelJob(base: LenientUri, job: Ident): F[Unit] def cancelJob(base: LenientUri, job: Ident): F[BasicResult]
} }
@ -44,10 +46,10 @@ object JoexClient {
() ()
} }
def cancelJob(base: LenientUri, job: Ident): F[Unit] = { def cancelJob(base: LenientUri, job: Ident): F[BasicResult] = {
val cancelUrl = base / "api" / "v1" / "job" / job.id / "cancel" val cancelUrl = base / "api" / "v1" / "job" / job.id / "cancel"
val req = Request[F](Method.POST, uri(cancelUrl)) val req = Request[F](Method.POST, uri(cancelUrl))
client.expect[String](req).map(_ => ()) client.expect[BasicResult](req)
} }
private def uri(u: LenientUri): Uri = private def uri(u: LenientUri): Uri =

View File

@ -36,7 +36,7 @@ that a job is some time waiting until it is picked up by a job
executor. You can always start more job executors to help out. executor. You can always start more job executors to help out.
If a job fails, it is retried after some time. Only if it fails too If a job fails, it is retried after some time. Only if it fails too
often (can be configured), it then is finished with *failed* state. If often (can be configured), it then is finished with *failed* state.
processing finally fails, the item is still created, just without
suggestions. But if processing is cancelled by the user, the item is For the document-processing task, if processing finally fails or a job
not created. is cancelled, the item is still created, just without suggestions.

View File

@ -454,19 +454,54 @@ object QItem {
n <- store.transact(RItem.deleteByIdAndCollective(itemId, collective)) n <- store.transact(RItem.deleteByIdAndCollective(itemId, collective))
} yield tn + rn + n + mn } yield tn + rn + n + mn
def findByFileIds(fileMetaIds: Seq[Ident]): ConnectionIO[Vector[RItem]] = { private def findByFileIdsQuery(fileMetaIds: NonEmptyList[Ident], limit: Option[Int]) = {
val IC = RItem.Columns val IC = RItem.Columns.all.map(_.prefix("i"))
val AC = RAttachment.Columns val aItem = RAttachment.Columns.itemId.prefix("a")
val q = val aId = RAttachment.Columns.id.prefix("a")
fr"SELECT DISTINCT" ++ commas( val aFileId = RAttachment.Columns.fileId.prefix("a")
IC.all.map(_.prefix("i").f) val iId = RItem.Columns.id.prefix("i")
) ++ fr"FROM" ++ RItem.table ++ fr"i" ++ val sId = RAttachmentSource.Columns.id.prefix("s")
fr"INNER JOIN" ++ RAttachment.table ++ fr"a ON" ++ AC.itemId val sFileId = RAttachmentSource.Columns.fileId.prefix("s")
.prefix("a") val rId = RAttachmentArchive.Columns.id.prefix("r")
.is(IC.id.prefix("i")) ++ val rFileId = RAttachmentArchive.Columns.fileId.prefix("r")
fr"WHERE" ++ AC.fileId.isOneOf(fileMetaIds) ++ orderBy(IC.created.prefix("i").asc) val m1Id = RFileMeta.Columns.id.prefix("m1")
val m2Id = RFileMeta.Columns.id.prefix("m2")
val m3Id = RFileMeta.Columns.id.prefix("m3")
q.query[RItem].to[Vector] val from =
RItem.table ++ fr"i INNER JOIN" ++ RAttachment.table ++ fr"a ON" ++ aItem.is(iId) ++
fr"INNER JOIN" ++ RAttachmentSource.table ++ fr"s ON" ++ aId.is(sId) ++
fr"INNER JOIN" ++ RFileMeta.table ++ fr"m1 ON" ++ m1Id.is(aFileId) ++
fr"INNER JOIN" ++ RFileMeta.table ++ fr"m2 ON" ++ m2Id.is(sFileId) ++
fr"LEFT OUTER JOIN" ++ RAttachmentArchive.table ++ fr"r ON" ++ aId.is(rId) ++
fr"LEFT OUTER JOIN" ++ RFileMeta.table ++ fr"m3 ON" ++ m3Id.is(rFileId)
val q = selectSimple(
IC,
from,
and(or(m1Id.isIn(fileMetaIds), m2Id.isIn(fileMetaIds), m3Id.isIn(fileMetaIds)))
)
limit match {
case Some(n) => q ++ fr"LIMIT $n"
case None => q
}
}
def findOneByFileIds(fileMetaIds: Seq[Ident]): ConnectionIO[Option[RItem]] =
NonEmptyList.fromList(fileMetaIds.toList) match {
case Some(nel) =>
findByFileIdsQuery(nel, Some(1)).query[RItem].option
case None =>
(None: Option[RItem]).pure[ConnectionIO]
}
def findByFileIds(fileMetaIds: Seq[Ident]): ConnectionIO[Vector[RItem]] =
NonEmptyList.fromList(fileMetaIds.toList) match {
case Some(nel) =>
findByFileIdsQuery(nel, None).query[RItem].to[Vector]
case None =>
Vector.empty[RItem].pure[ConnectionIO]
} }
def findByChecksum(checksum: String, collective: Ident): ConnectionIO[Vector[RItem]] = { def findByChecksum(checksum: String, collective: Ident): ConnectionIO[Vector[RItem]] = {

View File

@ -31,6 +31,12 @@ case class RJob(
def info: String = def info: String =
s"${id.id.substring(0, 9)}.../${group.id}/${task.id}/$priority" s"${id.id.substring(0, 9)}.../${group.id}/${task.id}/$priority"
def isFinalState: Boolean =
JobState.done.contains(state)
def isInProgress: Boolean =
JobState.inProgress.contains(state)
} }
object RJob { object RJob {
@ -121,6 +127,12 @@ object RJob {
def findByIdAndGroup(jobId: Ident, jobGroup: Ident): ConnectionIO[Option[RJob]] = def findByIdAndGroup(jobId: Ident, jobGroup: Ident): ConnectionIO[Option[RJob]] =
selectSimple(all, table, and(id.is(jobId), group.is(jobGroup))).query[RJob].option selectSimple(all, table, and(id.is(jobId), group.is(jobGroup))).query[RJob].option
def findById(jobId: Ident): ConnectionIO[Option[RJob]] =
selectSimple(all, table, id.is(jobId)).query[RJob].option
def findByIdAndWorker(jobId: Ident, workerId: Ident): ConnectionIO[Option[RJob]] =
selectSimple(all, table, and(id.is(jobId), worker.is(workerId))).query[RJob].option
def setRunningToWaiting(workerId: Ident): ConnectionIO[Int] = { def setRunningToWaiting(workerId: Ident): ConnectionIO[Int] = {
val states: Seq[JobState] = List(JobState.Running, JobState.Scheduled) val states: Seq[JobState] = List(JobState.Running, JobState.Scheduled)
updateRow( updateRow(