mirror of
https://github.com/TheAnachronism/docspell.git
synced 2025-06-22 02:18:26 +00:00
Create tasks to generate all previews
There is a task to generate preview images per attachment. It can either add them (if not present yet) or overwrite them (e.g. some config has changed). There is a task that selects all attachments without previews and submits a task to create it. This is submitted on start automatically to generate previews for all existing attachments.
This commit is contained in:
@ -18,6 +18,7 @@ import docspell.joex.learn.LearnClassifierTask
|
||||
import docspell.joex.notify._
|
||||
import docspell.joex.pdfconv.ConvertAllPdfTask
|
||||
import docspell.joex.pdfconv.PdfConvTask
|
||||
import docspell.joex.preview._
|
||||
import docspell.joex.process.ItemHandler
|
||||
import docspell.joex.process.ReProcessItem
|
||||
import docspell.joex.scanmailbox._
|
||||
@ -68,7 +69,10 @@ final class JoexAppImpl[F[_]: ConcurrentEffect: ContextShift: Timer](
|
||||
HouseKeepingTask
|
||||
.periodicTask[F](cfg.houseKeeping.schedule)
|
||||
.flatMap(pstore.insert) *>
|
||||
MigrationTask.job.flatMap(queue.insertIfNew)
|
||||
MigrationTask.job.flatMap(queue.insertIfNew) *>
|
||||
AllPreviewsTask
|
||||
.job(MakePreviewArgs.StoreMode.WhenMissing, None)
|
||||
.flatMap(queue.insertIfNew)
|
||||
}
|
||||
|
||||
object JoexAppImpl {
|
||||
@ -167,6 +171,20 @@ object JoexAppImpl {
|
||||
LearnClassifierTask.onCancel[F]
|
||||
)
|
||||
)
|
||||
.withTask(
|
||||
JobTask.json(
|
||||
MakePreviewArgs.taskName,
|
||||
MakePreviewTask[F](cfg.convert),
|
||||
MakePreviewTask.onCancel[F]
|
||||
)
|
||||
)
|
||||
.withTask(
|
||||
JobTask.json(
|
||||
AllPreviewsArgs.taskName,
|
||||
AllPreviewsTask[F](queue, joex),
|
||||
AllPreviewsTask.onCancel[F]
|
||||
)
|
||||
)
|
||||
.resource
|
||||
psch <- PeriodicScheduler.create(
|
||||
cfg.periodicScheduler,
|
||||
|
@ -0,0 +1,86 @@
|
||||
package docspell.joex.preview
|
||||
|
||||
import fs2.{Chunk, Stream}
|
||||
import docspell.common._
|
||||
import cats.effect._
|
||||
import cats.implicits._
|
||||
import docspell.store.queue.JobQueue
|
||||
import docspell.backend.ops.OJoex
|
||||
import docspell.joex.scheduler.Task
|
||||
import docspell.joex.scheduler.Context
|
||||
import docspell.store.records.RAttachment
|
||||
import docspell.store.records.RJob
|
||||
|
||||
object AllPreviewsTask {
|
||||
|
||||
type Args = AllPreviewsArgs
|
||||
|
||||
def apply[F[_]: Sync](queue: JobQueue[F], joex: OJoex[F]): Task[F, Args, Unit] =
|
||||
Task { ctx =>
|
||||
for {
|
||||
_ <- ctx.logger.info("Generating previews for attachments")
|
||||
n <- submitConversionJobs(ctx, queue)
|
||||
_ <- ctx.logger.info(s"Submitted $n jobs")
|
||||
_ <- joex.notifyAllNodes
|
||||
} yield ()
|
||||
}
|
||||
|
||||
def onCancel[F[_]: Sync]: Task[F, Args, Unit] =
|
||||
Task.log(_.warn("Cancelling all-previews task"))
|
||||
|
||||
def submitConversionJobs[F[_]: Sync](
|
||||
ctx: Context[F, Args],
|
||||
queue: JobQueue[F]
|
||||
): F[Int] =
|
||||
ctx.store
|
||||
.transact(RAttachment.findWithoutPreview(ctx.args.collective, 50))
|
||||
.chunks
|
||||
.flatMap(createJobs[F](ctx))
|
||||
.chunks
|
||||
.evalMap(jobs => queue.insertAllIfNew(jobs.toVector).map(_ => jobs.size))
|
||||
.evalTap(n => ctx.logger.debug(s"Submitted $n jobs …"))
|
||||
.compile
|
||||
.foldMonoid
|
||||
|
||||
private def createJobs[F[_]: Sync](
|
||||
ctx: Context[F, Args]
|
||||
)(ras: Chunk[RAttachment]): Stream[F, RJob] = {
|
||||
val collectiveOrSystem = ctx.args.collective.getOrElse(DocspellSystem.taskGroup)
|
||||
|
||||
def mkJob(ra: RAttachment): F[RJob] =
|
||||
for {
|
||||
id <- Ident.randomId[F]
|
||||
now <- Timestamp.current[F]
|
||||
} yield RJob.newJob(
|
||||
id,
|
||||
MakePreviewArgs.taskName,
|
||||
collectiveOrSystem,
|
||||
MakePreviewArgs(ra.id, ctx.args.storeMode),
|
||||
s"Create preview ${ra.id.id}/${ra.name.getOrElse("-")}",
|
||||
now,
|
||||
collectiveOrSystem,
|
||||
Priority.Low,
|
||||
Some(MakePreviewArgs.taskName / ra.id)
|
||||
)
|
||||
|
||||
val jobs = ras.traverse(mkJob)
|
||||
Stream.evalUnChunk(jobs)
|
||||
}
|
||||
|
||||
def job[F[_]: Sync](storeMode: MakePreviewArgs.StoreMode, cid: Option[Ident]): F[RJob] =
|
||||
for {
|
||||
id <- Ident.randomId[F]
|
||||
now <- Timestamp.current[F]
|
||||
} yield RJob.newJob(
|
||||
id,
|
||||
AllPreviewsArgs.taskName,
|
||||
cid.getOrElse(DocspellSystem.taskGroup),
|
||||
AllPreviewsArgs(cid, storeMode),
|
||||
"Create preview images",
|
||||
now,
|
||||
DocspellSystem.taskGroup,
|
||||
Priority.Low,
|
||||
Some(DocspellSystem.allPreviewTaskTracker)
|
||||
)
|
||||
|
||||
}
|
@ -0,0 +1,57 @@
|
||||
package docspell.joex.preview
|
||||
|
||||
import cats.implicits._
|
||||
import cats.effect._
|
||||
import docspell.common._
|
||||
import docspell.joex.scheduler.Task
|
||||
import docspell.store.records.RAttachmentPreview
|
||||
import docspell.joex.scheduler.Context
|
||||
import docspell.joex.process.AttachmentPreview
|
||||
import docspell.convert.ConvertConfig
|
||||
import docspell.extract.pdfbox.PdfboxPreview
|
||||
import docspell.store.records.RAttachment
|
||||
|
||||
object MakePreviewTask {
|
||||
|
||||
type Args = MakePreviewArgs
|
||||
|
||||
def apply[F[_]: Sync](cfg: ConvertConfig): Task[F, Args, Unit] =
|
||||
Task { ctx =>
|
||||
for {
|
||||
exists <- previewExists(ctx)
|
||||
preview <- PdfboxPreview(30)
|
||||
_ <-
|
||||
if (exists)
|
||||
ctx.logger.info(
|
||||
s"Preview already exists for attachment ${ctx.args.attachment}. Skipping."
|
||||
)
|
||||
else
|
||||
ctx.logger.info(
|
||||
s"Generating preview image for attachment ${ctx.args.attachment}"
|
||||
) *> generatePreview(ctx, preview, cfg)
|
||||
} yield ()
|
||||
}
|
||||
|
||||
def onCancel[F[_]: Sync]: Task[F, Args, Unit] =
|
||||
Task.log(_.warn("Cancelling make-preview task"))
|
||||
|
||||
private def generatePreview[F[_]: Sync](
|
||||
ctx: Context[F, Args],
|
||||
preview: PdfboxPreview[F],
|
||||
cfg: ConvertConfig
|
||||
): F[Unit] =
|
||||
for {
|
||||
ra <- ctx.store.transact(RAttachment.findById(ctx.args.attachment))
|
||||
_ <- ra
|
||||
.map(AttachmentPreview.createPreview(ctx, preview, cfg.chunkSize))
|
||||
.getOrElse(().pure[F])
|
||||
} yield ()
|
||||
|
||||
private def previewExists[F[_]: Sync](ctx: Context[F, Args]): F[Boolean] =
|
||||
if (ctx.args.store == MakePreviewArgs.StoreMode.WhenMissing)
|
||||
ctx.store.transact(
|
||||
RAttachmentPreview.findById(ctx.args.attachment).map(_.isDefined)
|
||||
)
|
||||
else
|
||||
false.pure[F]
|
||||
}
|
@ -15,6 +15,7 @@ import docspell.store.records._
|
||||
import docspell.store.syntax.MimeTypes._
|
||||
|
||||
import bitpeace.{Mimetype, MimetypeHint, RangeDef}
|
||||
import docspell.store.queries.QAttachment
|
||||
|
||||
/** Goes through all attachments that must be already converted into a
|
||||
* pdf. If it is a pdf, the first page is converted into a small
|
||||
@ -31,14 +32,14 @@ object AttachmentPreview {
|
||||
s"Creating preview images for ${item.attachments.size} files…"
|
||||
)
|
||||
preview <- PdfboxPreview(24)
|
||||
_ <- item.attachments.traverse(createPreview(ctx, preview, cfg))
|
||||
_ <- item.attachments.traverse(createPreview(ctx, preview, cfg.chunkSize))
|
||||
} yield item
|
||||
}
|
||||
|
||||
def createPreview[F[_]: Sync](
|
||||
ctx: Context[F, _],
|
||||
preview: PdfboxPreview[F],
|
||||
cfg: ConvertConfig
|
||||
chunkSize: Int
|
||||
)(
|
||||
ra: RAttachment
|
||||
): F[Option[RAttachmentPreview]] =
|
||||
@ -46,7 +47,7 @@ object AttachmentPreview {
|
||||
case MimeType.PdfMatch(_) =>
|
||||
preview.previewPNG(loadFile(ctx)(ra)).flatMap {
|
||||
case Some(out) =>
|
||||
createRecord(ctx, out, ra, cfg.chunkSize).map(_.some)
|
||||
createRecord(ctx, out, ra, chunkSize).map(_.some)
|
||||
case None =>
|
||||
(None: Option[RAttachmentPreview]).pure[F]
|
||||
}
|
||||
@ -55,7 +56,7 @@ object AttachmentPreview {
|
||||
(None: Option[RAttachmentPreview]).pure[F]
|
||||
}
|
||||
|
||||
def createRecord[F[_]: Sync](
|
||||
private def createRecord[F[_]: Sync](
|
||||
ctx: Context[F, _],
|
||||
png: Stream[F, Byte],
|
||||
ra: RAttachment,
|
||||
@ -75,6 +76,7 @@ object AttachmentPreview {
|
||||
.lastOrError
|
||||
now <- Timestamp.current[F]
|
||||
rp = RAttachmentPreview(ra.id, Ident.unsafe(fileMeta.id), name.map(_.fullName), now)
|
||||
_ <- QAttachment.deletePreview(ctx.store)(ra.id)
|
||||
_ <- ctx.store.transact(RAttachmentPreview.insert(rp))
|
||||
} yield rp
|
||||
}
|
||||
|
Reference in New Issue
Block a user