mirror of
https://github.com/TheAnachronism/docspell.git
synced 2025-06-22 02:18:26 +00:00
Adopt backend to collective-id
This commit is contained in:
@ -13,7 +13,7 @@ trait AttachedEvent[R] {
|
||||
|
||||
def value: R
|
||||
|
||||
def event(account: AccountId, baseUrl: Option[LenientUri]): Iterable[Event]
|
||||
def event(account: AccountInfo, baseUrl: Option[LenientUri]): Iterable[Event]
|
||||
|
||||
def map[U](f: R => U): AttachedEvent[U]
|
||||
}
|
||||
@ -24,7 +24,7 @@ object AttachedEvent {
|
||||
def only[R](v: R): AttachedEvent[R] =
|
||||
new AttachedEvent[R] {
|
||||
val value = v
|
||||
def event(account: AccountId, baseUrl: Option[LenientUri]): Iterable[Event] =
|
||||
def event(account: AccountInfo, baseUrl: Option[LenientUri]): Iterable[Event] =
|
||||
Iterable.empty[Event]
|
||||
|
||||
def map[U](f: R => U): AttachedEvent[U] =
|
||||
@ -33,10 +33,10 @@ object AttachedEvent {
|
||||
|
||||
def apply[R](
|
||||
v: R
|
||||
)(mkEvent: (AccountId, Option[LenientUri]) => Event): AttachedEvent[R] =
|
||||
)(mkEvent: (AccountInfo, Option[LenientUri]) => Event): AttachedEvent[R] =
|
||||
new AttachedEvent[R] {
|
||||
val value = v
|
||||
def event(account: AccountId, baseUrl: Option[LenientUri]): Iterable[Event] =
|
||||
def event(account: AccountInfo, baseUrl: Option[LenientUri]): Iterable[Event] =
|
||||
Some(mkEvent(account, baseUrl))
|
||||
|
||||
def map[U](f: R => U): AttachedEvent[U] =
|
||||
|
@ -9,12 +9,11 @@ package docspell.backend
|
||||
import cats.data.{NonEmptyList => Nel}
|
||||
import cats.effect.Sync
|
||||
import cats.syntax.all._
|
||||
|
||||
import docspell.backend.BackendCommands.EventContext
|
||||
import docspell.backend.ops.OCustomFields.SetValue
|
||||
import docspell.backend.ops._
|
||||
import docspell.common.bc._
|
||||
import docspell.common.{AccountId, Ident, LenientUri}
|
||||
import docspell.common._
|
||||
|
||||
private[backend] class BackendCommands[F[_]: Sync](
|
||||
itemOps: OItem[F],
|
||||
@ -25,14 +24,14 @@ private[backend] class BackendCommands[F[_]: Sync](
|
||||
) extends BackendCommandRunner[F, Unit] {
|
||||
private[this] val logger = docspell.logging.getLogger[F]
|
||||
|
||||
def run(collective: Ident, cmd: BackendCommand): F[Unit] =
|
||||
def run(collective: CollectiveId, cmd: BackendCommand): F[Unit] =
|
||||
doRun(collective, cmd).attempt.flatMap {
|
||||
case Right(_) => ().pure[F]
|
||||
case Left(ex) =>
|
||||
logger.error(ex)(s"Backend command $cmd failed for collective ${collective.id}.")
|
||||
logger.error(ex)(s"Backend command $cmd failed for collective $collective.")
|
||||
}
|
||||
|
||||
def doRun(collective: Ident, cmd: BackendCommand): F[Unit] =
|
||||
def doRun(collective: CollectiveId, cmd: BackendCommand): F[Unit] =
|
||||
cmd match {
|
||||
case BackendCommand.ItemUpdate(item, actions) =>
|
||||
actions.traverse_(a => runItemAction(collective, item, a))
|
||||
@ -41,38 +40,38 @@ private[backend] class BackendCommands[F[_]: Sync](
|
||||
actions.traverse_(a => runAttachAction(collective, item, attach, a))
|
||||
}
|
||||
|
||||
def runAll(collective: Ident, cmds: List[BackendCommand]): F[Unit] =
|
||||
def runAll(collective: CollectiveId, cmds: List[BackendCommand]): F[Unit] =
|
||||
cmds.traverse_(run(collective, _))
|
||||
|
||||
def runItemAction(collective: Ident, item: Ident, action: ItemAction): F[Unit] =
|
||||
def runItemAction(collective: CollectiveId, item: Ident, action: ItemAction): F[Unit] =
|
||||
action match {
|
||||
case ItemAction.AddTags(tags) =>
|
||||
logger.debug(s"Setting tags $tags on ${item.id} for ${collective.id}") *>
|
||||
logger.debug(s"Setting tags $tags on ${item.id} for ${collective.value}") *>
|
||||
itemOps
|
||||
.linkTags(item, tags.toList, collective)
|
||||
.flatMap(sendEvents)
|
||||
|
||||
case ItemAction.RemoveTags(tags) =>
|
||||
logger.debug(s"Remove tags $tags on ${item.id} for ${collective.id}") *>
|
||||
logger.debug(s"Remove tags $tags on ${item.id} for ${collective.value}") *>
|
||||
itemOps
|
||||
.removeTagsMultipleItems(Nel.of(item), tags.toList, collective)
|
||||
.flatMap(sendEvents)
|
||||
|
||||
case ItemAction.ReplaceTags(tags) =>
|
||||
logger.debug(s"Replace tags $tags on ${item.id} for ${collective.id}") *>
|
||||
logger.debug(s"Replace tags $tags on ${item.id} for $collective") *>
|
||||
itemOps
|
||||
.setTags(item, tags.toList, collective)
|
||||
.flatMap(sendEvents)
|
||||
|
||||
case ItemAction.SetFolder(folder) =>
|
||||
logger.debug(s"Set folder $folder on ${item.id} for ${collective.id}") *>
|
||||
logger.debug(s"Set folder $folder on ${item.id} for $collective") *>
|
||||
itemOps
|
||||
.setFolder(item, folder, collective)
|
||||
.void
|
||||
|
||||
case ItemAction.RemoveTagsCategory(cats) =>
|
||||
logger.debug(
|
||||
s"Remove tags in categories $cats on ${item.id} for ${collective.id}"
|
||||
s"Remove tags in categories $cats on ${item.id} for $collective"
|
||||
) *>
|
||||
itemOps
|
||||
.removeTagsOfCategories(item, collective, cats)
|
||||
@ -80,51 +79,51 @@ private[backend] class BackendCommands[F[_]: Sync](
|
||||
|
||||
case ItemAction.SetCorrOrg(id) =>
|
||||
logger.debug(
|
||||
s"Set correspondent organization ${id.map(_.id)} for ${collective.id}"
|
||||
s"Set correspondent organization ${id.map(_.id)} for $collective"
|
||||
) *>
|
||||
itemOps.setCorrOrg(Nel.of(item), id, collective).void
|
||||
|
||||
case ItemAction.SetCorrPerson(id) =>
|
||||
logger.debug(
|
||||
s"Set correspondent person ${id.map(_.id)} for ${collective.id}"
|
||||
s"Set correspondent person ${id.map(_.id)} for $collective"
|
||||
) *>
|
||||
itemOps.setCorrPerson(Nel.of(item), id, collective).void
|
||||
|
||||
case ItemAction.SetConcPerson(id) =>
|
||||
logger.debug(
|
||||
s"Set concerning person ${id.map(_.id)} for ${collective.id}"
|
||||
s"Set concerning person ${id.map(_.id)} for $collective"
|
||||
) *>
|
||||
itemOps.setConcPerson(Nel.of(item), id, collective).void
|
||||
|
||||
case ItemAction.SetConcEquipment(id) =>
|
||||
logger.debug(
|
||||
s"Set concerning equipment ${id.map(_.id)} for ${collective.id}"
|
||||
s"Set concerning equipment ${id.map(_.id)} for $collective"
|
||||
) *>
|
||||
itemOps.setConcEquip(Nel.of(item), id, collective).void
|
||||
|
||||
case ItemAction.SetField(field, value) =>
|
||||
logger.debug(
|
||||
s"Set field on item ${item.id} ${field.id} to '$value' for ${collective.id}"
|
||||
s"Set field on item ${item.id} ${field.id} to '$value' for $collective"
|
||||
) *>
|
||||
fieldOps
|
||||
.setValue(item, SetValue(field, value, collective))
|
||||
.flatMap(sendEvents)
|
||||
|
||||
case ItemAction.SetNotes(notes) =>
|
||||
logger.debug(s"Set notes on item ${item.id} for ${collective.id}") *>
|
||||
logger.debug(s"Set notes on item ${item.id} for $collective") *>
|
||||
itemOps.setNotes(item, notes, collective).void
|
||||
|
||||
case ItemAction.AddNotes(notes, sep) =>
|
||||
logger.debug(s"Add notes on item ${item.id} for ${collective.id}") *>
|
||||
logger.debug(s"Add notes on item ${item.id} for $collective") *>
|
||||
itemOps.addNotes(item, notes, sep, collective).void
|
||||
|
||||
case ItemAction.SetName(name) =>
|
||||
logger.debug(s"Set name '$name' on item ${item.id} for ${collective.id}") *>
|
||||
logger.debug(s"Set name '$name' on item ${item.id} for $collective") *>
|
||||
itemOps.setName(item, name, collective).void
|
||||
}
|
||||
|
||||
def runAttachAction(
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
itemId: Ident,
|
||||
attachId: Ident,
|
||||
action: AttachmentAction
|
||||
@ -150,7 +149,7 @@ private[backend] class BackendCommands[F[_]: Sync](
|
||||
object BackendCommands {
|
||||
|
||||
/** If supplied, notification events will be send. */
|
||||
case class EventContext(account: AccountId, baseUrl: Option[LenientUri])
|
||||
case class EventContext(account: AccountInfo, baseUrl: Option[LenientUri])
|
||||
|
||||
def fromBackend[F[_]: Sync](
|
||||
backendApp: BackendApp[F],
|
||||
|
@ -8,24 +8,23 @@ package docspell.backend
|
||||
|
||||
import cats.effect._
|
||||
import cats.implicits._
|
||||
|
||||
import docspell.backend.MailAddressCodec
|
||||
import docspell.backend.task.DownloadZipArgs
|
||||
import docspell.common._
|
||||
import docspell.notification.api.PeriodicQueryArgs
|
||||
import docspell.scheduler.Job
|
||||
import docspell.scheduler.usertask.UserTaskScope
|
||||
|
||||
object JobFactory extends MailAddressCodec {
|
||||
def existingItemAddon[F[_]: Sync](
|
||||
args: ItemAddonTaskArgs,
|
||||
submitter: AccountId
|
||||
submitter: UserTaskScope
|
||||
): F[Job[ItemAddonTaskArgs]] =
|
||||
Job.createNew(
|
||||
ItemAddonTaskArgs.taskName,
|
||||
submitter.collective,
|
||||
submitter,
|
||||
args,
|
||||
"Run addons on item",
|
||||
submitter.user,
|
||||
Priority.High,
|
||||
args.addonRunConfigs
|
||||
.map(_.take(23))
|
||||
@ -39,179 +38,167 @@ object JobFactory extends MailAddressCodec {
|
||||
def downloadZip[F[_]: Sync](
|
||||
args: DownloadZipArgs,
|
||||
summaryId: Ident,
|
||||
submitter: AccountId
|
||||
submitter: UserTaskScope
|
||||
): F[Job[DownloadZipArgs]] =
|
||||
Job.createNew(
|
||||
DownloadZipArgs.taskName,
|
||||
submitter.collective,
|
||||
submitter,
|
||||
args,
|
||||
s"Prepare zip file for query",
|
||||
submitter.user,
|
||||
Priority.High,
|
||||
Some(summaryId)
|
||||
)
|
||||
|
||||
def integrityCheck[F[_]: Sync](
|
||||
args: FileIntegrityCheckArgs,
|
||||
submitter: AccountId = DocspellSystem.account
|
||||
submitter: UserTaskScope = UserTaskScope.system
|
||||
): F[Job[FileIntegrityCheckArgs]] =
|
||||
Job.createNew(
|
||||
FileIntegrityCheckArgs.taskName,
|
||||
submitter.collective,
|
||||
submitter,
|
||||
args,
|
||||
s"Check integrity of files",
|
||||
submitter.user,
|
||||
Priority.Low,
|
||||
Some(FileIntegrityCheckArgs.taskName)
|
||||
)
|
||||
|
||||
def fileCopy[F[_]: Sync](
|
||||
args: FileCopyTaskArgs,
|
||||
submitter: AccountId = DocspellSystem.account
|
||||
submitter: UserTaskScope = UserTaskScope.system
|
||||
): F[Job[FileCopyTaskArgs]] =
|
||||
Job.createNew(
|
||||
FileCopyTaskArgs.taskName,
|
||||
submitter.collective,
|
||||
submitter,
|
||||
args,
|
||||
"Copying all files",
|
||||
submitter.user,
|
||||
Priority.High,
|
||||
Some(FileCopyTaskArgs.taskName)
|
||||
)
|
||||
|
||||
def periodicQuery[F[_]: Sync](
|
||||
args: PeriodicQueryArgs,
|
||||
submitter: AccountId
|
||||
submitter: UserTaskScope
|
||||
): F[Job[PeriodicQueryArgs]] =
|
||||
Job.createNew(
|
||||
PeriodicQueryArgs.taskName,
|
||||
submitter.collective,
|
||||
submitter,
|
||||
args,
|
||||
s"Running periodic query, notify via ${args.channels.map(_.channelType)}",
|
||||
submitter.user,
|
||||
Priority.Low,
|
||||
None
|
||||
)
|
||||
|
||||
def makePageCount[F[_]: Sync](
|
||||
args: MakePageCountArgs,
|
||||
account: Option[AccountId]
|
||||
submitter: UserTaskScope
|
||||
): F[Job[MakePageCountArgs]] =
|
||||
Job.createNew(
|
||||
MakePageCountArgs.taskName,
|
||||
account.map(_.collective).getOrElse(DocspellSystem.taskGroup),
|
||||
submitter,
|
||||
args,
|
||||
s"Find page-count metadata for ${args.attachment.id}",
|
||||
account.map(_.user).getOrElse(DocspellSystem.user),
|
||||
Priority.Low,
|
||||
Some(MakePageCountArgs.taskName / args.attachment)
|
||||
)
|
||||
|
||||
def makePreview[F[_]: Sync](
|
||||
args: MakePreviewArgs,
|
||||
account: Option[AccountId]
|
||||
submitter: UserTaskScope
|
||||
): F[Job[MakePreviewArgs]] =
|
||||
Job.createNew(
|
||||
MakePreviewArgs.taskName,
|
||||
account.map(_.collective).getOrElse(DocspellSystem.taskGroup),
|
||||
submitter,
|
||||
args,
|
||||
s"Generate preview image",
|
||||
account.map(_.user).getOrElse(DocspellSystem.user),
|
||||
Priority.Low,
|
||||
Some(MakePreviewArgs.taskName / args.attachment)
|
||||
)
|
||||
|
||||
def allPreviews[F[_]: Sync](
|
||||
args: AllPreviewsArgs,
|
||||
submitter: Option[Ident]
|
||||
submitter: UserTaskScope
|
||||
): F[Job[AllPreviewsArgs]] =
|
||||
Job.createNew(
|
||||
AllPreviewsArgs.taskName,
|
||||
args.collective.getOrElse(DocspellSystem.taskGroup),
|
||||
submitter,
|
||||
args,
|
||||
"Create preview images",
|
||||
submitter.getOrElse(DocspellSystem.user),
|
||||
Priority.Low,
|
||||
Some(DocspellSystem.allPreviewTaskTracker)
|
||||
)
|
||||
|
||||
def convertAllPdfs[F[_]: Sync](
|
||||
collective: Option[Ident],
|
||||
submitter: Option[Ident],
|
||||
args: ConvertAllPdfArgs,
|
||||
submitter: UserTaskScope,
|
||||
prio: Priority
|
||||
): F[Job[ConvertAllPdfArgs]] =
|
||||
Job.createNew(
|
||||
ConvertAllPdfArgs.taskName,
|
||||
collective.getOrElse(DocspellSystem.taskGroup),
|
||||
ConvertAllPdfArgs(collective),
|
||||
submitter,
|
||||
args,
|
||||
s"Convert all pdfs not yet converted",
|
||||
submitter.getOrElse(DocspellSystem.user),
|
||||
prio,
|
||||
collective
|
||||
.map(c => c / ConvertAllPdfArgs.taskName)
|
||||
args.collective
|
||||
.map(c => c.valueAsIdent / ConvertAllPdfArgs.taskName)
|
||||
.orElse(ConvertAllPdfArgs.taskName.some)
|
||||
)
|
||||
|
||||
def reprocessItem[F[_]: Sync](
|
||||
args: ReProcessItemArgs,
|
||||
account: AccountId,
|
||||
submitter: UserTaskScope,
|
||||
prio: Priority
|
||||
): F[Job[ReProcessItemArgs]] =
|
||||
Job.createNew(
|
||||
ReProcessItemArgs.taskName,
|
||||
account.collective,
|
||||
submitter,
|
||||
args,
|
||||
s"Re-process files of item ${args.itemId.id}",
|
||||
account.user,
|
||||
prio,
|
||||
Some(ReProcessItemArgs.taskName / args.itemId)
|
||||
)
|
||||
|
||||
def multiUpload[F[_]: Sync](
|
||||
args: ProcessItemArgs,
|
||||
account: AccountId,
|
||||
submitter: UserTaskScope,
|
||||
prio: Priority,
|
||||
tracker: Option[Ident]
|
||||
): F[Job[ProcessItemArgs]] =
|
||||
Job.createNew(
|
||||
ProcessItemArgs.multiUploadTaskName,
|
||||
account.collective,
|
||||
submitter,
|
||||
args,
|
||||
args.makeSubject,
|
||||
account.user,
|
||||
prio,
|
||||
tracker
|
||||
)
|
||||
|
||||
def processItem[F[_]: Sync](
|
||||
args: ProcessItemArgs,
|
||||
account: AccountId,
|
||||
submitter: UserTaskScope,
|
||||
prio: Priority,
|
||||
tracker: Option[Ident]
|
||||
): F[Job[ProcessItemArgs]] =
|
||||
Job.createNew(
|
||||
ProcessItemArgs.taskName,
|
||||
account.collective,
|
||||
submitter,
|
||||
args,
|
||||
args.makeSubject,
|
||||
account.user,
|
||||
prio,
|
||||
tracker
|
||||
)
|
||||
|
||||
def processItems[F[_]: Sync](
|
||||
args: List[ProcessItemArgs],
|
||||
account: AccountId,
|
||||
submitter: UserTaskScope,
|
||||
prio: Priority,
|
||||
tracker: Option[Ident]
|
||||
): F[List[Job[ProcessItemArgs]]] = {
|
||||
def create(arg: ProcessItemArgs): F[Job[ProcessItemArgs]] =
|
||||
Job.createNew(
|
||||
ProcessItemArgs.taskName,
|
||||
account.collective,
|
||||
submitter,
|
||||
arg,
|
||||
arg.makeSubject,
|
||||
account.user,
|
||||
prio,
|
||||
tracker
|
||||
)
|
||||
@ -222,22 +209,23 @@ object JobFactory extends MailAddressCodec {
|
||||
def reIndexAll[F[_]: Sync]: F[Job[ReIndexTaskArgs]] =
|
||||
Job.createNew(
|
||||
ReIndexTaskArgs.taskName,
|
||||
DocspellSystem.taskGroup,
|
||||
UserTaskScope.system,
|
||||
ReIndexTaskArgs(None),
|
||||
"Recreate full-text index",
|
||||
DocspellSystem.taskGroup,
|
||||
Priority.Low,
|
||||
Some(DocspellSystem.migrationTaskTracker)
|
||||
)
|
||||
|
||||
def reIndex[F[_]: Sync](account: AccountId): F[Job[ReIndexTaskArgs]] = {
|
||||
val args = ReIndexTaskArgs(Some(account.collective))
|
||||
def reIndex[F[_]: Sync](
|
||||
cid: CollectiveId,
|
||||
submitterUserId: Option[Ident]
|
||||
): F[Job[ReIndexTaskArgs]] = {
|
||||
val args = ReIndexTaskArgs(Some(cid))
|
||||
Job.createNew(
|
||||
ReIndexTaskArgs.taskName,
|
||||
account.collective,
|
||||
UserTaskScope(cid, submitterUserId),
|
||||
args,
|
||||
"Recreate full-text index",
|
||||
account.user,
|
||||
Priority.Low,
|
||||
Some(ReIndexTaskArgs.tracker(args))
|
||||
)
|
||||
|
@ -167,7 +167,7 @@ object Login {
|
||||
(for {
|
||||
_ <- validateToken
|
||||
key <- EitherT.fromOptionF(
|
||||
store.transact(RTotp.findEnabledByLogin(sf.token.account.userId, true)),
|
||||
store.transact(RTotp.findEnabledByUserId(sf.token.account.userId, true)),
|
||||
Result.invalidAuth
|
||||
)
|
||||
now <- EitherT.right[Result](Timestamp.current[F])
|
||||
|
@ -24,7 +24,7 @@ trait CreateIndex[F[_]] {
|
||||
*/
|
||||
def reIndexData(
|
||||
logger: Logger[F],
|
||||
collective: Option[Ident],
|
||||
collective: Option[CollectiveId],
|
||||
itemIds: Option[NonEmptyList[Ident]],
|
||||
chunkSize: Int
|
||||
): F[Unit]
|
||||
@ -40,7 +40,7 @@ object CreateIndex {
|
||||
new CreateIndex[F] {
|
||||
def reIndexData(
|
||||
logger: Logger[F],
|
||||
collective: Option[Ident],
|
||||
collective: Option[CollectiveId],
|
||||
itemIds: Option[NonEmptyList[Ident]],
|
||||
chunkSize: Int
|
||||
): F[Unit] = {
|
||||
|
@ -21,7 +21,7 @@ import docspell.store.queries.QCustomField.FieldValue
|
||||
import docspell.store.records._
|
||||
|
||||
trait Merge[F[_]] {
|
||||
def merge(items: NonEmptyList[Ident], collective: Ident): F[Merge.Result[RItem]]
|
||||
def merge(items: NonEmptyList[Ident], collective: CollectiveId): F[Merge.Result[RItem]]
|
||||
}
|
||||
|
||||
object Merge {
|
||||
@ -41,7 +41,10 @@ object Merge {
|
||||
createIndex: CreateIndex[F]
|
||||
): Merge[F] =
|
||||
new Merge[F] {
|
||||
def merge(givenIds: NonEmptyList[Ident], collective: Ident): F[Result[RItem]] =
|
||||
def merge(
|
||||
givenIds: NonEmptyList[Ident],
|
||||
collective: CollectiveId
|
||||
): F[Result[RItem]] =
|
||||
(for {
|
||||
items <- loadItems(givenIds, collective)
|
||||
ids = items.map(_.id)
|
||||
@ -65,7 +68,7 @@ object Merge {
|
||||
|
||||
def loadItems(
|
||||
items: NonEmptyList[Ident],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): EitherT[F, Error, NonEmptyList[RItem]] = {
|
||||
val loaded =
|
||||
store
|
||||
|
@ -26,7 +26,7 @@ import docspell.store.records.AddonRunConfigResolved
|
||||
trait AddonOps[F[_]] {
|
||||
|
||||
def execAll(
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
trigger: Set[AddonTriggerType],
|
||||
runConfigIds: Set[Ident],
|
||||
logger: Option[Logger[F]]
|
||||
@ -34,7 +34,7 @@ trait AddonOps[F[_]] {
|
||||
middleware: Middleware[F]
|
||||
): F[ExecResult]
|
||||
|
||||
def execById(collective: Ident, runConfigId: Ident, logger: Logger[F])(
|
||||
def execById(collective: CollectiveId, runConfigId: Ident, logger: Logger[F])(
|
||||
middleware: Middleware[F]
|
||||
): F[ExecResult]
|
||||
|
||||
@ -42,13 +42,16 @@ trait AddonOps[F[_]] {
|
||||
* filtered by given ids and triggers.
|
||||
*/
|
||||
def findAddonRefs(
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
trigger: Set[AddonTriggerType],
|
||||
runConfigIds: Set[Ident]
|
||||
): F[List[AddonRunConfigRef]]
|
||||
|
||||
/** Find enabled addon run config reference given an addon task id */
|
||||
def findAddonRef(collective: Ident, runConfigId: Ident): F[Option[AddonRunConfigRef]]
|
||||
def findAddonRef(
|
||||
collective: CollectiveId,
|
||||
runConfigId: Ident
|
||||
): F[Option[AddonRunConfigRef]]
|
||||
|
||||
/** Creates an executor for addons given a configuration. */
|
||||
def getExecutor(cfg: AddonExecutorConfig): F[AddonExecutor[F]]
|
||||
@ -58,7 +61,7 @@ trait AddonOps[F[_]] {
|
||||
object AddonOps {
|
||||
case class AddonRunConfigRef(
|
||||
id: Ident,
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
userId: Option[Ident],
|
||||
name: String,
|
||||
refs: List[AddonRef]
|
||||
@ -110,7 +113,7 @@ object AddonOps {
|
||||
private val prepare = new AddonPrepare[F](store)
|
||||
|
||||
def execAll(
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
trigger: Set[AddonTriggerType],
|
||||
runConfigIds: Set[Ident],
|
||||
logger: Option[Logger[F]]
|
||||
@ -125,7 +128,7 @@ object AddonOps {
|
||||
results <- runCfgs.traverse(r => execRunConfig(log, r, custom))
|
||||
} yield ExecResult(results.flatMap(_.result), runCfgs)
|
||||
|
||||
def execById(collective: Ident, runConfigId: Ident, logger: Logger[F])(
|
||||
def execById(collective: CollectiveId, runConfigId: Ident, logger: Logger[F])(
|
||||
custom: Middleware[F]
|
||||
): F[ExecResult] =
|
||||
(for {
|
||||
@ -167,7 +170,7 @@ object AddonOps {
|
||||
Async[F].pure(AddonExecutor(cfg, urlReader))
|
||||
|
||||
def findAddonRefs(
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
trigger: Set[AddonTriggerType],
|
||||
runConfigIds: Set[Ident]
|
||||
): F[List[AddonRunConfigRef]] =
|
||||
@ -183,7 +186,7 @@ object AddonOps {
|
||||
.map(_.map(AddonRunConfigRef.fromResolved))
|
||||
|
||||
def findAddonRef(
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
runConfigId: Ident
|
||||
): F[Option[AddonRunConfigRef]] =
|
||||
OptionT(
|
||||
|
@ -10,7 +10,6 @@ import cats.data.OptionT
|
||||
import cats.effect.kernel.Sync
|
||||
import cats.syntax.all._
|
||||
import fs2.io.file.{Files, Path}
|
||||
|
||||
import docspell.addons._
|
||||
import docspell.addons.out.{AddonOutput, ItemFile, NewItem}
|
||||
import docspell.backend.JobFactory
|
||||
@ -20,6 +19,7 @@ import docspell.common.bc.BackendCommandRunner
|
||||
import docspell.common.syntax.file._
|
||||
import docspell.logging.Logger
|
||||
import docspell.scheduler.JobStore
|
||||
import docspell.scheduler.usertask.UserTaskScope
|
||||
import docspell.store.Store
|
||||
import docspell.store.records._
|
||||
|
||||
@ -32,7 +32,7 @@ final private[joex] class AddonPostProcess[F[_]: Sync: Files](
|
||||
|
||||
def onResult(
|
||||
logger: Logger[F],
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
result: AddonExecutionResult,
|
||||
outputDir: Path
|
||||
): F[Unit] =
|
||||
@ -45,7 +45,7 @@ final private[joex] class AddonPostProcess[F[_]: Sync: Files](
|
||||
|
||||
def onSuccess(
|
||||
logger: Logger[F],
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
output: AddonOutput,
|
||||
outputDir: Path
|
||||
): F[Unit] =
|
||||
@ -60,7 +60,7 @@ final private[joex] class AddonPostProcess[F[_]: Sync: Files](
|
||||
|
||||
def submitNewItem(
|
||||
logger: Logger[F],
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
outputDir: Path
|
||||
)(newItem: NewItem): F[Unit] =
|
||||
for {
|
||||
@ -85,13 +85,17 @@ final private[joex] class AddonPostProcess[F[_]: Sync: Files](
|
||||
newItem.toProcessMeta(collective, collLang, "addon"),
|
||||
uploaded.map(f => ProcessItemArgs.File(f._1.some, f._2))
|
||||
)
|
||||
account = AccountId(collective, DocspellSystem.user)
|
||||
job <- JobFactory.processItem(args, account, Priority.High, None)
|
||||
job <- JobFactory.processItem(
|
||||
args,
|
||||
UserTaskScope.collective(collective),
|
||||
Priority.High,
|
||||
None
|
||||
)
|
||||
_ <- jobStore.insert(job.encode)
|
||||
_ <- logger.debug(s"Submitted job for processing: ${job.id}")
|
||||
} yield ()
|
||||
|
||||
def updateOne(logger: Logger[F], collective: Ident, outputDir: Path)(
|
||||
def updateOne(logger: Logger[F], collective: CollectiveId, outputDir: Path)(
|
||||
itemFile: ItemFile
|
||||
): F[Unit] =
|
||||
for {
|
||||
@ -123,7 +127,7 @@ final private[joex] class AddonPostProcess[F[_]: Sync: Files](
|
||||
|
||||
def submitNewFiles(
|
||||
logger: Logger[F],
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
outputDir: Path
|
||||
)(itemFile: ItemFile): F[Unit] =
|
||||
for {
|
||||
@ -131,7 +135,7 @@ final private[joex] class AddonPostProcess[F[_]: Sync: Files](
|
||||
collLang <- store.transact(RCollective.findLanguage(collective))
|
||||
newFiles <- itemFile.resolveNewFiles(logger, outputDir)
|
||||
byMeta = newFiles.groupBy(_._1.metadata).view.mapValues(_.map(_._2))
|
||||
account = AccountId(collective, DocspellSystem.user)
|
||||
submitter = UserTaskScope.collective(collective)
|
||||
_ <- byMeta.toList.traverse_ { case (meta, files) =>
|
||||
for {
|
||||
uploaded <- files.traverse(file =>
|
||||
@ -151,7 +155,7 @@ final private[joex] class AddonPostProcess[F[_]: Sync: Files](
|
||||
meta.toProcessMeta(collective, itemFile.itemId, collLang, "addon"),
|
||||
uploaded.map(f => ProcessItemArgs.File(f._1.some, f._2))
|
||||
)
|
||||
job <- JobFactory.processItem(args, account, Priority.High, None)
|
||||
job <- JobFactory.processItem(args, submitter, Priority.High, None)
|
||||
_ <- jobStore.insert(job.encode)
|
||||
_ <- logger.debug(s"Submitted job for processing: ${job.id}")
|
||||
} yield ()
|
||||
@ -168,19 +172,29 @@ final private[joex] class AddonPostProcess[F[_]: Sync: Files](
|
||||
.semiflatMap(run)
|
||||
.getOrElseF(logger.warn(s"Cannot find attachment for $key to update text!"))
|
||||
|
||||
private def setText(collective: Ident, ra: RAttachment, readText: F[String]): F[Unit] =
|
||||
private def setText(
|
||||
collective: CollectiveId,
|
||||
ra: RAttachment,
|
||||
readText: F[String]
|
||||
): F[Unit] =
|
||||
attachOps.setExtractedText(collective, ra.itemId, ra.id, readText)
|
||||
|
||||
private def replacePdf(
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
ra: RAttachment,
|
||||
file: Path,
|
||||
generatePreview: Boolean
|
||||
): F[Unit] =
|
||||
attachOps.addOrReplacePdf(collective, ra.id, file.readAll, generatePreview)
|
||||
attachOps.addOrReplacePdf(
|
||||
collective,
|
||||
ra.id,
|
||||
file.readAll,
|
||||
generatePreview,
|
||||
UserTaskScope.collective(collective)
|
||||
)
|
||||
|
||||
private def replacePreview(
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
attachId: Ident,
|
||||
imageData: Path
|
||||
): F[Unit] =
|
||||
|
@ -9,15 +9,14 @@ package docspell.backend.joex
|
||||
import cats.data.{Kleisli, OptionT}
|
||||
import cats.effect._
|
||||
import cats.syntax.all._
|
||||
|
||||
import docspell.addons.Middleware
|
||||
import docspell.backend.auth.AuthToken
|
||||
import docspell.backend.joex.AddonOps.AddonRunConfigRef
|
||||
import docspell.common._
|
||||
import docspell.logging.Logger
|
||||
import docspell.store.Store
|
||||
import docspell.store.records.{RNode, RUser}
|
||||
|
||||
import docspell.store.queries.QLogin
|
||||
import docspell.store.records.RNode
|
||||
import scodec.bits.ByteVector
|
||||
|
||||
private[joex] class AddonPrepare[F[_]: Sync](store: Store[F]) extends LoggerExtension {
|
||||
@ -46,8 +45,7 @@ private[joex] class AddonPrepare[F[_]: Sync](store: Store[F]) extends LoggerExte
|
||||
): F[Middleware[F]] =
|
||||
(for {
|
||||
userId <- OptionT.fromOption[F](runConfigRef.userId)
|
||||
user <- OptionT(store.transact(RUser.getIdByIdOrLogin(userId)))
|
||||
account = AccountId(runConfigRef.collective, user.login)
|
||||
account <- OptionT(store.transact(QLogin.findUser(userId))).map(_.account)
|
||||
env =
|
||||
Middleware.prepare[F](
|
||||
Kleisli(input => makeDscEnv(account, tokenValidity).map(input.addEnv))
|
||||
@ -58,7 +56,7 @@ private[joex] class AddonPrepare[F[_]: Sync](store: Store[F]) extends LoggerExte
|
||||
* Additionally a random rest-server is looked up from the database to set its url.
|
||||
*/
|
||||
def makeDscEnv(
|
||||
accountId: AccountId,
|
||||
account: AccountInfo,
|
||||
tokenValidity: Duration
|
||||
): F[Map[String, String]] =
|
||||
for {
|
||||
@ -71,7 +69,7 @@ private[joex] class AddonPrepare[F[_]: Sync](store: Store[F]) extends LoggerExte
|
||||
secret = serverNode.flatMap(_.serverSecret)
|
||||
|
||||
token <- AuthToken.user(
|
||||
accountId,
|
||||
account,
|
||||
false,
|
||||
secret.getOrElse(ByteVector.empty),
|
||||
tokenValidity.some
|
||||
|
@ -0,0 +1,17 @@
|
||||
package docspell.backend.joex
|
||||
|
||||
import docspell.common.AccountId
|
||||
import docspell.scheduler.FindJobOwner
|
||||
import docspell.store.Store
|
||||
import docspell.store.queries.QLogin
|
||||
|
||||
/** Finds the job submitter account by using the group as collective and submitter as
|
||||
* login.
|
||||
*/
|
||||
object FindJobOwnerAccount {
|
||||
def apply[F[_]](store: Store[F]): FindJobOwner[F] =
|
||||
FindJobOwner.of { job =>
|
||||
val accountId = AccountId(job.group, job.submitter)
|
||||
store.transact(QLogin.findAccount(accountId))
|
||||
}
|
||||
}
|
@ -9,16 +9,15 @@ package docspell.backend.ops
|
||||
import cats.data.NonEmptyList
|
||||
import cats.effect._
|
||||
import cats.syntax.all._
|
||||
|
||||
import docspell.backend.ops.AddonRunConfigError._
|
||||
import docspell.backend.ops.OAddons.{AddonRunConfigResult, AddonRunInsert}
|
||||
import docspell.common.Ident
|
||||
import docspell.common.CollectiveId
|
||||
import docspell.store.Store
|
||||
import docspell.store.records.RAddonArchive
|
||||
|
||||
object AddonRunConfigValidate {
|
||||
|
||||
def apply[F[_]: Sync](store: Store[F], cid: Ident)(
|
||||
def apply[F[_]: Sync](store: Store[F], cid: CollectiveId)(
|
||||
cfg: AddonRunInsert
|
||||
): F[AddonRunConfigResult[AddonRunInsert]] = {
|
||||
val init: AddonRunConfigResult[Unit] = ().asRight
|
||||
@ -31,7 +30,7 @@ object AddonRunConfigValidate {
|
||||
.map(_.as(cfg))
|
||||
}
|
||||
|
||||
def checkTriggers[F[_]: Sync](store: Store[F], cid: Ident)(
|
||||
def checkTriggers[F[_]: Sync](store: Store[F], cid: CollectiveId)(
|
||||
cfg: AddonRunInsert
|
||||
): F[AddonRunConfigResult[Unit]] =
|
||||
for {
|
||||
|
@ -11,12 +11,11 @@ import cats.effect._
|
||||
import cats.syntax.all._
|
||||
import fs2.Stream
|
||||
import fs2.io.file.Path
|
||||
|
||||
import docspell.addons.{AddonMeta, RunnerType}
|
||||
import docspell.backend.Config
|
||||
import docspell.backend.ops.AddonValidationError._
|
||||
import docspell.backend.ops.OAddons.AddonValidationResult
|
||||
import docspell.common.{Ident, LenientUri, UrlReader}
|
||||
import docspell.common.{CollectiveId, LenientUri, UrlReader}
|
||||
import docspell.joexapi.model.AddonSupport
|
||||
import docspell.store.Store
|
||||
import docspell.store.records.RAddonArchive
|
||||
@ -29,7 +28,7 @@ final class AddonValidate[F[_]: Async](
|
||||
private[this] val logger = docspell.logging.getLogger[F]
|
||||
|
||||
def fromUrl(
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
url: LenientUri,
|
||||
reader: UrlReader[F],
|
||||
localUrl: Option[LenientUri] = None,
|
||||
@ -47,7 +46,7 @@ final class AddonValidate[F[_]: Async](
|
||||
else archive(collective, reader(localUrl.getOrElse(url)).asRight, checkExisting)
|
||||
|
||||
def archive(
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
addonData: Either[Path, Stream[F, Byte]],
|
||||
checkExisting: Boolean = true
|
||||
): F[AddonValidationResult[AddonMeta]] =
|
||||
|
@ -30,45 +30,46 @@ trait OAddons[F[_]] {
|
||||
* exists.
|
||||
*/
|
||||
def registerAddon(
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
url: LenientUri,
|
||||
logger: Option[Logger[F]]
|
||||
): F[AddonValidationResult[(RAddonArchive, AddonMeta)]]
|
||||
|
||||
/** Refreshes an existing addon by downloading it again and updating metadata. */
|
||||
def refreshAddon(
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
addonId: Ident
|
||||
): F[AddonValidationResult[(RAddonArchive, AddonMeta)]]
|
||||
|
||||
/** Look into the addon at the given url and return its metadata. */
|
||||
def inspectAddon(
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
url: LenientUri
|
||||
): F[AddonValidationResult[AddonMeta]]
|
||||
|
||||
/** Deletes the addon if it exists. */
|
||||
def deleteAddon(collective: Ident, addonId: Ident): F[Boolean]
|
||||
def deleteAddon(collective: CollectiveId, addonId: Ident): F[Boolean]
|
||||
|
||||
def getAllAddons(collective: Ident): F[List[RAddonArchive]]
|
||||
def getAllAddons(collective: CollectiveId): F[List[RAddonArchive]]
|
||||
|
||||
/** Inserts or updates the addon run configuration. If it already exists (and the given
|
||||
* id is non empty), it will be completely replaced with the given one.
|
||||
*/
|
||||
def upsertAddonRunConfig(
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
runConfig: AddonRunInsert
|
||||
): F[AddonRunConfigResult[Ident]]
|
||||
|
||||
/** Deletes this task from the database. */
|
||||
def deleteAddonRunConfig(collective: Ident, runConfigId: Ident): F[Boolean]
|
||||
def deleteAddonRunConfig(collective: CollectiveId, runConfigId: Ident): F[Boolean]
|
||||
|
||||
def getAllAddonRunConfigs(collective: Ident): F[List[AddonRunInfo]]
|
||||
def getAllAddonRunConfigs(collective: CollectiveId): F[List[AddonRunInfo]]
|
||||
|
||||
def runAddonForItem(
|
||||
account: AccountId,
|
||||
cid: CollectiveId,
|
||||
itemIds: NonEmptyList[Ident],
|
||||
addonRunConfigIds: Set[Ident]
|
||||
addonRunConfigIds: Set[Ident],
|
||||
submitter: UserTaskScope
|
||||
): F[Unit]
|
||||
}
|
||||
|
||||
@ -141,7 +142,7 @@ object OAddons {
|
||||
private val zip = MimeType.zip.asString
|
||||
private val addonValidate = new AddonValidate[F](cfg, store, joex)
|
||||
|
||||
def getAllAddonRunConfigs(collective: Ident): F[List[AddonRunInfo]] =
|
||||
def getAllAddonRunConfigs(collective: CollectiveId): F[List[AddonRunInfo]] =
|
||||
for {
|
||||
all <- store.transact(AddonRunConfigData.findAll(collective))
|
||||
runConfigIDs = all.map(_.runConfig.id).toSet
|
||||
@ -168,7 +169,7 @@ object OAddons {
|
||||
} yield result
|
||||
|
||||
def upsertAddonRunConfig(
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
runConfig: AddonRunInsert
|
||||
): F[AddonRunConfigResult[Ident]] = {
|
||||
val insertDataRaw = AddonRunConfigData(
|
||||
@ -246,7 +247,10 @@ object OAddons {
|
||||
.value
|
||||
}
|
||||
|
||||
def deleteAddonRunConfig(collective: Ident, runConfigId: Ident): F[Boolean] = {
|
||||
def deleteAddonRunConfig(
|
||||
collective: CollectiveId,
|
||||
runConfigId: Ident
|
||||
): F[Boolean] = {
|
||||
val deleteRunConfig =
|
||||
(for {
|
||||
e <- OptionT(RAddonRunConfig.findById(collective, runConfigId))
|
||||
@ -264,20 +268,20 @@ object OAddons {
|
||||
} yield deleted
|
||||
}
|
||||
|
||||
def getAllAddons(collective: Ident): F[List[RAddonArchive]] =
|
||||
def getAllAddons(collective: CollectiveId): F[List[RAddonArchive]] =
|
||||
store.transact(RAddonArchive.listAll(collective))
|
||||
|
||||
def deleteAddon(collective: Ident, addonId: Ident): F[Boolean] =
|
||||
def deleteAddon(collective: CollectiveId, addonId: Ident): F[Boolean] =
|
||||
store.transact(RAddonArchive.deleteById(collective, addonId)).map(_ > 0)
|
||||
|
||||
def inspectAddon(
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
url: LenientUri
|
||||
): F[AddonValidationResult[AddonMeta]] =
|
||||
addonValidate.fromUrl(collective, url, urlReader, checkExisting = false)
|
||||
|
||||
def registerAddon(
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
url: LenientUri,
|
||||
logger: Option[Logger[F]]
|
||||
): F[AddonValidationResult[(RAddonArchive, AddonMeta)]] = {
|
||||
@ -294,7 +298,9 @@ object OAddons {
|
||||
.as(AddonValidationResult.failure[(RAddonArchive, AddonMeta)](error))
|
||||
}
|
||||
|
||||
log.info(s"Store addon file from '${url.asString} for ${collective.id}") *>
|
||||
log.info(
|
||||
s"Store addon file from '${url.asString} for collective ${collective.value}"
|
||||
) *>
|
||||
storeAddonFromUrl(collective, url).flatMapF { file =>
|
||||
val localUrl = FileUrlReader.url(file)
|
||||
for {
|
||||
@ -306,7 +312,7 @@ object OAddons {
|
||||
}
|
||||
|
||||
def refreshAddon(
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
addonId: Ident
|
||||
): F[AddonValidationResult[(RAddonArchive, AddonMeta)]] = {
|
||||
val findAddon = store
|
||||
@ -371,7 +377,7 @@ object OAddons {
|
||||
}
|
||||
|
||||
private def insertAddon(
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
url: LenientUri,
|
||||
meta: AddonMeta,
|
||||
file: FileKey
|
||||
@ -392,7 +398,7 @@ object OAddons {
|
||||
.onError(_ => store.fileRepo.delete(file))
|
||||
} yield record
|
||||
|
||||
private def storeAddonFromUrl(collective: Ident, url: LenientUri) =
|
||||
private def storeAddonFromUrl(collective: CollectiveId, url: LenientUri) =
|
||||
for {
|
||||
urlFile <- EitherT.pure(url.path.segments.lastOption)
|
||||
file <- EitherT(
|
||||
@ -412,15 +418,16 @@ object OAddons {
|
||||
} yield file
|
||||
|
||||
def runAddonForItem(
|
||||
account: AccountId,
|
||||
cid: CollectiveId,
|
||||
itemIds: NonEmptyList[Ident],
|
||||
addonRunConfigIds: Set[Ident]
|
||||
addonRunConfigIds: Set[Ident],
|
||||
submitter: UserTaskScope
|
||||
): F[Unit] =
|
||||
for {
|
||||
jobs <- itemIds.traverse(id =>
|
||||
JobFactory.existingItemAddon(
|
||||
ItemAddonTaskArgs(account.collective, id, addonRunConfigIds),
|
||||
account
|
||||
ItemAddonTaskArgs(cid, id, addonRunConfigIds),
|
||||
submitter
|
||||
)
|
||||
)
|
||||
_ <- jobStore.insertAllIfNew(jobs.map(_.encode).toList)
|
||||
|
@ -10,13 +10,13 @@ import cats.data.{NonEmptyList => Nel, OptionT}
|
||||
import cats.effect._
|
||||
import cats.syntax.all._
|
||||
import fs2.Stream
|
||||
|
||||
import docspell.backend.JobFactory
|
||||
import docspell.common.MakePreviewArgs.StoreMode
|
||||
import docspell.common._
|
||||
import docspell.files.TikaMimetype
|
||||
import docspell.ftsclient.{FtsClient, TextData}
|
||||
import docspell.scheduler.JobStore
|
||||
import docspell.scheduler.usertask.UserTaskScope
|
||||
import docspell.store.Store
|
||||
import docspell.store.queries.QAttachment
|
||||
import docspell.store.records._
|
||||
@ -24,21 +24,22 @@ import docspell.store.records._
|
||||
trait OAttachment[F[_]] {
|
||||
|
||||
def setExtractedText(
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
itemId: Ident,
|
||||
attachId: Ident,
|
||||
newText: F[String]
|
||||
): F[Unit]
|
||||
|
||||
def addOrReplacePdf(
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
attachId: Ident,
|
||||
pdfData: Stream[F, Byte],
|
||||
regeneratePreview: Boolean
|
||||
regeneratePreview: Boolean,
|
||||
submitter: UserTaskScope
|
||||
): F[Unit]
|
||||
|
||||
def addOrReplacePreview(
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
attachId: Ident,
|
||||
imageData: Stream[F, Byte]
|
||||
): F[Unit]
|
||||
@ -55,7 +56,7 @@ object OAttachment {
|
||||
private[this] val logger = docspell.logging.getLogger[F]
|
||||
|
||||
def setExtractedText(
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
itemId: Ident,
|
||||
attachId: Ident,
|
||||
newText: F[String]
|
||||
@ -104,24 +105,22 @@ object OAttachment {
|
||||
} yield ()
|
||||
|
||||
def addOrReplacePdf(
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
attachId: Ident,
|
||||
pdfData: Stream[F, Byte],
|
||||
regeneratePreview: Boolean
|
||||
regeneratePreview: Boolean,
|
||||
submitter: UserTaskScope
|
||||
): F[Unit] = {
|
||||
def generatePreview(ra: RAttachment): F[Unit] =
|
||||
JobFactory
|
||||
.makePreview(MakePreviewArgs(ra.id, StoreMode.Replace), None)
|
||||
.makePreview(MakePreviewArgs(ra.id, StoreMode.Replace), submitter)
|
||||
.map(_.encode)
|
||||
.flatMap(jobStore.insert) *>
|
||||
logger.info(s"Job submitted to re-generate preview from new pdf")
|
||||
|
||||
def generatePageCount(ra: RAttachment): F[Unit] =
|
||||
JobFactory
|
||||
.makePageCount(
|
||||
MakePageCountArgs(ra.id),
|
||||
AccountId(collective, DocspellSystem.user).some
|
||||
)
|
||||
.makePageCount(MakePageCountArgs(ra.id), submitter)
|
||||
.map(_.encode)
|
||||
.flatMap(jobStore.insert) *>
|
||||
logger.info(s"Job submitted to find page count from new pdf")
|
||||
@ -168,7 +167,7 @@ object OAttachment {
|
||||
}
|
||||
|
||||
def addOrReplacePreview(
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
attachId: Ident,
|
||||
imageData: Stream[F, Byte]
|
||||
): F[Unit] = {
|
||||
|
@ -11,29 +11,31 @@ import cats.data.OptionT
|
||||
import cats.effect.{Async, Resource}
|
||||
import cats.implicits._
|
||||
|
||||
import docspell.common.AccountId
|
||||
import docspell.common._
|
||||
import docspell.store.Store
|
||||
import docspell.store.records.RClientSettingsCollective
|
||||
import docspell.store.records.RClientSettingsUser
|
||||
import docspell.store.records.RUser
|
||||
|
||||
import io.circe.Json
|
||||
|
||||
trait OClientSettings[F[_]] {
|
||||
|
||||
def deleteUser(clientId: Ident, account: AccountId): F[Boolean]
|
||||
def saveUser(clientId: Ident, account: AccountId, data: Json): F[Unit]
|
||||
def loadUser(clientId: Ident, account: AccountId): F[Option[RClientSettingsUser]]
|
||||
def deleteUser(clientId: Ident, userId: Ident): F[Boolean]
|
||||
def saveUser(clientId: Ident, userId: Ident, data: Json): F[Unit]
|
||||
def loadUser(clientId: Ident, userId: Ident): F[Option[RClientSettingsUser]]
|
||||
|
||||
def deleteCollective(clientId: Ident, account: AccountId): F[Boolean]
|
||||
def saveCollective(clientId: Ident, account: AccountId, data: Json): F[Unit]
|
||||
def deleteCollective(clientId: Ident, collectiveId: CollectiveId): F[Boolean]
|
||||
def saveCollective(clientId: Ident, collectiveId: CollectiveId, data: Json): F[Unit]
|
||||
def loadCollective(
|
||||
clientId: Ident,
|
||||
account: AccountId
|
||||
collectiveId: CollectiveId
|
||||
): F[Option[RClientSettingsCollective]]
|
||||
|
||||
def loadMerged(clientId: Ident, account: AccountId): F[Option[Json]]
|
||||
def loadMerged(
|
||||
clientId: Ident,
|
||||
collectiveId: CollectiveId,
|
||||
userId: Ident
|
||||
): F[Option[Json]]
|
||||
}
|
||||
|
||||
object OClientSettings {
|
||||
@ -41,22 +43,18 @@ object OClientSettings {
|
||||
Resource.pure[F, OClientSettings[F]](new OClientSettings[F] {
|
||||
val log = docspell.logging.getLogger[F]
|
||||
|
||||
private def getUserId(account: AccountId): OptionT[F, Ident] =
|
||||
OptionT(store.transact(RUser.findByAccount(account))).map(_.uid)
|
||||
|
||||
def deleteCollective(clientId: Ident, account: AccountId): F[Boolean] =
|
||||
def deleteCollective(clientId: Ident, collectiveId: CollectiveId): F[Boolean] =
|
||||
store
|
||||
.transact(RClientSettingsCollective.delete(clientId, account.collective))
|
||||
.transact(RClientSettingsCollective.delete(clientId, collectiveId))
|
||||
.map(_ > 0)
|
||||
|
||||
def deleteUser(clientId: Ident, account: AccountId): F[Boolean] =
|
||||
def deleteUser(clientId: Ident, userId: Ident): F[Boolean] =
|
||||
(for {
|
||||
_ <- OptionT.liftF(
|
||||
log.debug(
|
||||
s"Deleting client settings for client ${clientId.id} and account $account"
|
||||
s"Deleting client settings for client ${clientId.id} and user ${userId.id}"
|
||||
)
|
||||
)
|
||||
userId <- getUserId(account)
|
||||
n <- OptionT.liftF(
|
||||
store.transact(
|
||||
RClientSettingsUser.delete(clientId, userId)
|
||||
@ -64,24 +62,27 @@ object OClientSettings {
|
||||
)
|
||||
} yield n > 0).getOrElse(false)
|
||||
|
||||
def saveCollective(clientId: Ident, account: AccountId, data: Json): F[Unit] =
|
||||
def saveCollective(
|
||||
clientId: Ident,
|
||||
collectiveId: CollectiveId,
|
||||
data: Json
|
||||
): F[Unit] =
|
||||
for {
|
||||
n <- store.transact(
|
||||
RClientSettingsCollective.upsert(clientId, account.collective, data)
|
||||
RClientSettingsCollective.upsert(clientId, collectiveId, data)
|
||||
)
|
||||
_ <-
|
||||
if (n <= 0) Async[F].raiseError(new IllegalStateException("No rows updated!"))
|
||||
else ().pure[F]
|
||||
} yield ()
|
||||
|
||||
def saveUser(clientId: Ident, account: AccountId, data: Json): F[Unit] =
|
||||
def saveUser(clientId: Ident, userId: Ident, data: Json): F[Unit] =
|
||||
(for {
|
||||
_ <- OptionT.liftF(
|
||||
log.debug(
|
||||
s"Storing client settings for client ${clientId.id} and account $account"
|
||||
s"Storing client settings for client ${clientId.id} and user ${userId.id}"
|
||||
)
|
||||
)
|
||||
userId <- getUserId(account)
|
||||
n <- OptionT.liftF(
|
||||
store.transact(RClientSettingsUser.upsert(clientId, userId, data))
|
||||
)
|
||||
@ -93,25 +94,24 @@ object OClientSettings {
|
||||
|
||||
def loadCollective(
|
||||
clientId: Ident,
|
||||
account: AccountId
|
||||
collectiveId: CollectiveId
|
||||
): F[Option[RClientSettingsCollective]] =
|
||||
store.transact(RClientSettingsCollective.find(clientId, account.collective))
|
||||
store.transact(RClientSettingsCollective.find(clientId, collectiveId))
|
||||
|
||||
def loadUser(clientId: Ident, account: AccountId): F[Option[RClientSettingsUser]] =
|
||||
def loadUser(clientId: Ident, userId: Ident): F[Option[RClientSettingsUser]] =
|
||||
(for {
|
||||
_ <- OptionT.liftF(
|
||||
log.debug(
|
||||
s"Loading client settings for client ${clientId.id} and account $account"
|
||||
s"Loading client settings for client ${clientId.id} and user ${userId.id}"
|
||||
)
|
||||
)
|
||||
userId <- getUserId(account)
|
||||
data <- OptionT(store.transact(RClientSettingsUser.find(clientId, userId)))
|
||||
} yield data).value
|
||||
|
||||
def loadMerged(clientId: Ident, account: AccountId) =
|
||||
def loadMerged(clientId: Ident, collectiveId: CollectiveId, userId: Ident) =
|
||||
for {
|
||||
collData <- loadCollective(clientId, account)
|
||||
userData <- loadUser(clientId, account)
|
||||
collData <- loadCollective(clientId, collectiveId)
|
||||
userData <- loadUser(clientId, userId)
|
||||
mergedData = collData.map(_.settingsData) |+| userData.map(_.settingsData)
|
||||
} yield mergedData
|
||||
|
||||
|
@ -6,10 +6,10 @@
|
||||
|
||||
package docspell.backend.ops
|
||||
|
||||
import cats.data.OptionT
|
||||
import cats.effect.{Async, Resource}
|
||||
import cats.implicits._
|
||||
import fs2.Stream
|
||||
|
||||
import docspell.backend.JobFactory
|
||||
import docspell.backend.PasswordCrypt
|
||||
import docspell.backend.ops.OCollective._
|
||||
@ -20,35 +20,39 @@ import docspell.store.UpdateResult
|
||||
import docspell.store.queries.{QCollective, QUser}
|
||||
import docspell.store.records._
|
||||
import docspell.store.{AddResult, Store}
|
||||
|
||||
import com.github.eikek.calev._
|
||||
|
||||
trait OCollective[F[_]] {
|
||||
|
||||
def find(name: Ident): F[Option[RCollective]]
|
||||
|
||||
def updateSettings(collective: Ident, settings: OCollective.Settings): F[AddResult]
|
||||
def updateSettings(
|
||||
collective: CollectiveId,
|
||||
collectiveName: Ident,
|
||||
settings: OCollective.Settings
|
||||
): F[AddResult]
|
||||
|
||||
def findSettings(collective: Ident): F[Option[OCollective.Settings]]
|
||||
def findSettings(collective: CollectiveId): F[Option[OCollective.Settings]]
|
||||
|
||||
def listUser(collective: Ident): F[Vector[RUser]]
|
||||
def listUser(collective: CollectiveId): F[Vector[RUser]]
|
||||
|
||||
def add(s: RUser): F[AddResult]
|
||||
|
||||
def update(s: RUser): F[AddResult]
|
||||
|
||||
/** Deletes the user and all its data. */
|
||||
def deleteUser(login: Ident, collective: Ident): F[UpdateResult]
|
||||
def deleteUser(userId: Ident): F[UpdateResult]
|
||||
|
||||
/** Return an excerpt of what would be deleted, when the user is deleted. */
|
||||
def getDeleteUserData(accountId: AccountId): F[DeleteUserData]
|
||||
def getDeleteUserData(cid: CollectiveId, userId: Ident): F[DeleteUserData]
|
||||
|
||||
def insights(collective: Ident): F[InsightData]
|
||||
def insights(collective: CollectiveId): F[InsightData]
|
||||
|
||||
def tagCloud(collective: Ident): F[List[TagCount]]
|
||||
def tagCloud(collective: CollectiveId): F[List[TagCount]]
|
||||
|
||||
def changePassword(
|
||||
accountId: AccountId,
|
||||
collectiveId: CollectiveId,
|
||||
userId: Ident,
|
||||
current: Password,
|
||||
newPass: Password
|
||||
): F[PassChangeResult]
|
||||
@ -56,20 +60,21 @@ trait OCollective[F[_]] {
|
||||
def resetPassword(accountId: AccountId): F[PassResetResult]
|
||||
|
||||
def getContacts(
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
query: Option[String],
|
||||
kind: Option[ContactKind]
|
||||
): Stream[F, RContact]
|
||||
|
||||
def findEnabledSource(sourceId: Ident): F[Option[RSource]]
|
||||
|
||||
def addPassword(collective: Ident, pw: Password): F[Unit]
|
||||
def addPassword(collective: CollectiveId, pw: Password): F[Unit]
|
||||
|
||||
def getPasswords(collective: Ident): F[List[RCollectivePassword]]
|
||||
def getPasswords(collective: CollectiveId): F[List[RCollectivePassword]]
|
||||
|
||||
/** Removes a password from the list given the id of `RCollectivePassword` */
|
||||
def removePassword(id: Ident): F[Unit]
|
||||
|
||||
def startLearnClassifier(collective: Ident): F[Unit]
|
||||
def startLearnClassifier(collective: CollectiveId): F[Unit]
|
||||
|
||||
def startEmptyTrash(args: EmptyTrashArgs): F[Unit]
|
||||
|
||||
@ -78,7 +83,8 @@ trait OCollective[F[_]] {
|
||||
*/
|
||||
def generatePreviews(
|
||||
storeMode: MakePreviewArgs.StoreMode,
|
||||
account: AccountId
|
||||
collectiveId: CollectiveId,
|
||||
submitter: UserTaskScope
|
||||
): F[UpdateResult]
|
||||
}
|
||||
|
||||
@ -137,26 +143,33 @@ object OCollective {
|
||||
): Resource[F, OCollective[F]] =
|
||||
Resource.pure[F, OCollective[F]](new OCollective[F] {
|
||||
def find(name: Ident): F[Option[RCollective]] =
|
||||
store.transact(RCollective.findById(name))
|
||||
store.transact(RCollective.findByName(name))
|
||||
|
||||
def updateSettings(collective: Ident, sett: Settings): F[AddResult] =
|
||||
def updateSettings(
|
||||
collectiveId: CollectiveId,
|
||||
collectiveName: Ident,
|
||||
sett: Settings
|
||||
): F[AddResult] =
|
||||
store
|
||||
.transact(RCollective.updateSettings(collective, sett))
|
||||
.transact(RCollective.updateSettings(collectiveId, sett))
|
||||
.attempt
|
||||
.map(AddResult.fromUpdate)
|
||||
.flatMap(res =>
|
||||
updateLearnClassifierTask(collective, sett) *> updateEmptyTrashTask(
|
||||
collective,
|
||||
updateLearnClassifierTask(collectiveId, sett) *> updateEmptyTrashTask(
|
||||
collectiveId,
|
||||
sett
|
||||
) *> res.pure[F]
|
||||
)
|
||||
|
||||
private def updateLearnClassifierTask(coll: Ident, sett: Settings): F[Unit] =
|
||||
private def updateLearnClassifierTask(
|
||||
cid: CollectiveId,
|
||||
sett: Settings
|
||||
): F[Unit] =
|
||||
for {
|
||||
id <- Ident.randomId[F]
|
||||
on = sett.classifier.exists(_.enabled)
|
||||
timer = sett.classifier.map(_.schedule).getOrElse(CalEvent.unsafe(""))
|
||||
args = LearnClassifierArgs(coll)
|
||||
args = LearnClassifierArgs(cid)
|
||||
ut = UserTask(
|
||||
id,
|
||||
LearnClassifierArgs.taskName,
|
||||
@ -165,36 +178,41 @@ object OCollective {
|
||||
None,
|
||||
args
|
||||
)
|
||||
_ <- uts.updateOneTask(UserTaskScope(coll), args.makeSubject.some, ut)
|
||||
_ <- uts.updateOneTask(UserTaskScope.collective(cid), args.makeSubject.some, ut)
|
||||
_ <- joex.notifyAllNodes
|
||||
} yield ()
|
||||
|
||||
private def updateEmptyTrashTask(coll: Ident, sett: Settings): F[Unit] =
|
||||
private def updateEmptyTrashTask(
|
||||
cid: CollectiveId,
|
||||
sett: Settings
|
||||
): F[Unit] =
|
||||
for {
|
||||
id <- Ident.randomId[F]
|
||||
settings = sett.emptyTrash.getOrElse(EmptyTrash.default)
|
||||
args = EmptyTrashArgs(coll, settings.minAge)
|
||||
args = EmptyTrashArgs(cid, settings.minAge)
|
||||
ut = UserTask(id, EmptyTrashArgs.taskName, true, settings.schedule, None, args)
|
||||
_ <- uts.updateOneTask(UserTaskScope(coll), args.makeSubject.some, ut)
|
||||
_ <- uts.updateOneTask(UserTaskScope.collective(cid), args.makeSubject.some, ut)
|
||||
_ <- joex.notifyAllNodes
|
||||
} yield ()
|
||||
|
||||
def addPassword(collective: Ident, pw: Password): F[Unit] =
|
||||
def addPassword(collective: CollectiveId, pw: Password): F[Unit] =
|
||||
for {
|
||||
cpass <- RCollectivePassword.createNew[F](collective, pw)
|
||||
_ <- store.transact(RCollectivePassword.upsert(cpass))
|
||||
} yield ()
|
||||
|
||||
def getPasswords(collective: Ident): F[List[RCollectivePassword]] =
|
||||
def getPasswords(collective: CollectiveId): F[List[RCollectivePassword]] =
|
||||
store.transact(RCollectivePassword.findAll(collective))
|
||||
|
||||
def removePassword(id: Ident): F[Unit] =
|
||||
store.transact(RCollectivePassword.deleteById(id)).map(_ => ())
|
||||
|
||||
def startLearnClassifier(collective: Ident): F[Unit] =
|
||||
def startLearnClassifier(
|
||||
collectiveId: CollectiveId
|
||||
): F[Unit] =
|
||||
for {
|
||||
id <- Ident.randomId[F]
|
||||
args = LearnClassifierArgs(collective)
|
||||
args = LearnClassifierArgs(collectiveId)
|
||||
ut = UserTask(
|
||||
id,
|
||||
LearnClassifierArgs.taskName,
|
||||
@ -204,7 +222,11 @@ object OCollective {
|
||||
args
|
||||
)
|
||||
_ <- uts
|
||||
.executeNow(UserTaskScope(collective), args.makeSubject.some, ut)
|
||||
.executeNow(
|
||||
UserTaskScope.collective(args.collectiveId),
|
||||
args.makeSubject.some,
|
||||
ut
|
||||
)
|
||||
} yield ()
|
||||
|
||||
def startEmptyTrash(args: EmptyTrashArgs): F[Unit] =
|
||||
@ -219,13 +241,17 @@ object OCollective {
|
||||
args
|
||||
)
|
||||
_ <- uts
|
||||
.executeNow(UserTaskScope(args.collective), args.makeSubject.some, ut)
|
||||
.executeNow(
|
||||
UserTaskScope.collective(args.collective),
|
||||
args.makeSubject.some,
|
||||
ut
|
||||
)
|
||||
} yield ()
|
||||
|
||||
def findSettings(collective: Ident): F[Option[OCollective.Settings]] =
|
||||
def findSettings(collective: CollectiveId): F[Option[OCollective.Settings]] =
|
||||
store.transact(RCollective.getSettings(collective))
|
||||
|
||||
def listUser(collective: Ident): F[Vector[RUser]] =
|
||||
def listUser(collective: CollectiveId): F[Vector[RUser]] =
|
||||
store.transact(RUser.findAll(collective, _.login))
|
||||
|
||||
def add(s: RUser): F[AddResult] =
|
||||
@ -240,47 +266,48 @@ object OCollective {
|
||||
def update(s: RUser): F[AddResult] =
|
||||
store.add(RUser.update(s), RUser.exists(s.login))
|
||||
|
||||
def getDeleteUserData(accountId: AccountId): F[DeleteUserData] =
|
||||
store.transact(QUser.getUserData(accountId))
|
||||
def getDeleteUserData(cid: CollectiveId, userId: Ident): F[DeleteUserData] =
|
||||
store.transact(QUser.getUserData(cid, userId))
|
||||
|
||||
def deleteUser(login: Ident, collective: Ident): F[UpdateResult] =
|
||||
def deleteUser(userId: Ident): F[UpdateResult] =
|
||||
UpdateResult.fromUpdate(
|
||||
store.transact(QUser.deleteUserAndData(AccountId(collective, login)))
|
||||
store.transact(QUser.deleteUserAndData(userId))
|
||||
)
|
||||
|
||||
def insights(collective: Ident): F[InsightData] =
|
||||
def insights(collective: CollectiveId): F[InsightData] =
|
||||
store.transact(QCollective.getInsights(collective))
|
||||
|
||||
def tagCloud(collective: Ident): F[List[TagCount]] =
|
||||
def tagCloud(collective: CollectiveId): F[List[TagCount]] =
|
||||
store.transact(QCollective.tagCloud(collective))
|
||||
|
||||
def resetPassword(accountId: AccountId): F[PassResetResult] =
|
||||
for {
|
||||
newPass <- Password.generate[F]
|
||||
optUser <- store.transact(RUser.findByAccount(accountId))
|
||||
n <- store.transact(
|
||||
RUser.updatePassword(accountId, PasswordCrypt.crypt(newPass))
|
||||
(for {
|
||||
user <- OptionT(store.transact(RUser.findByAccount(accountId)))
|
||||
newPass <- OptionT.liftF(Password.generate[F])
|
||||
|
||||
doUpdate = store.transact(
|
||||
RUser.updatePassword(user.cid, user.uid, PasswordCrypt.crypt(newPass))
|
||||
)
|
||||
res =
|
||||
if (optUser.exists(_.source != AccountSource.Local))
|
||||
PassResetResult.userNotLocal
|
||||
else if (n <= 0) PassResetResult.notFound
|
||||
else PassResetResult.success(newPass)
|
||||
} yield res
|
||||
res <-
|
||||
if (user.source != AccountSource.Local)
|
||||
OptionT.pure[F](PassResetResult.userNotLocal)
|
||||
else OptionT.liftF(doUpdate.as(PassResetResult.success(newPass)))
|
||||
} yield res).getOrElse(PassResetResult.notFound)
|
||||
|
||||
def changePassword(
|
||||
accountId: AccountId,
|
||||
collectiveId: CollectiveId,
|
||||
userId: Ident,
|
||||
current: Password,
|
||||
newPass: Password
|
||||
): F[PassChangeResult] = {
|
||||
val q = for {
|
||||
optUser <- RUser.findByAccount(accountId)
|
||||
optUser <- RUser.findById(userId, collectiveId.some)
|
||||
check = optUser.map(_.password).map(p => PasswordCrypt.check(current, p))
|
||||
n <-
|
||||
check
|
||||
.filter(identity)
|
||||
.traverse(_ =>
|
||||
RUser.updatePassword(accountId, PasswordCrypt.crypt(newPass))
|
||||
RUser.updatePassword(collectiveId, userId, PasswordCrypt.crypt(newPass))
|
||||
)
|
||||
res = check match {
|
||||
case Some(true) =>
|
||||
@ -299,7 +326,7 @@ object OCollective {
|
||||
}
|
||||
|
||||
def getContacts(
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
query: Option[String],
|
||||
kind: Option[ContactKind]
|
||||
): Stream[F, RContact] =
|
||||
@ -310,12 +337,13 @@ object OCollective {
|
||||
|
||||
def generatePreviews(
|
||||
storeMode: MakePreviewArgs.StoreMode,
|
||||
account: AccountId
|
||||
collectiveId: CollectiveId,
|
||||
submitter: UserTaskScope
|
||||
): F[UpdateResult] =
|
||||
for {
|
||||
job <- JobFactory.allPreviews[F](
|
||||
AllPreviewsArgs(Some(account.collective), storeMode),
|
||||
Some(account.user)
|
||||
AllPreviewsArgs(Some(collectiveId), storeMode),
|
||||
submitter
|
||||
)
|
||||
_ <- jobStore.insertIfNew(job.encode)
|
||||
} yield UpdateResult.success
|
||||
|
@ -36,13 +36,13 @@ trait OCustomFields[F[_]] {
|
||||
|
||||
/** Find all fields using an optional query on the name and label */
|
||||
def findAll(
|
||||
coll: Ident,
|
||||
coll: CollectiveId,
|
||||
nameQuery: Option[String],
|
||||
order: CustomFieldOrder
|
||||
): F[Vector[CustomFieldData]]
|
||||
|
||||
/** Find one field by its id */
|
||||
def findById(coll: Ident, fieldId: Ident): F[Option[CustomFieldData]]
|
||||
def findById(coll: CollectiveId, fieldId: Ident): F[Option[CustomFieldData]]
|
||||
|
||||
/** Create a new non-existing field. */
|
||||
def create(field: NewCustomField): F[AddResult]
|
||||
@ -51,7 +51,7 @@ trait OCustomFields[F[_]] {
|
||||
def change(field: RCustomField): F[UpdateResult]
|
||||
|
||||
/** Deletes the field by name or id. */
|
||||
def delete(coll: Ident, fieldIdOrName: Ident): F[UpdateResult]
|
||||
def delete(coll: CollectiveId, fieldIdOrName: Ident): F[UpdateResult]
|
||||
|
||||
/** Sets a value given a field an an item. Existing values are overwritten. */
|
||||
def setValue(item: Ident, value: SetValue): F[AttachedEvent[SetValueResult]]
|
||||
@ -80,13 +80,13 @@ object OCustomFields {
|
||||
name: Ident,
|
||||
label: Option[String],
|
||||
ftype: CustomFieldType,
|
||||
cid: Ident
|
||||
cid: CollectiveId
|
||||
)
|
||||
|
||||
case class SetValue(
|
||||
field: Ident,
|
||||
value: String,
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
)
|
||||
|
||||
sealed trait SetValueResult
|
||||
@ -106,7 +106,7 @@ object OCustomFields {
|
||||
case class RemoveValue(
|
||||
field: Ident,
|
||||
item: Nel[Ident],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
)
|
||||
|
||||
sealed trait CustomFieldOrder
|
||||
@ -158,7 +158,7 @@ object OCustomFields {
|
||||
store.transact(QCustomField.findAllValues(itemIds))
|
||||
|
||||
def findAll(
|
||||
coll: Ident,
|
||||
coll: CollectiveId,
|
||||
nameQuery: Option[String],
|
||||
order: CustomFieldOrder
|
||||
): F[Vector[CustomFieldData]] =
|
||||
@ -170,7 +170,7 @@ object OCustomFields {
|
||||
)
|
||||
)
|
||||
|
||||
def findById(coll: Ident, field: Ident): F[Option[CustomFieldData]] =
|
||||
def findById(coll: CollectiveId, field: Ident): F[Option[CustomFieldData]] =
|
||||
store.transact(QCustomField.findById(field, coll))
|
||||
|
||||
def create(field: NewCustomField): F[AddResult] = {
|
||||
@ -188,7 +188,7 @@ object OCustomFields {
|
||||
def change(field: RCustomField): F[UpdateResult] =
|
||||
UpdateResult.fromUpdate(store.transact(RCustomField.update(field)))
|
||||
|
||||
def delete(coll: Ident, fieldIdOrName: Ident): F[UpdateResult] = {
|
||||
def delete(coll: CollectiveId, fieldIdOrName: Ident): F[UpdateResult] = {
|
||||
val update =
|
||||
for {
|
||||
field <- OptionT(RCustomField.findByIdOrName(fieldIdOrName, coll))
|
||||
|
@ -7,12 +7,10 @@
|
||||
package docspell.backend.ops
|
||||
|
||||
import java.security.MessageDigest
|
||||
|
||||
import cats.data.OptionT
|
||||
import cats.effect._
|
||||
import cats.syntax.all._
|
||||
import fs2.{Pipe, Stream}
|
||||
|
||||
import docspell.backend.JobFactory
|
||||
import docspell.backend.ops.ODownloadAll.model._
|
||||
import docspell.backend.ops.OJob.JobCancelResult
|
||||
@ -21,11 +19,11 @@ import docspell.common._
|
||||
import docspell.query.ItemQuery.Expr.ValidItemStates
|
||||
import docspell.query.{ItemQuery, ItemQueryParser}
|
||||
import docspell.scheduler.JobStore
|
||||
import docspell.scheduler.usertask.UserTaskScope
|
||||
import docspell.store.Store
|
||||
import docspell.store.file.FileMetadata
|
||||
import docspell.store.queries.{QItem, Query}
|
||||
import docspell.store.records.{RDownloadQuery, RFileMeta, RJob}
|
||||
|
||||
import io.circe.generic.semiauto.{deriveDecoder, deriveEncoder}
|
||||
import io.circe.{Decoder, Encoder}
|
||||
import scodec.bits.ByteVector
|
||||
@ -34,12 +32,12 @@ trait ODownloadAll[F[_]] {
|
||||
|
||||
/** Calculates what kind of zip file would be created and checks the server thresholds.
|
||||
*/
|
||||
def getSummary(account: AccountId, req: DownloadRequest): F[DownloadSummary]
|
||||
def getSummary(account: AccountInfo, req: DownloadRequest): F[DownloadSummary]
|
||||
|
||||
/** Same as `getSummary` but also submits the job to really create the zip file if
|
||||
* allowed and necessary.
|
||||
*/
|
||||
def submit(accountId: AccountId, req: DownloadRequest): F[DownloadSummary]
|
||||
def submit(accountId: AccountInfo, req: DownloadRequest): F[DownloadSummary]
|
||||
|
||||
/** Given the id from the summary, cancels a running job. */
|
||||
def cancelDownload(accountId: AccountId, id: Ident): F[OJob.JobCancelResult]
|
||||
@ -65,7 +63,7 @@ object ODownloadAll {
|
||||
private[this] val logger = docspell.logging.getLogger[F]
|
||||
|
||||
def getSummary(
|
||||
account: AccountId,
|
||||
account: AccountInfo,
|
||||
req: DownloadRequest
|
||||
): F[DownloadSummary] = {
|
||||
val query = req.toQuery(account)
|
||||
@ -83,16 +81,16 @@ object ODownloadAll {
|
||||
}
|
||||
|
||||
def submit(
|
||||
accountId: AccountId,
|
||||
account: AccountInfo,
|
||||
req: DownloadRequest
|
||||
): F[DownloadSummary] = for {
|
||||
_ <- logger.info(s"Download all request: $req")
|
||||
summary <- getSummary(accountId, req)
|
||||
args = DownloadZipArgs(accountId, req)
|
||||
summary <- getSummary(account, req)
|
||||
args = DownloadZipArgs(account.asAccountId, req)
|
||||
_ <- OptionT
|
||||
.whenF(summary.state == DownloadState.NotPresent) {
|
||||
JobFactory
|
||||
.downloadZip(args, summary.id, accountId)
|
||||
.downloadZip(args, summary.id, UserTaskScope(account))
|
||||
.flatMap(job =>
|
||||
logger.info(s"Submitting download all job: $job") *> jobStore
|
||||
.insertIfNew(job.encode)
|
||||
@ -173,9 +171,9 @@ object ODownloadAll {
|
||||
maxFiles: Int,
|
||||
maxSize: ByteSize
|
||||
) {
|
||||
def toQuery(accountId: AccountId): Query =
|
||||
def toQuery(account: AccountInfo): Query =
|
||||
Query
|
||||
.all(accountId)
|
||||
.all(account)
|
||||
.withFix(_.andQuery(ValidItemStates))
|
||||
.withCond(_ => Query.QueryExpr(query.expr))
|
||||
|
||||
|
@ -9,26 +9,25 @@ package docspell.backend.ops
|
||||
import cats.data.NonEmptyList
|
||||
import cats.effect.{Async, Resource}
|
||||
import cats.implicits._
|
||||
|
||||
import docspell.common.{AccountId, Ident}
|
||||
import docspell.common._
|
||||
import docspell.store.records.{REquipment, RItem}
|
||||
import docspell.store.{AddResult, Store}
|
||||
|
||||
trait OEquipment[F[_]] {
|
||||
|
||||
def findAll(
|
||||
account: AccountId,
|
||||
collectiveId: CollectiveId,
|
||||
nameQuery: Option[String],
|
||||
order: OEquipment.EquipmentOrder
|
||||
): F[Vector[REquipment]]
|
||||
|
||||
def find(account: AccountId, id: Ident): F[Option[REquipment]]
|
||||
def find(collectiveId: CollectiveId, id: Ident): F[Option[REquipment]]
|
||||
|
||||
def add(s: REquipment): F[AddResult]
|
||||
|
||||
def update(s: REquipment): F[AddResult]
|
||||
|
||||
def delete(id: Ident, collective: Ident): F[AddResult]
|
||||
def delete(id: Ident, collective: CollectiveId): F[AddResult]
|
||||
}
|
||||
|
||||
object OEquipment {
|
||||
@ -58,16 +57,16 @@ object OEquipment {
|
||||
def apply[F[_]: Async](store: Store[F]): Resource[F, OEquipment[F]] =
|
||||
Resource.pure[F, OEquipment[F]](new OEquipment[F] {
|
||||
def findAll(
|
||||
account: AccountId,
|
||||
collective: CollectiveId,
|
||||
nameQuery: Option[String],
|
||||
order: EquipmentOrder
|
||||
): F[Vector[REquipment]] =
|
||||
store.transact(
|
||||
REquipment.findAll(account.collective, nameQuery, EquipmentOrder(order))
|
||||
REquipment.findAll(collective, nameQuery, EquipmentOrder(order))
|
||||
)
|
||||
|
||||
def find(account: AccountId, id: Ident): F[Option[REquipment]] =
|
||||
store.transact(REquipment.findById(id)).map(_.filter(_.cid == account.collective))
|
||||
def find(cid: CollectiveId, id: Ident): F[Option[REquipment]] =
|
||||
store.transact(REquipment.findById(id)).map(_.filter(_.cid == cid))
|
||||
|
||||
def add(e: REquipment): F[AddResult] = {
|
||||
def insert = REquipment.insert(e)
|
||||
@ -85,7 +84,7 @@ object OEquipment {
|
||||
store.add(insert, exists).map(_.fold(identity, _.withMsg(msg), identity))
|
||||
}
|
||||
|
||||
def delete(id: Ident, collective: Ident): F[AddResult] = {
|
||||
def delete(id: Ident, collective: CollectiveId): F[AddResult] = {
|
||||
val io = for {
|
||||
n0 <- RItem.removeConcEquip(collective, id)
|
||||
n1 <- REquipment.delete(id, collective)
|
||||
|
@ -7,8 +7,8 @@
|
||||
package docspell.backend.ops
|
||||
|
||||
import cats.data.{NonEmptyList => Nel}
|
||||
import cats.syntax.option._
|
||||
import cats.effect._
|
||||
|
||||
import docspell.common._
|
||||
import docspell.store.queries.QFolder
|
||||
import docspell.store.records.{RFolder, RUser}
|
||||
@ -17,38 +17,43 @@ import docspell.store.{AddResult, Store}
|
||||
trait OFolder[F[_]] {
|
||||
|
||||
def findAll(
|
||||
account: AccountId,
|
||||
collectiveId: CollectiveId,
|
||||
userId: Ident,
|
||||
ownerLogin: Option[Ident],
|
||||
query: Option[String],
|
||||
order: OFolder.FolderOrder
|
||||
): F[Vector[OFolder.FolderItem]]
|
||||
|
||||
def findById(id: Ident, account: AccountId): F[Option[OFolder.FolderDetail]]
|
||||
def findById(
|
||||
id: Ident,
|
||||
collectiveId: CollectiveId,
|
||||
userId: Ident
|
||||
): F[Option[OFolder.FolderDetail]]
|
||||
|
||||
/** Adds a new folder. If `login` is non-empty, the `folder.user` property is ignored
|
||||
* and the user-id is determined by the given login name.
|
||||
*/
|
||||
def add(folder: RFolder, login: Option[Ident]): F[AddResult]
|
||||
def add(folder: RFolder, userId: Option[Ident]): F[AddResult]
|
||||
|
||||
def changeName(
|
||||
folder: Ident,
|
||||
account: AccountId,
|
||||
userId: Ident,
|
||||
name: String
|
||||
): F[OFolder.FolderChangeResult]
|
||||
|
||||
def addMember(
|
||||
folder: Ident,
|
||||
account: AccountId,
|
||||
userId: Ident,
|
||||
member: Ident
|
||||
): F[OFolder.FolderChangeResult]
|
||||
|
||||
def removeMember(
|
||||
folder: Ident,
|
||||
account: AccountId,
|
||||
userId: Ident,
|
||||
member: Ident
|
||||
): F[OFolder.FolderChangeResult]
|
||||
|
||||
def delete(id: Ident, account: AccountId): F[OFolder.FolderChangeResult]
|
||||
def delete(id: Ident, userId: Ident): F[OFolder.FolderChangeResult]
|
||||
}
|
||||
|
||||
object OFolder {
|
||||
@ -94,23 +99,35 @@ object OFolder {
|
||||
def apply[F[_]](store: Store[F]): Resource[F, OFolder[F]] =
|
||||
Resource.pure[F, OFolder[F]](new OFolder[F] {
|
||||
def findAll(
|
||||
account: AccountId,
|
||||
collectiveId: CollectiveId,
|
||||
userId: Ident,
|
||||
ownerLogin: Option[Ident],
|
||||
query: Option[String],
|
||||
order: FolderOrder
|
||||
): F[Vector[FolderItem]] =
|
||||
store.transact(
|
||||
QFolder.findAll(account, None, ownerLogin, query, FolderOrder(order))
|
||||
QFolder.findAll(
|
||||
collectiveId,
|
||||
userId,
|
||||
None,
|
||||
ownerLogin,
|
||||
query,
|
||||
FolderOrder(order)
|
||||
)
|
||||
)
|
||||
|
||||
def findById(id: Ident, account: AccountId): F[Option[FolderDetail]] =
|
||||
store.transact(QFolder.findById(id, account))
|
||||
def findById(
|
||||
id: Ident,
|
||||
collectiveId: CollectiveId,
|
||||
userId: Ident
|
||||
): F[Option[FolderDetail]] =
|
||||
store.transact(QFolder.findById(id, collectiveId, userId))
|
||||
|
||||
def add(folder: RFolder, login: Option[Ident]): F[AddResult] = {
|
||||
val insert = login match {
|
||||
case Some(n) =>
|
||||
def add(folder: RFolder, userId: Option[Ident]): F[AddResult] = {
|
||||
val insert = userId match {
|
||||
case Some(uid) =>
|
||||
for {
|
||||
user <- RUser.findByAccount(AccountId(folder.collectiveId, n))
|
||||
user <- RUser.findById(uid, folder.collectiveId.some)
|
||||
s = user.map(u => folder.copy(owner = u.uid)).getOrElse(folder)
|
||||
n <- RFolder.insert(s)
|
||||
} yield n
|
||||
@ -124,26 +141,26 @@ object OFolder {
|
||||
|
||||
def changeName(
|
||||
folder: Ident,
|
||||
account: AccountId,
|
||||
userId: Ident,
|
||||
name: String
|
||||
): F[FolderChangeResult] =
|
||||
store.transact(QFolder.changeName(folder, account, name))
|
||||
store.transact(QFolder.changeName(folder, userId, name))
|
||||
|
||||
def addMember(
|
||||
folder: Ident,
|
||||
account: AccountId,
|
||||
userId: Ident,
|
||||
member: Ident
|
||||
): F[FolderChangeResult] =
|
||||
store.transact(QFolder.addMember(folder, account, member))
|
||||
store.transact(QFolder.addMember(folder, userId, member))
|
||||
|
||||
def removeMember(
|
||||
folder: Ident,
|
||||
account: AccountId,
|
||||
userId: Ident,
|
||||
member: Ident
|
||||
): F[FolderChangeResult] =
|
||||
store.transact(QFolder.removeMember(folder, account, member))
|
||||
store.transact(QFolder.removeMember(folder, userId, member))
|
||||
|
||||
def delete(id: Ident, account: AccountId): F[FolderChangeResult] =
|
||||
store.transact(QFolder.delete(id, account))
|
||||
def delete(id: Ident, userId: Ident): F[FolderChangeResult] =
|
||||
store.transact(QFolder.delete(id, userId))
|
||||
})
|
||||
}
|
||||
|
@ -23,7 +23,7 @@ trait OFulltext[F[_]] {
|
||||
/** Clears the full-text index for the given collective and starts a task indexing all
|
||||
* their data.
|
||||
*/
|
||||
def reindexCollective(account: AccountId): F[Unit]
|
||||
def reindexCollective(cid: CollectiveId, submitterUserId: Option[Ident]): F[Unit]
|
||||
}
|
||||
|
||||
object OFulltext {
|
||||
@ -40,13 +40,13 @@ object OFulltext {
|
||||
_ <- jobStore.insertIfNew(job.encode)
|
||||
} yield ()
|
||||
|
||||
def reindexCollective(account: AccountId): F[Unit] =
|
||||
def reindexCollective(cid: CollectiveId, submitterUserId: Option[Ident]): F[Unit] =
|
||||
for {
|
||||
_ <- logger.debug(s"Re-index collective: $account")
|
||||
_ <- logger.debug(s"Re-index collective: $cid")
|
||||
exist <- store.transact(
|
||||
RJob.findNonFinalByTracker(DocspellSystem.migrationTaskTracker)
|
||||
)
|
||||
job <- JobFactory.reIndex(account)
|
||||
job <- JobFactory.reIndex(cid, submitterUserId)
|
||||
_ <-
|
||||
if (exist.isDefined) ().pure[F]
|
||||
else jobStore.insertIfNew(job.encode)
|
||||
|
@ -9,7 +9,6 @@ package docspell.backend.ops
|
||||
import cats.data.{NonEmptyList => Nel, OptionT}
|
||||
import cats.effect.{Async, Resource}
|
||||
import cats.implicits._
|
||||
|
||||
import docspell.backend.AttachedEvent
|
||||
import docspell.backend.JobFactory
|
||||
import docspell.backend.fulltext.CreateIndex
|
||||
@ -19,10 +18,10 @@ import docspell.ftsclient.FtsClient
|
||||
import docspell.logging.Logger
|
||||
import docspell.notification.api.Event
|
||||
import docspell.scheduler.JobStore
|
||||
import docspell.scheduler.usertask.UserTaskScope
|
||||
import docspell.store.queries.{QAttachment, QItem, QMoveAttachment}
|
||||
import docspell.store.records._
|
||||
import docspell.store.{AddResult, Store, UpdateResult}
|
||||
|
||||
import doobie.implicits._
|
||||
|
||||
trait OItem[F[_]] {
|
||||
@ -31,7 +30,7 @@ trait OItem[F[_]] {
|
||||
def setTags(
|
||||
item: Ident,
|
||||
tagIds: List[String],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[AttachedEvent[UpdateResult]]
|
||||
|
||||
/** Sets tags for multiple items. The tags of the items will be replaced with the given
|
||||
@ -40,11 +39,15 @@ trait OItem[F[_]] {
|
||||
def setTagsMultipleItems(
|
||||
items: Nel[Ident],
|
||||
tags: List[String],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[AttachedEvent[UpdateResult]]
|
||||
|
||||
/** Create a new tag and add it to the item. */
|
||||
def addNewTag(collective: Ident, item: Ident, tag: RTag): F[AttachedEvent[AddResult]]
|
||||
def addNewTag(
|
||||
collective: CollectiveId,
|
||||
item: Ident,
|
||||
tag: RTag
|
||||
): F[AttachedEvent[AddResult]]
|
||||
|
||||
/** Apply all tags to the given item. Tags must exist, but can be IDs or names. Existing
|
||||
* tags on the item are left unchanged.
|
||||
@ -52,54 +55,58 @@ trait OItem[F[_]] {
|
||||
def linkTags(
|
||||
item: Ident,
|
||||
tags: List[String],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[AttachedEvent[UpdateResult]]
|
||||
|
||||
def linkTagsMultipleItems(
|
||||
items: Nel[Ident],
|
||||
tags: List[String],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[AttachedEvent[UpdateResult]]
|
||||
|
||||
def removeTagsOfCategories(
|
||||
item: Ident,
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
categories: Set[String]
|
||||
): F[AttachedEvent[UpdateResult]]
|
||||
|
||||
def removeTagsMultipleItems(
|
||||
items: Nel[Ident],
|
||||
tags: List[String],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[AttachedEvent[UpdateResult]]
|
||||
|
||||
/** Toggles tags of the given item. Tags must exist, but can be IDs or names. */
|
||||
def toggleTags(
|
||||
item: Ident,
|
||||
tags: List[String],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[AttachedEvent[UpdateResult]]
|
||||
|
||||
def setDirection(
|
||||
item: Nel[Ident],
|
||||
direction: Direction,
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[UpdateResult]
|
||||
|
||||
/** Set or remove the folder on an item. Folder can be the id or name. */
|
||||
def setFolder(item: Ident, folder: Option[String], collective: Ident): F[UpdateResult]
|
||||
def setFolder(
|
||||
item: Ident,
|
||||
folder: Option[String],
|
||||
collective: CollectiveId
|
||||
): F[UpdateResult]
|
||||
|
||||
/** Set or remove the folder on multiple items. Folder can be the id or name. */
|
||||
def setFolderMultiple(
|
||||
items: Nel[Ident],
|
||||
folder: Option[String],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[UpdateResult]
|
||||
|
||||
def setCorrOrg(
|
||||
items: Nel[Ident],
|
||||
org: Option[Ident],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[UpdateResult]
|
||||
|
||||
def addCorrOrg(item: Ident, org: OOrganization.OrgAndContacts): F[AddResult]
|
||||
@ -107,7 +114,7 @@ trait OItem[F[_]] {
|
||||
def setCorrPerson(
|
||||
items: Nel[Ident],
|
||||
person: Option[Ident],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[UpdateResult]
|
||||
|
||||
def addCorrPerson(item: Ident, person: OOrganization.PersonAndContacts): F[AddResult]
|
||||
@ -115,7 +122,7 @@ trait OItem[F[_]] {
|
||||
def setConcPerson(
|
||||
items: Nel[Ident],
|
||||
person: Option[Ident],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[UpdateResult]
|
||||
|
||||
def addConcPerson(item: Ident, person: OOrganization.PersonAndContacts): F[AddResult]
|
||||
@ -123,64 +130,68 @@ trait OItem[F[_]] {
|
||||
def setConcEquip(
|
||||
items: Nel[Ident],
|
||||
equip: Option[Ident],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[UpdateResult]
|
||||
|
||||
def addConcEquip(item: Ident, equip: REquipment): F[AddResult]
|
||||
|
||||
def setNotes(item: Ident, notes: Option[String], collective: Ident): F[UpdateResult]
|
||||
def setNotes(
|
||||
item: Ident,
|
||||
notes: Option[String],
|
||||
collective: CollectiveId
|
||||
): F[UpdateResult]
|
||||
|
||||
def addNotes(
|
||||
item: Ident,
|
||||
notes: String,
|
||||
separator: Option[String],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[UpdateResult]
|
||||
|
||||
def setName(item: Ident, name: String, collective: Ident): F[UpdateResult]
|
||||
def setName(item: Ident, name: String, collective: CollectiveId): F[UpdateResult]
|
||||
|
||||
def setNameMultiple(
|
||||
items: Nel[Ident],
|
||||
name: String,
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[UpdateResult]
|
||||
|
||||
def setState(item: Ident, state: ItemState, collective: Ident): F[AddResult] =
|
||||
def setState(item: Ident, state: ItemState, collective: CollectiveId): F[AddResult] =
|
||||
setStates(Nel.of(item), state, collective)
|
||||
|
||||
def setStates(
|
||||
item: Nel[Ident],
|
||||
state: ItemState,
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[AddResult]
|
||||
|
||||
def restore(items: Nel[Ident], collective: Ident): F[UpdateResult]
|
||||
def restore(items: Nel[Ident], collective: CollectiveId): F[UpdateResult]
|
||||
|
||||
def setItemDate(
|
||||
item: Nel[Ident],
|
||||
date: Option[Timestamp],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[UpdateResult]
|
||||
|
||||
def setItemDueDate(
|
||||
item: Nel[Ident],
|
||||
date: Option[Timestamp],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[UpdateResult]
|
||||
|
||||
def getProposals(item: Ident, collective: Ident): F[MetaProposalList]
|
||||
def getProposals(item: Ident, collective: CollectiveId): F[MetaProposalList]
|
||||
|
||||
def deleteItem(itemId: Ident, collective: Ident): F[Int]
|
||||
def deleteItem(itemId: Ident, collective: CollectiveId): F[Int]
|
||||
|
||||
def deleteItemMultiple(items: Nel[Ident], collective: Ident): F[Int]
|
||||
def deleteItemMultiple(items: Nel[Ident], collective: CollectiveId): F[Int]
|
||||
|
||||
def deleteAttachment(id: Ident, collective: Ident): F[Int]
|
||||
def deleteAttachment(id: Ident, collective: CollectiveId): F[Int]
|
||||
|
||||
def setDeletedState(items: Nel[Ident], collective: Ident): F[Int]
|
||||
def setDeletedState(items: Nel[Ident], collective: CollectiveId): F[Int]
|
||||
|
||||
def deleteAttachmentMultiple(
|
||||
attachments: Nel[Ident],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[Int]
|
||||
|
||||
def moveAttachmentBefore(itemId: Ident, source: Ident, target: Ident): F[AddResult]
|
||||
@ -188,7 +199,7 @@ trait OItem[F[_]] {
|
||||
def setAttachmentName(
|
||||
attachId: Ident,
|
||||
name: Option[String],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[UpdateResult]
|
||||
|
||||
/** Submits the item for re-processing. The list of attachment ids can be used to only
|
||||
@ -196,28 +207,30 @@ trait OItem[F[_]] {
|
||||
* attachments are reprocessed. This call only submits the job into the queue.
|
||||
*/
|
||||
def reprocess(
|
||||
cid: CollectiveId,
|
||||
item: Ident,
|
||||
attachments: List[Ident],
|
||||
account: AccountId
|
||||
submitter: UserTaskScope
|
||||
): F[UpdateResult]
|
||||
|
||||
def reprocessAll(
|
||||
cid: CollectiveId,
|
||||
items: Nel[Ident],
|
||||
account: AccountId
|
||||
submitter: UserTaskScope
|
||||
): F[UpdateResult]
|
||||
|
||||
/** Submits a task that finds all non-converted pdfs and triggers converting them using
|
||||
* ocrmypdf. Each file is converted by a separate task.
|
||||
*/
|
||||
def convertAllPdf(
|
||||
collective: Option[Ident],
|
||||
submitter: Option[Ident]
|
||||
collective: Option[CollectiveId],
|
||||
submitter: UserTaskScope
|
||||
): F[UpdateResult]
|
||||
|
||||
/** Submits a task that (re)generates the preview image for an attachment. */
|
||||
def generatePreview(
|
||||
args: MakePreviewArgs,
|
||||
account: AccountId
|
||||
account: UserTaskScope
|
||||
): F[UpdateResult]
|
||||
|
||||
/** Submits a task that (re)generates the preview images for all attachments. */
|
||||
@ -227,7 +240,7 @@ trait OItem[F[_]] {
|
||||
def merge(
|
||||
logger: Logger[F],
|
||||
items: Nel[Ident],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[UpdateResult]
|
||||
}
|
||||
|
||||
@ -248,7 +261,7 @@ object OItem {
|
||||
def merge(
|
||||
logger: Logger[F],
|
||||
items: Nel[Ident],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[UpdateResult] =
|
||||
Merge(logger, store, this, createIndex).merge(items, collective).attempt.map {
|
||||
case Right(Right(_)) => UpdateResult.success
|
||||
@ -269,14 +282,14 @@ object OItem {
|
||||
def linkTags(
|
||||
item: Ident,
|
||||
tags: List[String],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[AttachedEvent[UpdateResult]] =
|
||||
linkTagsMultipleItems(Nel.of(item), tags, collective)
|
||||
|
||||
def linkTagsMultipleItems(
|
||||
items: Nel[Ident],
|
||||
tags: List[String],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[AttachedEvent[UpdateResult]] =
|
||||
tags.distinct match {
|
||||
case Nil => AttachedEvent.only(UpdateResult.success).pure[F]
|
||||
@ -305,7 +318,7 @@ object OItem {
|
||||
|
||||
def removeTagsOfCategories(
|
||||
item: Ident,
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
categories: Set[String]
|
||||
): F[AttachedEvent[UpdateResult]] =
|
||||
if (categories.isEmpty) {
|
||||
@ -328,7 +341,7 @@ object OItem {
|
||||
def removeTagsMultipleItems(
|
||||
items: Nel[Ident],
|
||||
tags: List[String],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[AttachedEvent[UpdateResult]] =
|
||||
tags.distinct match {
|
||||
case Nil => AttachedEvent.only(UpdateResult.success).pure[F]
|
||||
@ -354,7 +367,7 @@ object OItem {
|
||||
def toggleTags(
|
||||
item: Ident,
|
||||
tags: List[String],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[AttachedEvent[UpdateResult]] =
|
||||
tags.distinct match {
|
||||
case Nil => AttachedEvent.only(UpdateResult.success).pure[F]
|
||||
@ -383,14 +396,14 @@ object OItem {
|
||||
def setTags(
|
||||
item: Ident,
|
||||
tagIds: List[String],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[AttachedEvent[UpdateResult]] =
|
||||
setTagsMultipleItems(Nel.of(item), tagIds, collective)
|
||||
|
||||
def setTagsMultipleItems(
|
||||
items: Nel[Ident],
|
||||
tags: List[String],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[AttachedEvent[UpdateResult]] = {
|
||||
val dbTask =
|
||||
for {
|
||||
@ -411,7 +424,7 @@ object OItem {
|
||||
}
|
||||
|
||||
def addNewTag(
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
item: Ident,
|
||||
tag: RTag
|
||||
): F[AttachedEvent[AddResult]] =
|
||||
@ -448,7 +461,7 @@ object OItem {
|
||||
def setDirection(
|
||||
items: Nel[Ident],
|
||||
direction: Direction,
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[UpdateResult] =
|
||||
UpdateResult.fromUpdate(
|
||||
store
|
||||
@ -458,7 +471,7 @@ object OItem {
|
||||
def setFolder(
|
||||
item: Ident,
|
||||
folder: Option[String],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[UpdateResult] =
|
||||
for {
|
||||
result <- store.transact(RItem.updateFolder(item, collective, folder)).attempt
|
||||
@ -478,7 +491,7 @@ object OItem {
|
||||
def setFolderMultiple(
|
||||
items: Nel[Ident],
|
||||
folder: Option[String],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[UpdateResult] =
|
||||
for {
|
||||
results <- items.traverse(i => setFolder(i, folder, collective))
|
||||
@ -499,7 +512,7 @@ object OItem {
|
||||
def setCorrOrg(
|
||||
items: Nel[Ident],
|
||||
org: Option[Ident],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[UpdateResult] =
|
||||
UpdateResult.fromUpdate(
|
||||
store
|
||||
@ -533,7 +546,7 @@ object OItem {
|
||||
def setCorrPerson(
|
||||
items: Nel[Ident],
|
||||
person: Option[Ident],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[UpdateResult] =
|
||||
UpdateResult.fromUpdate(
|
||||
store
|
||||
@ -571,7 +584,7 @@ object OItem {
|
||||
def setConcPerson(
|
||||
items: Nel[Ident],
|
||||
person: Option[Ident],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[UpdateResult] =
|
||||
UpdateResult.fromUpdate(
|
||||
store
|
||||
@ -609,7 +622,7 @@ object OItem {
|
||||
def setConcEquip(
|
||||
items: Nel[Ident],
|
||||
equip: Option[Ident],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[UpdateResult] =
|
||||
UpdateResult.fromUpdate(
|
||||
store
|
||||
@ -640,7 +653,7 @@ object OItem {
|
||||
def setNotes(
|
||||
item: Ident,
|
||||
notes: Option[String],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[UpdateResult] =
|
||||
UpdateResult
|
||||
.fromUpdate(
|
||||
@ -662,7 +675,7 @@ object OItem {
|
||||
item: Ident,
|
||||
notes: String,
|
||||
separator: Option[String],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[UpdateResult] =
|
||||
store
|
||||
.transact(RItem.appendNotes(item, collective, notes, separator))
|
||||
@ -685,7 +698,11 @@ object OItem {
|
||||
UpdateResult.notFound.pure[F]
|
||||
}
|
||||
|
||||
def setName(item: Ident, name: String, collective: Ident): F[UpdateResult] =
|
||||
def setName(
|
||||
item: Ident,
|
||||
name: String,
|
||||
collective: CollectiveId
|
||||
): F[UpdateResult] =
|
||||
UpdateResult
|
||||
.fromUpdate(
|
||||
store
|
||||
@ -705,7 +722,7 @@ object OItem {
|
||||
def setNameMultiple(
|
||||
items: Nel[Ident],
|
||||
name: String,
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[UpdateResult] =
|
||||
for {
|
||||
results <- items.traverse(i => setName(i, name, collective))
|
||||
@ -726,7 +743,7 @@ object OItem {
|
||||
def setStates(
|
||||
items: Nel[Ident],
|
||||
state: ItemState,
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[AddResult] =
|
||||
store
|
||||
.transact(RItem.updateStateForCollective(items, state, collective))
|
||||
@ -735,7 +752,7 @@ object OItem {
|
||||
|
||||
def restore(
|
||||
items: Nel[Ident],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[UpdateResult] =
|
||||
UpdateResult.fromUpdate(for {
|
||||
n <- store
|
||||
@ -748,7 +765,7 @@ object OItem {
|
||||
def setItemDate(
|
||||
items: Nel[Ident],
|
||||
date: Option[Timestamp],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[UpdateResult] =
|
||||
UpdateResult.fromUpdate(
|
||||
store
|
||||
@ -758,42 +775,42 @@ object OItem {
|
||||
def setItemDueDate(
|
||||
items: Nel[Ident],
|
||||
date: Option[Timestamp],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[UpdateResult] =
|
||||
UpdateResult.fromUpdate(
|
||||
store
|
||||
.transact(RItem.updateDueDate(items, collective, date))
|
||||
)
|
||||
|
||||
def deleteItem(itemId: Ident, collective: Ident): F[Int] =
|
||||
def deleteItem(itemId: Ident, collective: CollectiveId): F[Int] =
|
||||
QItem
|
||||
.delete(store)(itemId, collective)
|
||||
.flatTap(_ => fts.removeItem(logger, itemId))
|
||||
|
||||
def deleteItemMultiple(items: Nel[Ident], collective: Ident): F[Int] =
|
||||
def deleteItemMultiple(items: Nel[Ident], collective: CollectiveId): F[Int] =
|
||||
for {
|
||||
itemIds <- store.transact(RItem.filterItems(items, collective))
|
||||
results <- itemIds.traverse(item => deleteItem(item, collective))
|
||||
n = results.sum
|
||||
} yield n
|
||||
|
||||
def setDeletedState(items: Nel[Ident], collective: Ident): F[Int] =
|
||||
def setDeletedState(items: Nel[Ident], collective: CollectiveId): F[Int] =
|
||||
for {
|
||||
n <- store.transact(RItem.setState(items, collective, ItemState.Deleted))
|
||||
_ <- items.traverse(id => fts.removeItem(logger, id))
|
||||
} yield n
|
||||
|
||||
def getProposals(item: Ident, collective: Ident): F[MetaProposalList] =
|
||||
def getProposals(item: Ident, collective: CollectiveId): F[MetaProposalList] =
|
||||
store.transact(QAttachment.getMetaProposals(item, collective))
|
||||
|
||||
def deleteAttachment(id: Ident, collective: Ident): F[Int] =
|
||||
def deleteAttachment(id: Ident, collective: CollectiveId): F[Int] =
|
||||
QAttachment
|
||||
.deleteSingleAttachment(store)(id, collective)
|
||||
.flatTap(_ => fts.removeAttachment(logger, id))
|
||||
|
||||
def deleteAttachmentMultiple(
|
||||
attachments: Nel[Ident],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[Int] =
|
||||
for {
|
||||
attachmentIds <- store.transact(
|
||||
@ -808,7 +825,7 @@ object OItem {
|
||||
def setAttachmentName(
|
||||
attachId: Ident,
|
||||
name: Option[String],
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[UpdateResult] =
|
||||
UpdateResult
|
||||
.fromUpdate(
|
||||
@ -833,49 +850,52 @@ object OItem {
|
||||
)
|
||||
|
||||
def reprocess(
|
||||
cid: CollectiveId,
|
||||
item: Ident,
|
||||
attachments: List[Ident],
|
||||
account: AccountId
|
||||
submitter: UserTaskScope
|
||||
): F[UpdateResult] =
|
||||
(for {
|
||||
_ <- OptionT(
|
||||
store.transact(RItem.findByIdAndCollective(item, account.collective))
|
||||
store.transact(RItem.findByIdAndCollective(item, cid))
|
||||
)
|
||||
args = ReProcessItemArgs(item, attachments)
|
||||
job <- OptionT.liftF(
|
||||
JobFactory.reprocessItem[F](args, account, Priority.Low)
|
||||
JobFactory.reprocessItem[F](args, submitter, Priority.Low)
|
||||
)
|
||||
_ <- OptionT.liftF(jobStore.insertIfNew(job.encode))
|
||||
} yield UpdateResult.success).getOrElse(UpdateResult.notFound)
|
||||
|
||||
def reprocessAll(
|
||||
cid: CollectiveId,
|
||||
items: Nel[Ident],
|
||||
account: AccountId
|
||||
submitter: UserTaskScope
|
||||
): F[UpdateResult] =
|
||||
UpdateResult.fromUpdate(for {
|
||||
items <- store.transact(RItem.filterItems(items, account.collective))
|
||||
items <- store.transact(RItem.filterItems(items, cid))
|
||||
jobs <- items
|
||||
.map(item => ReProcessItemArgs(item, Nil))
|
||||
.traverse(arg => JobFactory.reprocessItem[F](arg, account, Priority.Low))
|
||||
.traverse(arg => JobFactory.reprocessItem[F](arg, submitter, Priority.Low))
|
||||
.map(_.map(_.encode))
|
||||
_ <- jobStore.insertAllIfNew(jobs)
|
||||
} yield items.size)
|
||||
|
||||
def convertAllPdf(
|
||||
collective: Option[Ident],
|
||||
submitter: Option[Ident]
|
||||
collective: Option[CollectiveId],
|
||||
submitter: UserTaskScope
|
||||
): F[UpdateResult] =
|
||||
for {
|
||||
job <- JobFactory.convertAllPdfs[F](collective, submitter, Priority.Low)
|
||||
job <- JobFactory
|
||||
.convertAllPdfs[F](ConvertAllPdfArgs(collective), submitter, Priority.Low)
|
||||
_ <- jobStore.insertIfNew(job.encode)
|
||||
} yield UpdateResult.success
|
||||
|
||||
def generatePreview(
|
||||
args: MakePreviewArgs,
|
||||
account: AccountId
|
||||
submitter: UserTaskScope
|
||||
): F[UpdateResult] =
|
||||
for {
|
||||
job <- JobFactory.makePreview[F](args, account.some)
|
||||
job <- JobFactory.makePreview[F](args, submitter)
|
||||
_ <- jobStore.insertIfNew(job.encode)
|
||||
} yield UpdateResult.success
|
||||
|
||||
@ -883,7 +903,8 @@ object OItem {
|
||||
storeMode: MakePreviewArgs.StoreMode
|
||||
): F[UpdateResult] =
|
||||
for {
|
||||
job <- JobFactory.allPreviews[F](AllPreviewsArgs(None, storeMode), None)
|
||||
job <- JobFactory
|
||||
.allPreviews[F](AllPreviewsArgs(None, storeMode), UserTaskScope.system)
|
||||
_ <- jobStore.insertIfNew(job.encode)
|
||||
} yield UpdateResult.success
|
||||
|
||||
|
@ -9,10 +9,9 @@ package docspell.backend.ops
|
||||
import cats.data.NonEmptyList
|
||||
import cats.effect._
|
||||
import cats.implicits._
|
||||
|
||||
import docspell.backend.ops.OItemLink.LinkResult
|
||||
import docspell.backend.ops.search.OSearch
|
||||
import docspell.common.{AccountId, Ident}
|
||||
import docspell.common._
|
||||
import docspell.query.ItemQuery
|
||||
import docspell.query.ItemQueryDsl._
|
||||
import docspell.store.qb.Batch
|
||||
@ -22,12 +21,16 @@ import docspell.store.{AddResult, Store}
|
||||
|
||||
trait OItemLink[F[_]] {
|
||||
|
||||
def addAll(cid: Ident, target: Ident, related: NonEmptyList[Ident]): F[LinkResult]
|
||||
def addAll(
|
||||
cid: CollectiveId,
|
||||
target: Ident,
|
||||
related: NonEmptyList[Ident]
|
||||
): F[LinkResult]
|
||||
|
||||
def removeAll(cid: Ident, target: Ident, related: NonEmptyList[Ident]): F[Unit]
|
||||
def removeAll(cid: CollectiveId, target: Ident, related: NonEmptyList[Ident]): F[Unit]
|
||||
|
||||
def getRelated(
|
||||
account: AccountId,
|
||||
account: AccountInfo,
|
||||
item: Ident,
|
||||
batch: Batch
|
||||
): F[Vector[ListItemWithTags]]
|
||||
@ -48,12 +51,12 @@ object OItemLink {
|
||||
def apply[F[_]: Sync](store: Store[F], search: OSearch[F]): OItemLink[F] =
|
||||
new OItemLink[F] {
|
||||
def getRelated(
|
||||
accountId: AccountId,
|
||||
accountId: AccountInfo,
|
||||
item: Ident,
|
||||
batch: Batch
|
||||
): F[Vector[ListItemWithTags]] =
|
||||
store
|
||||
.transact(RItemLink.findLinked(accountId.collective, item))
|
||||
.transact(RItemLink.findLinked(accountId.collectiveId, item))
|
||||
.map(ids => NonEmptyList.fromList(ids.toList))
|
||||
.flatMap {
|
||||
case Some(nel) =>
|
||||
@ -69,14 +72,22 @@ object OItemLink {
|
||||
Vector.empty[ListItemWithTags].pure[F]
|
||||
}
|
||||
|
||||
def addAll(cid: Ident, target: Ident, related: NonEmptyList[Ident]): F[LinkResult] =
|
||||
def addAll(
|
||||
cid: CollectiveId,
|
||||
target: Ident,
|
||||
related: NonEmptyList[Ident]
|
||||
): F[LinkResult] =
|
||||
if (related.contains_(target)) LinkResult.linkTargetItemError.pure[F]
|
||||
else related.traverse(addSingle(cid, target, _)).as(LinkResult.Success)
|
||||
|
||||
def removeAll(cid: Ident, target: Ident, related: NonEmptyList[Ident]): F[Unit] =
|
||||
def removeAll(
|
||||
cid: CollectiveId,
|
||||
target: Ident,
|
||||
related: NonEmptyList[Ident]
|
||||
): F[Unit] =
|
||||
store.transact(RItemLink.deleteAll(cid, target, related)).void
|
||||
|
||||
def addSingle(cid: Ident, target: Ident, related: Ident): F[Unit] = {
|
||||
def addSingle(cid: CollectiveId, target: Ident, related: Ident): F[Unit] = {
|
||||
val exists = RItemLink.exists(cid, target, related)
|
||||
val insert = RItemLink.insertNew(cid, target, related)
|
||||
store.add(insert, exists).flatMap {
|
||||
|
@ -21,32 +21,39 @@ import docspell.store.records._
|
||||
import doobie.implicits._
|
||||
|
||||
trait OItemSearch[F[_]] {
|
||||
def findItem(id: Ident, collective: Ident): F[Option[ItemData]]
|
||||
def findItem(id: Ident, collective: CollectiveId): F[Option[ItemData]]
|
||||
|
||||
def findDeleted(collective: Ident, maxUpdate: Timestamp, limit: Int): F[Vector[RItem]]
|
||||
def findDeleted(
|
||||
collective: CollectiveId,
|
||||
maxUpdate: Timestamp,
|
||||
limit: Int
|
||||
): F[Vector[RItem]]
|
||||
|
||||
def findAttachment(id: Ident, collective: Ident): F[Option[AttachmentData[F]]]
|
||||
def findAttachment(id: Ident, collective: CollectiveId): F[Option[AttachmentData[F]]]
|
||||
|
||||
def findAttachmentSource(
|
||||
id: Ident,
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[Option[AttachmentSourceData[F]]]
|
||||
|
||||
def findAttachmentArchive(
|
||||
id: Ident,
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[Option[AttachmentArchiveData[F]]]
|
||||
|
||||
def findAttachmentPreview(
|
||||
id: Ident,
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[Option[AttachmentPreviewData[F]]]
|
||||
|
||||
def findItemPreview(item: Ident, collective: Ident): F[Option[AttachmentPreviewData[F]]]
|
||||
def findItemPreview(
|
||||
item: Ident,
|
||||
collective: CollectiveId
|
||||
): F[Option[AttachmentPreviewData[F]]]
|
||||
|
||||
def findAttachmentMeta(id: Ident, collective: Ident): F[Option[RAttachmentMeta]]
|
||||
def findAttachmentMeta(id: Ident, collective: CollectiveId): F[Option[RAttachmentMeta]]
|
||||
|
||||
def findByFileCollective(checksum: String, collective: Ident): F[Vector[RItem]]
|
||||
def findByFileCollective(checksum: String, collective: CollectiveId): F[Vector[RItem]]
|
||||
|
||||
def findByFileSource(checksum: String, sourceId: Ident): F[Option[Vector[RItem]]]
|
||||
|
||||
@ -114,12 +121,12 @@ object OItemSearch {
|
||||
def apply[F[_]: Async](store: Store[F]): Resource[F, OItemSearch[F]] =
|
||||
Resource.pure[F, OItemSearch[F]](new OItemSearch[F] {
|
||||
|
||||
def findItem(id: Ident, collective: Ident): F[Option[ItemData]] =
|
||||
def findItem(id: Ident, collective: CollectiveId): F[Option[ItemData]] =
|
||||
store
|
||||
.transact(QItem.findItem(id, collective))
|
||||
|
||||
def findDeleted(
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
maxUpdate: Timestamp,
|
||||
limit: Int
|
||||
): F[Vector[RItem]] =
|
||||
@ -129,7 +136,10 @@ object OItemSearch {
|
||||
.compile
|
||||
.toVector
|
||||
|
||||
def findAttachment(id: Ident, collective: Ident): F[Option[AttachmentData[F]]] =
|
||||
def findAttachment(
|
||||
id: Ident,
|
||||
collective: CollectiveId
|
||||
): F[Option[AttachmentData[F]]] =
|
||||
store
|
||||
.transact(RAttachment.findByIdAndCollective(id, collective))
|
||||
.flatMap {
|
||||
@ -148,7 +158,7 @@ object OItemSearch {
|
||||
|
||||
def findAttachmentSource(
|
||||
id: Ident,
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[Option[AttachmentSourceData[F]]] =
|
||||
store
|
||||
.transact(RAttachmentSource.findByIdAndCollective(id, collective))
|
||||
@ -168,7 +178,7 @@ object OItemSearch {
|
||||
|
||||
def findAttachmentPreview(
|
||||
id: Ident,
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[Option[AttachmentPreviewData[F]]] =
|
||||
store
|
||||
.transact(RAttachmentPreview.findByIdAndCollective(id, collective))
|
||||
@ -188,7 +198,7 @@ object OItemSearch {
|
||||
|
||||
def findItemPreview(
|
||||
item: Ident,
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[Option[AttachmentPreviewData[F]]] =
|
||||
store
|
||||
.transact(RAttachmentPreview.findByItemAndCollective(item, collective))
|
||||
@ -208,7 +218,7 @@ object OItemSearch {
|
||||
|
||||
def findAttachmentArchive(
|
||||
id: Ident,
|
||||
collective: Ident
|
||||
collective: CollectiveId
|
||||
): F[Option[AttachmentArchiveData[F]]] =
|
||||
store
|
||||
.transact(RAttachmentArchive.findByIdAndCollective(id, collective))
|
||||
@ -234,15 +244,21 @@ object OItemSearch {
|
||||
.map(fm => f(fm))
|
||||
.value
|
||||
|
||||
def findAttachmentMeta(id: Ident, collective: Ident): F[Option[RAttachmentMeta]] =
|
||||
def findAttachmentMeta(
|
||||
id: Ident,
|
||||
collective: CollectiveId
|
||||
): F[Option[RAttachmentMeta]] =
|
||||
store.transact(QAttachment.getAttachmentMeta(id, collective))
|
||||
|
||||
def findByFileCollective(checksum: String, collective: Ident): F[Vector[RItem]] =
|
||||
def findByFileCollective(
|
||||
checksum: String,
|
||||
collective: CollectiveId
|
||||
): F[Vector[RItem]] =
|
||||
store.transact(QItem.findByChecksum(checksum, collective, Set.empty))
|
||||
|
||||
def findByFileSource(checksum: String, sourceId: Ident): F[Option[Vector[RItem]]] =
|
||||
store.transact((for {
|
||||
coll <- OptionT(RSource.findCollective(sourceId))
|
||||
coll <- OptionT(RSource.findCollectiveId(sourceId))
|
||||
items <- OptionT.liftF(QItem.findByChecksum(checksum, coll, Set.empty))
|
||||
} yield items).value)
|
||||
})
|
||||
|
@ -21,33 +21,38 @@ import emil._
|
||||
|
||||
trait OMail[F[_]] {
|
||||
|
||||
def getSmtpSettings(accId: AccountId, nameQ: Option[String]): F[Vector[RUserEmail]]
|
||||
def getSmtpSettings(userId: Ident, nameQ: Option[String]): F[Vector[RUserEmail]]
|
||||
|
||||
def findSmtpSettings(accId: AccountId, name: Ident): OptionT[F, RUserEmail]
|
||||
def findSmtpSettings(userId: Ident, name: Ident): OptionT[F, RUserEmail]
|
||||
|
||||
def createSmtpSettings(accId: AccountId, data: SmtpSettings): F[AddResult]
|
||||
def createSmtpSettings(userId: Ident, data: SmtpSettings): F[AddResult]
|
||||
|
||||
def updateSmtpSettings(accId: AccountId, name: Ident, data: OMail.SmtpSettings): F[Int]
|
||||
def updateSmtpSettings(userId: Ident, name: Ident, data: OMail.SmtpSettings): F[Int]
|
||||
|
||||
def deleteSmtpSettings(accId: AccountId, name: Ident): F[Int]
|
||||
def deleteSmtpSettings(userId: Ident, name: Ident): F[Int]
|
||||
|
||||
def getImapSettings(accId: AccountId, nameQ: Option[String]): F[Vector[RUserImap]]
|
||||
def getImapSettings(userId: Ident, nameQ: Option[String]): F[Vector[RUserImap]]
|
||||
|
||||
def findImapSettings(accId: AccountId, name: Ident): OptionT[F, RUserImap]
|
||||
def findImapSettings(userId: Ident, name: Ident): OptionT[F, RUserImap]
|
||||
|
||||
def createImapSettings(accId: AccountId, data: ImapSettings): F[AddResult]
|
||||
def createImapSettings(userId: Ident, data: ImapSettings): F[AddResult]
|
||||
|
||||
def updateImapSettings(accId: AccountId, name: Ident, data: OMail.ImapSettings): F[Int]
|
||||
def updateImapSettings(userId: Ident, name: Ident, data: OMail.ImapSettings): F[Int]
|
||||
|
||||
def deleteImapSettings(accId: AccountId, name: Ident): F[Int]
|
||||
def deleteImapSettings(userId: Ident, name: Ident): F[Int]
|
||||
|
||||
def sendMail(accId: AccountId, name: Ident, m: ItemMail): F[SendResult]
|
||||
def sendMail(
|
||||
userId: Ident,
|
||||
collectiveId: CollectiveId,
|
||||
name: Ident,
|
||||
m: ItemMail
|
||||
): F[SendResult]
|
||||
|
||||
def getSentMailsForItem(accId: AccountId, itemId: Ident): F[Vector[Sent]]
|
||||
def getSentMailsForItem(collectiveId: CollectiveId, itemId: Ident): F[Vector[Sent]]
|
||||
|
||||
def getSentMail(accId: AccountId, mailId: Ident): OptionT[F, Sent]
|
||||
def getSentMail(collectiveId: CollectiveId, mailId: Ident): OptionT[F, Sent]
|
||||
|
||||
def deleteSentMail(accId: AccountId, mailId: Ident): F[Int]
|
||||
def deleteSentMail(collectiveId: CollectiveId, mailId: Ident): F[Int]
|
||||
}
|
||||
|
||||
object OMail {
|
||||
@ -124,9 +129,9 @@ object OMail {
|
||||
mailReplyTo: Option[MailAddress]
|
||||
) {
|
||||
|
||||
def toRecord(accId: AccountId) =
|
||||
RUserEmail.fromAccount(
|
||||
accId,
|
||||
def toRecord(userId: Ident) =
|
||||
RUserEmail.fromUser(
|
||||
userId,
|
||||
name,
|
||||
smtpHost,
|
||||
smtpPort,
|
||||
@ -150,9 +155,9 @@ object OMail {
|
||||
imapOAuth2: Boolean
|
||||
) {
|
||||
|
||||
def toRecord(accId: AccountId) =
|
||||
RUserImap.fromAccount(
|
||||
accId,
|
||||
def toRecord(userId: Ident) =
|
||||
RUserImap.fromUser(
|
||||
userId,
|
||||
name,
|
||||
imapHost,
|
||||
imapPort,
|
||||
@ -167,74 +172,79 @@ object OMail {
|
||||
def apply[F[_]: Async](store: Store[F], emil: Emil[F]): Resource[F, OMail[F]] =
|
||||
Resource.pure[F, OMail[F]](new OMail[F] {
|
||||
def getSmtpSettings(
|
||||
accId: AccountId,
|
||||
userId: Ident,
|
||||
nameQ: Option[String]
|
||||
): F[Vector[RUserEmail]] =
|
||||
store.transact(RUserEmail.findByAccount(accId, nameQ))
|
||||
store.transact(RUserEmail.findByAccount(userId, nameQ))
|
||||
|
||||
def findSmtpSettings(accId: AccountId, name: Ident): OptionT[F, RUserEmail] =
|
||||
OptionT(store.transact(RUserEmail.getByName(accId, name)))
|
||||
def findSmtpSettings(userId: Ident, name: Ident): OptionT[F, RUserEmail] =
|
||||
OptionT(store.transact(RUserEmail.getByName(userId, name)))
|
||||
|
||||
def createSmtpSettings(accId: AccountId, s: SmtpSettings): F[AddResult] =
|
||||
def createSmtpSettings(userId: Ident, s: SmtpSettings): F[AddResult] =
|
||||
(for {
|
||||
ru <- OptionT(store.transact(s.toRecord(accId).value))
|
||||
ru <- OptionT(store.transact(s.toRecord(userId).value))
|
||||
ins = RUserEmail.insert(ru)
|
||||
exists = RUserEmail.exists(ru.uid, ru.name)
|
||||
res <- OptionT.liftF(store.add(ins, exists))
|
||||
} yield res).getOrElse(AddResult.Failure(new Exception("User not found")))
|
||||
|
||||
def updateSmtpSettings(
|
||||
accId: AccountId,
|
||||
userId: Ident,
|
||||
name: Ident,
|
||||
data: SmtpSettings
|
||||
): F[Int] = {
|
||||
val op = for {
|
||||
um <- OptionT(RUserEmail.getByName(accId, name))
|
||||
ru <- data.toRecord(accId)
|
||||
um <- OptionT(RUserEmail.getByName(userId, name))
|
||||
ru <- data.toRecord(userId)
|
||||
n <- OptionT.liftF(RUserEmail.update(um.id, ru))
|
||||
} yield n
|
||||
|
||||
store.transact(op.value).map(_.getOrElse(0))
|
||||
}
|
||||
|
||||
def deleteSmtpSettings(accId: AccountId, name: Ident): F[Int] =
|
||||
store.transact(RUserEmail.delete(accId, name))
|
||||
def deleteSmtpSettings(userId: Ident, name: Ident): F[Int] =
|
||||
store.transact(RUserEmail.delete(userId, name))
|
||||
|
||||
def getImapSettings(accId: AccountId, nameQ: Option[String]): F[Vector[RUserImap]] =
|
||||
store.transact(RUserImap.findByAccount(accId, nameQ))
|
||||
def getImapSettings(userId: Ident, nameQ: Option[String]): F[Vector[RUserImap]] =
|
||||
store.transact(RUserImap.findByAccount(userId, nameQ))
|
||||
|
||||
def findImapSettings(accId: AccountId, name: Ident): OptionT[F, RUserImap] =
|
||||
OptionT(store.transact(RUserImap.getByName(accId, name)))
|
||||
def findImapSettings(userId: Ident, name: Ident): OptionT[F, RUserImap] =
|
||||
OptionT(store.transact(RUserImap.getByName(userId, name)))
|
||||
|
||||
def createImapSettings(accId: AccountId, data: ImapSettings): F[AddResult] =
|
||||
def createImapSettings(userId: Ident, data: ImapSettings): F[AddResult] =
|
||||
(for {
|
||||
ru <- OptionT(store.transact(data.toRecord(accId).value))
|
||||
ru <- OptionT(store.transact(data.toRecord(userId).value))
|
||||
ins = RUserImap.insert(ru)
|
||||
exists = RUserImap.exists(ru.uid, ru.name)
|
||||
res <- OptionT.liftF(store.add(ins, exists))
|
||||
} yield res).getOrElse(AddResult.Failure(new Exception("User not found")))
|
||||
|
||||
def updateImapSettings(
|
||||
accId: AccountId,
|
||||
userId: Ident,
|
||||
name: Ident,
|
||||
data: OMail.ImapSettings
|
||||
): F[Int] = {
|
||||
val op = for {
|
||||
um <- OptionT(RUserImap.getByName(accId, name))
|
||||
ru <- data.toRecord(accId)
|
||||
um <- OptionT(RUserImap.getByName(userId, name))
|
||||
ru <- data.toRecord(userId)
|
||||
n <- OptionT.liftF(RUserImap.update(um.id, ru))
|
||||
} yield n
|
||||
|
||||
store.transact(op.value).map(_.getOrElse(0))
|
||||
}
|
||||
|
||||
def deleteImapSettings(accId: AccountId, name: Ident): F[Int] =
|
||||
store.transact(RUserImap.delete(accId, name))
|
||||
def deleteImapSettings(userId: Ident, name: Ident): F[Int] =
|
||||
store.transact(RUserImap.delete(userId, name))
|
||||
|
||||
def sendMail(accId: AccountId, name: Ident, m: ItemMail): F[SendResult] = {
|
||||
def sendMail(
|
||||
userId: Ident,
|
||||
collectiveId: CollectiveId,
|
||||
name: Ident,
|
||||
m: ItemMail
|
||||
): F[SendResult] = {
|
||||
|
||||
val getSmtpSettings: OptionT[F, RUserEmail] =
|
||||
OptionT(store.transact(RUserEmail.getByName(accId, name)))
|
||||
OptionT(store.transact(RUserEmail.getByName(userId, name)))
|
||||
|
||||
def createMail(sett: RUserEmail): OptionT[F, Mail[F]] = {
|
||||
import _root_.emil.builder._
|
||||
@ -243,7 +253,7 @@ object OMail {
|
||||
_ <- OptionT.liftF(store.transact(RItem.existsById(m.item))).filter(identity)
|
||||
ras <- OptionT.liftF(
|
||||
store.transact(
|
||||
RAttachment.findByItemAndCollectiveWithMeta(m.item, accId.collective)
|
||||
RAttachment.findByItemAndCollectiveWithMeta(m.item, collectiveId)
|
||||
)
|
||||
)
|
||||
} yield {
|
||||
@ -275,7 +285,7 @@ object OMail {
|
||||
val save = for {
|
||||
data <- RSentMail.forItem(
|
||||
m.item,
|
||||
accId,
|
||||
userId,
|
||||
msgId,
|
||||
cfg.mailFrom,
|
||||
name,
|
||||
@ -307,17 +317,20 @@ object OMail {
|
||||
} yield conv).getOrElse(SendResult.NotFound)
|
||||
}
|
||||
|
||||
def getSentMailsForItem(accId: AccountId, itemId: Ident): F[Vector[Sent]] =
|
||||
def getSentMailsForItem(
|
||||
collectiveId: CollectiveId,
|
||||
itemId: Ident
|
||||
): F[Vector[Sent]] =
|
||||
store
|
||||
.transact(QMails.findMails(accId.collective, itemId))
|
||||
.transact(QMails.findMails(collectiveId, itemId))
|
||||
.map(_.map(t => Sent.create(t._1, t._2)))
|
||||
|
||||
def getSentMail(accId: AccountId, mailId: Ident): OptionT[F, Sent] =
|
||||
OptionT(store.transact(QMails.findMail(accId.collective, mailId))).map(t =>
|
||||
def getSentMail(collectiveId: CollectiveId, mailId: Ident): OptionT[F, Sent] =
|
||||
OptionT(store.transact(QMails.findMail(collectiveId, mailId))).map(t =>
|
||||
Sent.create(t._1, t._2)
|
||||
)
|
||||
|
||||
def deleteSentMail(accId: AccountId, mailId: Ident): F[Int] =
|
||||
store.transact(QMails.delete(accId.collective, mailId))
|
||||
def deleteSentMail(collectiveId: CollectiveId, mailId: Ident): F[Int] =
|
||||
store.transact(QMails.delete(collectiveId, mailId))
|
||||
})
|
||||
}
|
||||
|
@ -39,35 +39,35 @@ trait ONotification[F[_]] {
|
||||
|
||||
def findNotificationChannel(
|
||||
ref: ChannelRef,
|
||||
account: AccountId
|
||||
userId: Ident
|
||||
): F[Vector[NotificationChannel]]
|
||||
|
||||
def listChannels(account: AccountId): F[Vector[Channel]]
|
||||
def listChannels(userId: Ident): F[Vector[Channel]]
|
||||
|
||||
def deleteChannel(id: Ident, account: AccountId): F[UpdateResult]
|
||||
def deleteChannel(id: Ident, userId: Ident): F[UpdateResult]
|
||||
|
||||
def createChannel(channel: Channel, account: AccountId): F[AddResult]
|
||||
def createChannel(channel: Channel, userId: Ident): F[AddResult]
|
||||
|
||||
def updateChannel(channel: Channel, account: AccountId): F[UpdateResult]
|
||||
def updateChannel(channel: Channel, userId: Ident): F[UpdateResult]
|
||||
|
||||
def listHooks(account: AccountId): F[Vector[Hook]]
|
||||
def listHooks(userId: Ident): F[Vector[Hook]]
|
||||
|
||||
def deleteHook(id: Ident, account: AccountId): F[UpdateResult]
|
||||
def deleteHook(id: Ident, userId: Ident): F[UpdateResult]
|
||||
|
||||
def createHook(hook: Hook, account: AccountId): F[AddResult]
|
||||
def createHook(hook: Hook, userId: Ident): F[AddResult]
|
||||
|
||||
def updateHook(hook: Hook, account: AccountId): F[UpdateResult]
|
||||
def updateHook(hook: Hook, userId: Ident): F[UpdateResult]
|
||||
|
||||
def sampleEvent(
|
||||
evt: EventType,
|
||||
account: AccountId,
|
||||
account: AccountInfo,
|
||||
baseUrl: Option[LenientUri]
|
||||
): F[EventContext]
|
||||
|
||||
def sendSampleEvent(
|
||||
evt: EventType,
|
||||
channel: Nel[ChannelRef],
|
||||
account: AccountId,
|
||||
account: AccountInfo,
|
||||
baseUrl: Option[LenientUri]
|
||||
): F[ONotification.SendTestResult]
|
||||
}
|
||||
@ -81,13 +81,6 @@ object ONotification {
|
||||
Resource.pure[F, ONotification[F]](new ONotification[F] {
|
||||
val log = docspell.logging.getLogger[F]
|
||||
|
||||
def withUserId[A](
|
||||
account: AccountId
|
||||
)(f: Ident => F[UpdateResult]): F[UpdateResult] =
|
||||
OptionT(store.transact(RUser.findIdByAccount(account)))
|
||||
.semiflatMap(f)
|
||||
.getOrElse(UpdateResult.notFound)
|
||||
|
||||
def offerEvents(ev: Iterable[Event]): F[Unit] =
|
||||
ev.toList.traverse(notMod.offer).as(())
|
||||
|
||||
@ -100,7 +93,7 @@ object ONotification {
|
||||
|
||||
def sampleEvent(
|
||||
evt: EventType,
|
||||
account: AccountId,
|
||||
account: AccountInfo,
|
||||
baseUrl: Option[LenientUri]
|
||||
): F[EventContext] =
|
||||
Event
|
||||
@ -110,14 +103,14 @@ object ONotification {
|
||||
def sendSampleEvent(
|
||||
evt: EventType,
|
||||
channels: Nel[ChannelRef],
|
||||
account: AccountId,
|
||||
account: AccountInfo,
|
||||
baseUrl: Option[LenientUri]
|
||||
): F[SendTestResult] =
|
||||
(for {
|
||||
ev <- sampleEvent(evt, account, baseUrl)
|
||||
logbuf <- Logger.buffer()
|
||||
ch <- channels.toList.toVector.flatTraverse(
|
||||
findNotificationChannel(_, account)
|
||||
findNotificationChannel(_, account.userId)
|
||||
)
|
||||
_ <- notMod.send(logbuf._2.andThen(log), ev, ch)
|
||||
logs <- logbuf._1.get
|
||||
@ -131,54 +124,51 @@ object ONotification {
|
||||
SendTestResult(false, Vector(ev))
|
||||
}
|
||||
|
||||
def listChannels(account: AccountId): F[Vector[Channel]] =
|
||||
def listChannels(userId: Ident): F[Vector[Channel]] =
|
||||
store
|
||||
.transact(RNotificationChannel.getByAccount(account))
|
||||
.transact(RNotificationChannel.getByAccount(userId))
|
||||
.map(_.map(ChannelConv.makeChannel))
|
||||
|
||||
def deleteChannel(id: Ident, account: AccountId): F[UpdateResult] =
|
||||
def deleteChannel(id: Ident, userId: Ident): F[UpdateResult] =
|
||||
UpdateResult
|
||||
.fromUpdate(
|
||||
store.transact(RNotificationChannel.deleteByAccount(id, account))
|
||||
store.transact(RNotificationChannel.deleteByAccount(id, userId))
|
||||
)
|
||||
.flatTap(_ => log.info(s"Deleted channel ${id.id} for ${account.asString}"))
|
||||
.flatTap(_ => log.info(s"Deleted channel ${id.id} for ${userId.id}"))
|
||||
|
||||
def createChannel(channel: Channel, account: AccountId): F[AddResult] =
|
||||
def createChannel(channel: Channel, userId: Ident): F[AddResult] =
|
||||
(for {
|
||||
newId <- OptionT.liftF(Ident.randomId[F])
|
||||
userId <- OptionT(store.transact(RUser.findIdByAccount(account)))
|
||||
r <- ChannelConv.makeRecord[F](store, channel, newId, userId)
|
||||
_ <- OptionT.liftF(store.transact(RNotificationChannel.insert(r)))
|
||||
_ <- OptionT.liftF(log.debug(s"Created channel $r for $account"))
|
||||
_ <- OptionT.liftF(log.debug(s"Created channel $r for ${userId.id}"))
|
||||
} yield AddResult.Success)
|
||||
.getOrElse(AddResult.failure(new Exception("User not found!")))
|
||||
|
||||
def updateChannel(channel: Channel, account: AccountId): F[UpdateResult] =
|
||||
def updateChannel(channel: Channel, userId: Ident): F[UpdateResult] =
|
||||
(for {
|
||||
userId <- OptionT(store.transact(RUser.findIdByAccount(account)))
|
||||
r <- ChannelConv.makeRecord[F](store, channel, channel.id, userId)
|
||||
n <- OptionT.liftF(store.transact(RNotificationChannel.update(r)))
|
||||
} yield UpdateResult.fromUpdateRows(n)).getOrElse(UpdateResult.notFound)
|
||||
|
||||
def listHooks(account: AccountId): F[Vector[Hook]] =
|
||||
def listHooks(userId: Ident): F[Vector[Hook]] =
|
||||
store.transact(for {
|
||||
list <- RNotificationHook.findAllByAccount(account)
|
||||
list <- RNotificationHook.findAllByAccount(userId)
|
||||
res <- list.traverse((Hook.fromRecord _).tupled)
|
||||
} yield res)
|
||||
|
||||
def deleteHook(id: Ident, account: AccountId): F[UpdateResult] =
|
||||
def deleteHook(id: Ident, userId: Ident): F[UpdateResult] =
|
||||
UpdateResult
|
||||
.fromUpdate(store.transact(RNotificationHook.deleteByAccount(id, account)))
|
||||
.fromUpdate(store.transact(RNotificationHook.deleteByAccount(id, userId)))
|
||||
|
||||
def createHook(hook: Hook, account: AccountId): F[AddResult] =
|
||||
def createHook(hook: Hook, userId: Ident): F[AddResult] =
|
||||
(for {
|
||||
_ <- OptionT.liftF(log.debug(s"Creating new notification hook: $hook"))
|
||||
userId <- OptionT(store.transact(RUser.findIdByAccount(account)))
|
||||
hr <- OptionT.liftF(Hook.makeRecord(userId, hook))
|
||||
_ <- OptionT.liftF(
|
||||
store.transact(
|
||||
RNotificationHook.insert(hr) *> RNotificationHookChannel
|
||||
.updateAll(hr.id, hook.channels.toList)
|
||||
.updateAll(hr.id, hook.channels)
|
||||
)
|
||||
)
|
||||
_ <- OptionT.liftF(
|
||||
@ -187,13 +177,11 @@ object ONotification {
|
||||
} yield AddResult.Success)
|
||||
.getOrElse(AddResult.failure(new Exception("User or channel not found!")))
|
||||
|
||||
def updateHook(hook: Hook, account: AccountId): F[UpdateResult] = {
|
||||
def updateHook(hook: Hook, userId: Ident): F[UpdateResult] = {
|
||||
def withHook(f: RNotificationHook => F[UpdateResult]): F[UpdateResult] =
|
||||
withUserId(account)(userId =>
|
||||
OptionT(store.transact(RNotificationHook.getById(hook.id, userId)))
|
||||
.semiflatMap(f)
|
||||
.getOrElse(UpdateResult.notFound)
|
||||
)
|
||||
OptionT(store.transact(RNotificationHook.getById(hook.id, userId)))
|
||||
.semiflatMap(f)
|
||||
.getOrElse(UpdateResult.notFound)
|
||||
|
||||
def doUpdate(r: RNotificationHook): F[UpdateResult] =
|
||||
UpdateResult.fromUpdate(store.transact(for {
|
||||
@ -201,10 +189,7 @@ object ONotification {
|
||||
r.id,
|
||||
if (hook.allEvents) Nil else hook.events
|
||||
)
|
||||
nc <- RNotificationHookChannel.updateAll(
|
||||
r.id,
|
||||
hook.channels.toList
|
||||
)
|
||||
nc <- RNotificationHookChannel.updateAll(r.id, hook.channels)
|
||||
nr <- RNotificationHook.update(
|
||||
r.copy(
|
||||
enabled = hook.enabled,
|
||||
@ -230,10 +215,9 @@ object ONotification {
|
||||
|
||||
def findNotificationChannel(
|
||||
ref: ChannelRef,
|
||||
accountId: AccountId
|
||||
userId: Ident
|
||||
): F[Vector[NotificationChannel]] =
|
||||
(for {
|
||||
userId <- OptionT(store.transact(RUser.findIdByAccount(accountId)))
|
||||
rec <- OptionT(store.transact(RNotificationChannel.getByRef(ref, userId)))
|
||||
ch <- OptionT.liftF(store.transact(QNotification.readChannel(rec)))
|
||||
} yield ch).getOrElse(Vector.empty)
|
||||
|
@ -18,14 +18,15 @@ import docspell.store.records._
|
||||
|
||||
trait OOrganization[F[_]] {
|
||||
def findAllOrg(
|
||||
account: AccountId,
|
||||
collectiveId: CollectiveId,
|
||||
query: Option[String],
|
||||
order: OrganizationOrder
|
||||
): F[Vector[OrgAndContacts]]
|
||||
def findOrg(account: AccountId, orgId: Ident): F[Option[OrgAndContacts]]
|
||||
|
||||
def findOrg(collectiveId: CollectiveId, orgId: Ident): F[Option[OrgAndContacts]]
|
||||
|
||||
def findAllOrgRefs(
|
||||
account: AccountId,
|
||||
collectiveId: CollectiveId,
|
||||
nameQuery: Option[String],
|
||||
order: OrganizationOrder
|
||||
): F[Vector[IdRef]]
|
||||
@ -35,15 +36,15 @@ trait OOrganization[F[_]] {
|
||||
def updateOrg(s: OrgAndContacts): F[AddResult]
|
||||
|
||||
def findAllPerson(
|
||||
account: AccountId,
|
||||
collectiveId: CollectiveId,
|
||||
query: Option[String],
|
||||
order: PersonOrder
|
||||
): F[Vector[PersonAndContacts]]
|
||||
|
||||
def findPerson(account: AccountId, persId: Ident): F[Option[PersonAndContacts]]
|
||||
def findPerson(collectiveId: CollectiveId, persId: Ident): F[Option[PersonAndContacts]]
|
||||
|
||||
def findAllPersonRefs(
|
||||
account: AccountId,
|
||||
collectiveId: CollectiveId,
|
||||
nameQuery: Option[String],
|
||||
order: PersonOrder
|
||||
): F[Vector[IdRef]]
|
||||
@ -54,9 +55,9 @@ trait OOrganization[F[_]] {
|
||||
/** Update a person with their contacts. The additional organization is ignored. */
|
||||
def updatePerson(s: PersonAndContacts): F[AddResult]
|
||||
|
||||
def deleteOrg(orgId: Ident, collective: Ident): F[AddResult]
|
||||
def deleteOrg(orgId: Ident, collective: CollectiveId): F[AddResult]
|
||||
|
||||
def deletePerson(personId: Ident, collective: Ident): F[AddResult]
|
||||
def deletePerson(personId: Ident, collective: CollectiveId): F[AddResult]
|
||||
}
|
||||
|
||||
object OOrganization {
|
||||
@ -134,32 +135,32 @@ object OOrganization {
|
||||
Resource.pure[F, OOrganization[F]](new OOrganization[F] {
|
||||
|
||||
def findAllOrg(
|
||||
account: AccountId,
|
||||
collectiveId: CollectiveId,
|
||||
query: Option[String],
|
||||
order: OrganizationOrder
|
||||
): F[Vector[OrgAndContacts]] =
|
||||
store
|
||||
.transact(
|
||||
QOrganization
|
||||
.findOrgAndContact(account.collective, query, OrganizationOrder(order))
|
||||
.findOrgAndContact(collectiveId, query, OrganizationOrder(order))
|
||||
)
|
||||
.map { case (org, cont) => OrgAndContacts(org, cont) }
|
||||
.compile
|
||||
.toVector
|
||||
|
||||
def findOrg(account: AccountId, orgId: Ident): F[Option[OrgAndContacts]] =
|
||||
def findOrg(collectiveId: CollectiveId, orgId: Ident): F[Option[OrgAndContacts]] =
|
||||
store
|
||||
.transact(QOrganization.getOrgAndContact(account.collective, orgId))
|
||||
.transact(QOrganization.getOrgAndContact(collectiveId, orgId))
|
||||
.map(_.map { case (org, cont) => OrgAndContacts(org, cont) })
|
||||
|
||||
def findAllOrgRefs(
|
||||
account: AccountId,
|
||||
collectiveId: CollectiveId,
|
||||
nameQuery: Option[String],
|
||||
order: OrganizationOrder
|
||||
): F[Vector[IdRef]] =
|
||||
store.transact(
|
||||
ROrganization.findAllRef(
|
||||
account.collective,
|
||||
collectiveId,
|
||||
nameQuery,
|
||||
OrganizationOrder(order)
|
||||
)
|
||||
@ -172,31 +173,34 @@ object OOrganization {
|
||||
QOrganization.updateOrg(s.org, s.contacts, s.org.cid)(store)
|
||||
|
||||
def findAllPerson(
|
||||
account: AccountId,
|
||||
collectiveId: CollectiveId,
|
||||
query: Option[String],
|
||||
order: PersonOrder
|
||||
): F[Vector[PersonAndContacts]] =
|
||||
store
|
||||
.transact(
|
||||
QOrganization
|
||||
.findPersonAndContact(account.collective, query, PersonOrder(order))
|
||||
.findPersonAndContact(collectiveId, query, PersonOrder(order))
|
||||
)
|
||||
.map { case (person, org, cont) => PersonAndContacts(person, org, cont) }
|
||||
.compile
|
||||
.toVector
|
||||
|
||||
def findPerson(account: AccountId, persId: Ident): F[Option[PersonAndContacts]] =
|
||||
def findPerson(
|
||||
collectiveId: CollectiveId,
|
||||
persId: Ident
|
||||
): F[Option[PersonAndContacts]] =
|
||||
store
|
||||
.transact(QOrganization.getPersonAndContact(account.collective, persId))
|
||||
.transact(QOrganization.getPersonAndContact(collectiveId, persId))
|
||||
.map(_.map { case (pers, org, cont) => PersonAndContacts(pers, org, cont) })
|
||||
|
||||
def findAllPersonRefs(
|
||||
account: AccountId,
|
||||
collectiveId: CollectiveId,
|
||||
nameQuery: Option[String],
|
||||
order: PersonOrder
|
||||
): F[Vector[IdRef]] =
|
||||
store.transact(
|
||||
RPerson.findAllRef(account.collective, nameQuery, PersonOrder.nameOnly(order))
|
||||
RPerson.findAllRef(collectiveId, nameQuery, PersonOrder.nameOnly(order))
|
||||
)
|
||||
|
||||
def addPerson(s: PersonAndContacts): F[AddResult] =
|
||||
@ -205,13 +209,13 @@ object OOrganization {
|
||||
def updatePerson(s: PersonAndContacts): F[AddResult] =
|
||||
QOrganization.updatePerson(s.person, s.contacts, s.person.cid)(store)
|
||||
|
||||
def deleteOrg(orgId: Ident, collective: Ident): F[AddResult] =
|
||||
def deleteOrg(orgId: Ident, collective: CollectiveId): F[AddResult] =
|
||||
store
|
||||
.transact(QOrganization.deleteOrg(orgId, collective))
|
||||
.attempt
|
||||
.map(AddResult.fromUpdate)
|
||||
|
||||
def deletePerson(personId: Ident, collective: Ident): F[AddResult] =
|
||||
def deletePerson(personId: Ident, collective: CollectiveId): F[AddResult] =
|
||||
store
|
||||
.transact(QOrganization.deletePerson(personId, collective))
|
||||
.attempt
|
||||
|
@ -28,16 +28,16 @@ import scodec.bits.ByteVector
|
||||
trait OShare[F[_]] {
|
||||
|
||||
def findAll(
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
ownerLogin: Option[Ident],
|
||||
query: Option[String]
|
||||
): F[List[ShareData]]
|
||||
|
||||
def delete(id: Ident, collective: Ident): F[Boolean]
|
||||
def delete(id: Ident, collective: CollectiveId): F[Boolean]
|
||||
|
||||
def addNew(share: OShare.NewShare): F[OShare.ChangeResult]
|
||||
|
||||
def findOne(id: Ident, collective: Ident): OptionT[F, ShareData]
|
||||
def findOne(id: Ident, collective: CollectiveId): OptionT[F, ShareData]
|
||||
|
||||
def update(
|
||||
id: Ident,
|
||||
@ -71,7 +71,12 @@ trait OShare[F[_]] {
|
||||
*/
|
||||
def parseQuery(share: ShareQuery, qs: String): QueryParseResult
|
||||
|
||||
def sendMail(account: AccountId, connection: Ident, mail: ShareMail): F[SendResult]
|
||||
def sendMail(
|
||||
collectiveId: CollectiveId,
|
||||
userId: Ident,
|
||||
connection: Ident,
|
||||
mail: ShareMail
|
||||
): F[SendResult]
|
||||
}
|
||||
|
||||
object OShare {
|
||||
@ -97,7 +102,7 @@ object OShare {
|
||||
case object NotFound extends SendResult
|
||||
}
|
||||
|
||||
final case class ShareQuery(id: Ident, account: AccountId, query: ItemQuery)
|
||||
final case class ShareQuery(id: Ident, account: AccountInfo, query: ItemQuery)
|
||||
|
||||
sealed trait VerifyResult {
|
||||
def toEither: Either[String, ShareToken] =
|
||||
@ -143,7 +148,7 @@ object OShare {
|
||||
def queryWithFulltext: ChangeResult = QueryWithFulltext
|
||||
}
|
||||
|
||||
final case class ShareData(share: RShare, user: RUser)
|
||||
final case class ShareData(share: RShare, account: AccountInfo)
|
||||
|
||||
def apply[F[_]: Async](
|
||||
store: Store[F],
|
||||
@ -155,7 +160,7 @@ object OShare {
|
||||
private[this] val logger = docspell.logging.getLogger[F]
|
||||
|
||||
def findAll(
|
||||
collective: Ident,
|
||||
collective: CollectiveId,
|
||||
ownerLogin: Option[Ident],
|
||||
query: Option[String]
|
||||
): F[List[ShareData]] =
|
||||
@ -163,7 +168,7 @@ object OShare {
|
||||
.transact(RShare.findAllByCollective(collective, ownerLogin, query))
|
||||
.map(_.map(ShareData.tupled))
|
||||
|
||||
def delete(id: Ident, collective: Ident): F[Boolean] =
|
||||
def delete(id: Ident, collective: CollectiveId): F[Boolean] =
|
||||
store.transact(RShare.deleteByIdAndCid(id, collective)).map(_ > 0)
|
||||
|
||||
def addNew(share: NewShare): F[ChangeResult] =
|
||||
@ -225,7 +230,7 @@ object OShare {
|
||||
case _ => true
|
||||
}
|
||||
|
||||
def findOne(id: Ident, collective: Ident): OptionT[F, ShareData] =
|
||||
def findOne(id: Ident, collective: CollectiveId): OptionT[F, ShareData] =
|
||||
RShare
|
||||
.findOne(id, collective)
|
||||
.mapK(store.transform)
|
||||
@ -286,8 +291,8 @@ object OShare {
|
||||
RShare
|
||||
.findCurrentActive(id)
|
||||
.mapK(store.transform)
|
||||
.map { case (share, user) =>
|
||||
ShareQuery(share.id, user.accountId, share.query)
|
||||
.map { case (share, accInfo) =>
|
||||
ShareQuery(share.id, accInfo, share.query)
|
||||
}
|
||||
|
||||
def findAttachmentPreview(
|
||||
@ -298,7 +303,7 @@ object OShare {
|
||||
sq <- findShareQuery(shareId)
|
||||
_ <- checkAttachment(sq, AttachId(attachId.id))
|
||||
res <- OptionT(
|
||||
itemSearch.findAttachmentPreview(attachId, sq.account.collective)
|
||||
itemSearch.findAttachmentPreview(attachId, sq.account.collectiveId)
|
||||
)
|
||||
} yield res
|
||||
|
||||
@ -306,14 +311,14 @@ object OShare {
|
||||
for {
|
||||
sq <- findShareQuery(shareId)
|
||||
_ <- checkAttachment(sq, AttachId(attachId.id))
|
||||
res <- OptionT(itemSearch.findAttachment(attachId, sq.account.collective))
|
||||
res <- OptionT(itemSearch.findAttachment(attachId, sq.account.collectiveId))
|
||||
} yield res
|
||||
|
||||
def findItem(itemId: Ident, shareId: Ident): OptionT[F, ItemData] =
|
||||
for {
|
||||
sq <- findShareQuery(shareId)
|
||||
_ <- checkAttachment(sq, Expr.itemIdEq(itemId.id))
|
||||
res <- OptionT(itemSearch.findItem(itemId, sq.account.collective))
|
||||
res <- OptionT(itemSearch.findItem(itemId, sq.account.collectiveId))
|
||||
} yield res
|
||||
|
||||
/** Check whether the attachment with the given id is in the results of the given
|
||||
@ -343,12 +348,13 @@ object OShare {
|
||||
}
|
||||
|
||||
def sendMail(
|
||||
account: AccountId,
|
||||
collectiveId: CollectiveId,
|
||||
userId: Ident,
|
||||
connection: Ident,
|
||||
mail: ShareMail
|
||||
): F[SendResult] = {
|
||||
val getSmtpSettings: OptionT[F, RUserEmail] =
|
||||
OptionT(store.transact(RUserEmail.getByName(account, connection)))
|
||||
OptionT(store.transact(RUserEmail.getByName(userId, connection)))
|
||||
|
||||
def createMail(sett: RUserEmail): OptionT[F, Mail[F]] = {
|
||||
import _root_.emil.builder._
|
||||
@ -366,20 +372,19 @@ object OShare {
|
||||
)
|
||||
}
|
||||
|
||||
def sendMail(cfg: MailConfig, mail: Mail[F]): F[Either[SendResult, String]] =
|
||||
def doSendMail(cfg: MailConfig, mail: Mail[F]): F[Either[SendResult, String]] =
|
||||
emil(cfg).send(mail).map(_.head).attempt.map(_.left.map(SendResult.SendFailure))
|
||||
|
||||
(for {
|
||||
_ <- RShare
|
||||
.findCurrentActive(mail.shareId)
|
||||
.filter(_._2.cid == account.collective)
|
||||
.filter(_._2.collectiveId == collectiveId)
|
||||
.mapK(store.transform)
|
||||
mailCfg <- getSmtpSettings
|
||||
mail <- createMail(mailCfg)
|
||||
mid <- OptionT.liftF(sendMail(mailCfg.toMailConfig, mail))
|
||||
mid <- OptionT.liftF(doSendMail(mailCfg.toMailConfig, mail))
|
||||
conv = mid.fold(identity, id => SendResult.Success(id))
|
||||
} yield conv).getOrElse(SendResult.NotFound)
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
@ -9,7 +9,7 @@ package docspell.backend.ops
|
||||
import cats.effect.{Async, Resource}
|
||||
import cats.implicits._
|
||||
|
||||
import docspell.common.{AccountId, Ident}
|
||||
import docspell.common._
|
||||
import docspell.store.UpdateResult
|
||||
import docspell.store.records.RSource
|
||||
import docspell.store.records.SourceData
|
||||
@ -17,22 +17,22 @@ import docspell.store.{AddResult, Store}
|
||||
|
||||
trait OSource[F[_]] {
|
||||
|
||||
def findAll(account: AccountId): F[Vector[SourceData]]
|
||||
def findAll(collectiveId: CollectiveId): F[Vector[SourceData]]
|
||||
|
||||
def add(s: RSource, tags: List[String]): F[AddResult]
|
||||
|
||||
def update(s: RSource, tags: List[String]): F[AddResult]
|
||||
|
||||
def delete(id: Ident, collective: Ident): F[UpdateResult]
|
||||
def delete(id: Ident, collective: CollectiveId): F[UpdateResult]
|
||||
}
|
||||
|
||||
object OSource {
|
||||
|
||||
def apply[F[_]: Async](store: Store[F]): Resource[F, OSource[F]] =
|
||||
Resource.pure[F, OSource[F]](new OSource[F] {
|
||||
def findAll(account: AccountId): F[Vector[SourceData]] =
|
||||
def findAll(collectiveId: CollectiveId): F[Vector[SourceData]] =
|
||||
store
|
||||
.transact(SourceData.findAll(account.collective, _.abbrev))
|
||||
.transact(SourceData.findAll(collectiveId, _.abbrev))
|
||||
.compile
|
||||
.to(Vector)
|
||||
|
||||
@ -52,7 +52,7 @@ object OSource {
|
||||
store.add(insert, exists).map(_.fold(identity, _.withMsg(msg), identity))
|
||||
}
|
||||
|
||||
def delete(id: Ident, collective: Ident): F[UpdateResult] =
|
||||
def delete(id: Ident, collective: CollectiveId): F[UpdateResult] =
|
||||
UpdateResult.fromUpdate(store.transact(SourceData.delete(id, collective)))
|
||||
|
||||
})
|
||||
|
@ -10,7 +10,7 @@ import cats.data.NonEmptyList
|
||||
import cats.effect.{Async, Resource}
|
||||
import cats.implicits._
|
||||
|
||||
import docspell.common.{AccountId, Ident}
|
||||
import docspell.common._
|
||||
import docspell.store.records.RTagSource
|
||||
import docspell.store.records.{RTag, RTagItem}
|
||||
import docspell.store.{AddResult, Store}
|
||||
@ -18,7 +18,7 @@ import docspell.store.{AddResult, Store}
|
||||
trait OTag[F[_]] {
|
||||
|
||||
def findAll(
|
||||
account: AccountId,
|
||||
collectiveId: CollectiveId,
|
||||
query: Option[String],
|
||||
order: OTag.TagOrder
|
||||
): F[Vector[RTag]]
|
||||
@ -27,7 +27,7 @@ trait OTag[F[_]] {
|
||||
|
||||
def update(s: RTag): F[AddResult]
|
||||
|
||||
def delete(id: Ident, collective: Ident): F[AddResult]
|
||||
def delete(id: Ident, collective: CollectiveId): F[AddResult]
|
||||
|
||||
/** Load all tags given their ids. Ids that are not available are ignored. */
|
||||
def loadAll(ids: List[Ident]): F[Vector[RTag]]
|
||||
@ -66,11 +66,11 @@ object OTag {
|
||||
def apply[F[_]: Async](store: Store[F]): Resource[F, OTag[F]] =
|
||||
Resource.pure[F, OTag[F]](new OTag[F] {
|
||||
def findAll(
|
||||
account: AccountId,
|
||||
collectiveId: CollectiveId,
|
||||
query: Option[String],
|
||||
order: TagOrder
|
||||
): F[Vector[RTag]] =
|
||||
store.transact(RTag.findAll(account.collective, query, TagOrder(order)))
|
||||
store.transact(RTag.findAll(collectiveId, query, TagOrder(order)))
|
||||
|
||||
def add(t: RTag): F[AddResult] = {
|
||||
def insert = RTag.insert(t)
|
||||
@ -88,7 +88,7 @@ object OTag {
|
||||
store.add(insert, exists).map(_.fold(identity, _.withMsg(msg), identity))
|
||||
}
|
||||
|
||||
def delete(id: Ident, collective: Ident): F[AddResult] = {
|
||||
def delete(id: Ident, collective: CollectiveId): F[AddResult] = {
|
||||
val io = for {
|
||||
optTag <- RTag.findByIdAndCollective(id, collective)
|
||||
n0 <- optTag.traverse(t => RTagItem.deleteTag(t.tagId))
|
||||
@ -99,7 +99,7 @@ object OTag {
|
||||
}
|
||||
|
||||
def loadAll(ids: List[Ident]): F[Vector[RTag]] =
|
||||
if (ids.isEmpty) Vector.empty.pure[F]
|
||||
if (ids.isEmpty) Vector.empty[RTag].pure[F]
|
||||
else store.transact(RTag.findAllById(ids))
|
||||
})
|
||||
}
|
||||
|
@ -11,9 +11,9 @@ import cats.data.{EitherT, OptionT}
|
||||
import cats.effect._
|
||||
import cats.implicits._
|
||||
import fs2.Stream
|
||||
|
||||
import docspell.backend.JobFactory
|
||||
import docspell.common._
|
||||
import docspell.scheduler.usertask.UserTaskScope
|
||||
import docspell.scheduler.{Job, JobStore}
|
||||
import docspell.store.Store
|
||||
import docspell.store.records._
|
||||
@ -22,7 +22,8 @@ trait OUpload[F[_]] {
|
||||
|
||||
def submit(
|
||||
data: OUpload.UploadData[F],
|
||||
account: AccountId,
|
||||
collectiveId: CollectiveId,
|
||||
userId: Option[Ident],
|
||||
itemId: Option[Ident]
|
||||
): F[OUpload.UploadResult]
|
||||
|
||||
@ -38,12 +39,13 @@ trait OUpload[F[_]] {
|
||||
|
||||
final def submitEither(
|
||||
data: OUpload.UploadData[F],
|
||||
accOrSrc: Either[Ident, AccountId],
|
||||
accOrSrc: Either[Ident, CollectiveId],
|
||||
userId: Option[Ident],
|
||||
itemId: Option[Ident]
|
||||
): F[OUpload.UploadResult] =
|
||||
accOrSrc match {
|
||||
case Right(acc) =>
|
||||
submit(data, acc, itemId)
|
||||
submit(data, acc, userId, itemId)
|
||||
case Left(srcId) =>
|
||||
submit(data, srcId, itemId)
|
||||
}
|
||||
@ -90,7 +92,7 @@ object OUpload {
|
||||
|
||||
def noFiles: UploadResult = NoFiles
|
||||
|
||||
/** A source (`RSource') could not be found for a given source-id. */
|
||||
/** A source (`RSource`) could not be found for a given source-id. */
|
||||
case object NoSource extends UploadResult
|
||||
|
||||
def noSource: UploadResult = NoSource
|
||||
@ -99,6 +101,11 @@ object OUpload {
|
||||
case object NoItem extends UploadResult
|
||||
|
||||
def noItem: UploadResult = NoItem
|
||||
|
||||
/** A collective with the given id was not found */
|
||||
case object NoCollective extends UploadResult
|
||||
|
||||
def noCollective: UploadResult = NoCollective
|
||||
}
|
||||
|
||||
private def right[F[_]: Functor, A](a: F[A]): EitherT[F, UploadResult, A] =
|
||||
@ -110,26 +117,30 @@ object OUpload {
|
||||
): Resource[F, OUpload[F]] =
|
||||
Resource.pure[F, OUpload[F]](new OUpload[F] {
|
||||
private[this] val logger = docspell.logging.getLogger[F]
|
||||
|
||||
def submit(
|
||||
data: OUpload.UploadData[F],
|
||||
account: AccountId,
|
||||
collectiveId: CollectiveId,
|
||||
userId: Option[Ident],
|
||||
itemId: Option[Ident]
|
||||
): F[OUpload.UploadResult] =
|
||||
(for {
|
||||
_ <- checkExistingItem(itemId, account.collective)
|
||||
files <- right(data.files.traverse(saveFile(account)).map(_.flatten))
|
||||
_ <- checkExistingItem(itemId, collectiveId)
|
||||
coll <- OptionT(store.transact(RCollective.findById(collectiveId)))
|
||||
.toRight(UploadResult.noCollective)
|
||||
files <- right(data.files.traverse(saveFile(coll.id)).map(_.flatten))
|
||||
_ <- checkFileList(files)
|
||||
lang <- data.meta.language match {
|
||||
case Some(lang) => right(lang.pure[F])
|
||||
case None =>
|
||||
right(
|
||||
store
|
||||
.transact(RCollective.findLanguage(account.collective))
|
||||
.transact(RCollective.findLanguage(collectiveId))
|
||||
.map(_.getOrElse(Language.German))
|
||||
)
|
||||
}
|
||||
meta = ProcessItemArgs.ProcessMeta(
|
||||
account.collective,
|
||||
collectiveId,
|
||||
itemId,
|
||||
lang,
|
||||
data.meta.direction,
|
||||
@ -143,12 +154,18 @@ object OUpload {
|
||||
data.meta.attachmentsOnly
|
||||
)
|
||||
args = ProcessItemArgs(meta, files.toList)
|
||||
jobs <- right(makeJobs(data, args, account))
|
||||
jobs <- right(
|
||||
makeJobs(
|
||||
data,
|
||||
args,
|
||||
UserTaskScope(collectiveId, userId)
|
||||
)
|
||||
)
|
||||
_ <- right(logger.debug(s"Storing jobs: $jobs"))
|
||||
res <- right(submitJobs(jobs.map(_.encode)))
|
||||
_ <- right(
|
||||
store.transact(
|
||||
RSource.incrementCounter(data.meta.sourceAbbrev, account.collective)
|
||||
RSource.incrementCounter(data.meta.sourceAbbrev, collectiveId)
|
||||
)
|
||||
)
|
||||
} yield res).fold(identity, identity)
|
||||
@ -174,8 +191,7 @@ object OUpload {
|
||||
),
|
||||
priority = src.source.priority
|
||||
)
|
||||
accId = AccountId(src.source.cid, src.source.sid)
|
||||
result <- OptionT.liftF(submit(updata, accId, itemId))
|
||||
result <- OptionT.liftF(submit(updata, src.source.cid, None, itemId))
|
||||
} yield result).getOrElse(UploadResult.noSource)
|
||||
|
||||
private def submitJobs(jobs: List[Job[String]]): F[OUpload.UploadResult] =
|
||||
@ -186,13 +202,13 @@ object OUpload {
|
||||
|
||||
/** Saves the file into the database. */
|
||||
private def saveFile(
|
||||
accountId: AccountId
|
||||
collectiveId: CollectiveId
|
||||
)(file: File[F]): F[Option[ProcessItemArgs.File]] =
|
||||
logger.info(s"Receiving file $file") *>
|
||||
file.data
|
||||
.through(
|
||||
store.fileRepo.save(
|
||||
accountId.collective,
|
||||
collectiveId,
|
||||
FileCategory.AttachmentSource,
|
||||
MimeTypeHint(file.name, None)
|
||||
)
|
||||
@ -212,7 +228,7 @@ object OUpload {
|
||||
|
||||
private def checkExistingItem(
|
||||
itemId: Option[Ident],
|
||||
coll: Ident
|
||||
coll: CollectiveId
|
||||
): EitherT[F, UploadResult, Unit] =
|
||||
itemId match {
|
||||
case None =>
|
||||
@ -232,22 +248,22 @@ object OUpload {
|
||||
private def makeJobs(
|
||||
data: UploadData[F],
|
||||
args: ProcessItemArgs,
|
||||
account: AccountId
|
||||
submitter: UserTaskScope
|
||||
): F[List[Job[ProcessItemArgs]]] =
|
||||
if (data.meta.flattenArchives.getOrElse(false))
|
||||
JobFactory
|
||||
.multiUpload(args, account, data.priority, data.tracker)
|
||||
.multiUpload(args, submitter, data.priority, data.tracker)
|
||||
.map(List(_))
|
||||
else if (data.multiple)
|
||||
JobFactory.processItems(
|
||||
args.files.map(f => args.copy(files = List(f))),
|
||||
account,
|
||||
submitter,
|
||||
data.priority,
|
||||
data.tracker
|
||||
)
|
||||
else
|
||||
JobFactory
|
||||
.processItem[F](args, account, data.priority, data.tracker)
|
||||
.processItem[F](args, submitter, data.priority, data.tracker)
|
||||
.map(List(_))
|
||||
})
|
||||
}
|
||||
|
@ -79,7 +79,7 @@ trait OSearch[F[_]] {
|
||||
* `q.fix` part.
|
||||
*/
|
||||
def parseQueryString(
|
||||
accountId: AccountId,
|
||||
accountId: AccountInfo,
|
||||
mode: SearchMode,
|
||||
qs: String
|
||||
): QueryParseResult
|
||||
@ -94,7 +94,7 @@ object OSearch {
|
||||
private[this] val logger = docspell.logging.getLogger[F]
|
||||
|
||||
def parseQueryString(
|
||||
accountId: AccountId,
|
||||
accountId: AccountInfo,
|
||||
mode: SearchMode,
|
||||
qs: String
|
||||
): QueryParseResult = {
|
||||
@ -199,7 +199,7 @@ object OSearch {
|
||||
timed <- Duration.stopTime[F]
|
||||
resolved <- store
|
||||
.transact(
|
||||
QItem.findItemsWithTags(q.fix.account.collective, Stream.emits(items))
|
||||
QItem.findItemsWithTags(q.fix.account.collectiveId, Stream.emits(items))
|
||||
)
|
||||
.compile
|
||||
.toVector
|
||||
@ -233,13 +233,13 @@ object OSearch {
|
||||
}
|
||||
|
||||
private def createFtsQuery(
|
||||
account: AccountId,
|
||||
account: AccountInfo,
|
||||
ftq: String
|
||||
): F[FtsQuery] =
|
||||
store
|
||||
.transact(QFolder.getMemberFolders(account))
|
||||
.transact(QFolder.getMemberFolders(account.collectiveId, account.userId))
|
||||
.map(folders =>
|
||||
FtsQuery(ftq, account.collective, 500, 0)
|
||||
FtsQuery(ftq, account.collectiveId, 500, 0)
|
||||
.withFolders(folders)
|
||||
)
|
||||
|
||||
|
@ -6,14 +6,13 @@
|
||||
|
||||
package docspell.backend.signup
|
||||
|
||||
import cats.data.OptionT
|
||||
import cats.effect.{Async, Resource}
|
||||
import cats.implicits._
|
||||
|
||||
import docspell.backend.PasswordCrypt
|
||||
import docspell.common._
|
||||
import docspell.store.records.{RCollective, RInvitation, RUser}
|
||||
import docspell.store.{AddResult, Store}
|
||||
|
||||
import doobie.free.connection.ConnectionIO
|
||||
|
||||
trait OSignup[F[_]] {
|
||||
@ -48,7 +47,7 @@ object OSignup {
|
||||
def register(cfg: Config)(data: RegisterData): F[SignupResult] =
|
||||
cfg.mode match {
|
||||
case Config.Mode.Open =>
|
||||
addUser(data).map(SignupResult.fromAddResult)
|
||||
addNewAccount(data, AccountSource.Local).map(SignupResult.fromAddResult)
|
||||
|
||||
case Config.Mode.Closed =>
|
||||
SignupResult.signupClosed.pure[F]
|
||||
@ -61,7 +60,9 @@ object OSignup {
|
||||
min = now.minus(cfg.inviteTime)
|
||||
ok <- store.transact(RInvitation.useInvite(inv, min))
|
||||
res <-
|
||||
if (ok) addUser(data).map(SignupResult.fromAddResult)
|
||||
if (ok)
|
||||
addNewAccount(data, AccountSource.Local)
|
||||
.map(SignupResult.fromAddResult)
|
||||
else SignupResult.invalidInvitationKey.pure[F]
|
||||
_ <-
|
||||
if (retryInvite(res))
|
||||
@ -84,26 +85,37 @@ object OSignup {
|
||||
SignupResult
|
||||
.failure(new Exception("Account source must not be LOCAL!"))
|
||||
.pure[F]
|
||||
else
|
||||
for {
|
||||
recs <- makeRecords(data.collName, data.login, Password(""), data.source)
|
||||
cres <- store.add(
|
||||
RCollective.insert(recs._1),
|
||||
RCollective.existsById(data.collName)
|
||||
)
|
||||
ures <- store.add(RUser.insert(recs._2), RUser.exists(data.login))
|
||||
res = cres match {
|
||||
case AddResult.Failure(ex) =>
|
||||
SignupResult.failure(ex)
|
||||
case _ =>
|
||||
ures match {
|
||||
case AddResult.Failure(ex) =>
|
||||
SignupResult.failure(ex)
|
||||
case _ =>
|
||||
SignupResult.success
|
||||
}
|
||||
}
|
||||
} yield res
|
||||
else {
|
||||
val maybeInsert: ConnectionIO[Unit] =
|
||||
for {
|
||||
now <- Timestamp.current[ConnectionIO]
|
||||
cid <- OptionT(RCollective.findByName(data.collName))
|
||||
.map(_.id)
|
||||
.getOrElseF(
|
||||
RCollective.insert(RCollective.makeDefault(data.collName, now))
|
||||
)
|
||||
|
||||
uid <- Ident.randomId[ConnectionIO]
|
||||
newUser = RUser.makeDefault(
|
||||
uid,
|
||||
data.login,
|
||||
cid,
|
||||
Password(""),
|
||||
AccountSource.OpenId,
|
||||
now
|
||||
)
|
||||
_ <- OptionT(RUser.findByLogin(data.login, cid.some))
|
||||
.map(_ => 1)
|
||||
.getOrElseF(RUser.insert(newUser))
|
||||
} yield ()
|
||||
|
||||
store.transact(maybeInsert).attempt.map {
|
||||
case Left(ex) =>
|
||||
SignupResult.failure(ex)
|
||||
case Right(_) =>
|
||||
SignupResult.success
|
||||
}
|
||||
}
|
||||
|
||||
private def retryInvite(res: SignupResult): Boolean =
|
||||
res match {
|
||||
@ -119,41 +131,38 @@ object OSignup {
|
||||
false
|
||||
}
|
||||
|
||||
private def addUser(data: RegisterData): F[AddResult] = {
|
||||
def insert(coll: RCollective, user: RUser): ConnectionIO[Int] =
|
||||
private def addNewAccount(
|
||||
data: RegisterData,
|
||||
accountSource: AccountSource
|
||||
): F[AddResult] = {
|
||||
def insert: ConnectionIO[Int] =
|
||||
for {
|
||||
n1 <- RCollective.insert(coll)
|
||||
n2 <- RUser.insert(user)
|
||||
} yield n1 + n2
|
||||
now <- Timestamp.current[ConnectionIO]
|
||||
cid <- RCollective.insert(RCollective.makeDefault(data.collName, now))
|
||||
uid <- Ident.randomId[ConnectionIO]
|
||||
n2 <- RUser.insert(
|
||||
RUser.makeDefault(
|
||||
uid,
|
||||
data.login,
|
||||
cid,
|
||||
if (data.password.isEmpty) data.password
|
||||
else PasswordCrypt.crypt(data.password),
|
||||
accountSource,
|
||||
now
|
||||
)
|
||||
)
|
||||
} yield n2
|
||||
|
||||
def collectiveExists: ConnectionIO[Boolean] =
|
||||
RCollective.existsById(data.collName)
|
||||
RCollective.existsByName(data.collName)
|
||||
|
||||
val msg = s"The collective '${data.collName}' already exists."
|
||||
for {
|
||||
cu <- makeRecords(data.collName, data.login, data.password, AccountSource.Local)
|
||||
save <- store.add(insert(cu._1, cu._2), collectiveExists)
|
||||
} yield save.fold(identity, _.withMsg(msg), identity)
|
||||
exists <- store.transact(collectiveExists)
|
||||
saved <-
|
||||
if (exists) AddResult.entityExists(msg).pure[F]
|
||||
else store.transact(insert).attempt.map(AddResult.fromUpdate)
|
||||
} yield saved
|
||||
}
|
||||
|
||||
private def makeRecords(
|
||||
collName: Ident,
|
||||
login: Ident,
|
||||
password: Password,
|
||||
source: AccountSource
|
||||
): F[(RCollective, RUser)] =
|
||||
for {
|
||||
id2 <- Ident.randomId[F]
|
||||
now <- Timestamp.current[F]
|
||||
c = RCollective.makeDefault(collName, now)
|
||||
u = RUser.makeDefault(
|
||||
id2,
|
||||
login,
|
||||
collName,
|
||||
PasswordCrypt.crypt(password),
|
||||
source,
|
||||
now
|
||||
)
|
||||
} yield (c, u)
|
||||
})
|
||||
}
|
||||
|
@ -16,8 +16,18 @@ import scodec.bits.ByteVector
|
||||
|
||||
class AuthTokenTest extends CatsEffectSuite {
|
||||
|
||||
val user = AccountId(Ident.unsafe("demo"), Ident.unsafe("demo"))
|
||||
val john = AccountId(Ident.unsafe("demo"), Ident.unsafe("john"))
|
||||
val user = AccountInfo(
|
||||
CollectiveId(1),
|
||||
Ident.unsafe("demo"),
|
||||
Ident.unsafe("abc-def"),
|
||||
Ident.unsafe("demo")
|
||||
)
|
||||
val john = AccountInfo(
|
||||
CollectiveId(1),
|
||||
Ident.unsafe("demo"),
|
||||
Ident.unsafe("abc-hij"),
|
||||
Ident.unsafe("john")
|
||||
)
|
||||
val secret = ByteVector.fromValidHex("caffee")
|
||||
val otherSecret = ByteVector.fromValidHex("16bad")
|
||||
|
||||
|
Reference in New Issue
Block a user