Skip to content

Commit

Permalink
Non-empty comment handling refactors
Browse files Browse the repository at this point in the history
  • Loading branch information
mgoworko committed Oct 24, 2024
1 parent 0a0ddcb commit 329bdb3
Show file tree
Hide file tree
Showing 8 changed files with 63 additions and 59 deletions.
Original file line number Diff line number Diff line change
@@ -1,17 +1,17 @@
package pl.touk.nussknacker.engine.api

import io.circe.{Decoder, Encoder}

final case class Comment private (content: String) extends AnyVal {
class Comment private (val content: String) extends AnyVal {
override def toString: String = content
}

object Comment {

def from(content: String): Option[Comment] = {
if (content.isEmpty) None else Some(Comment(content))
if (content.isEmpty) None else Some(new Comment(content))
}

def unsafeFrom(content: String): Comment = {
from(content).getOrElse(throw new IllegalArgumentException("Comment content cannot be empty"))
}

implicit val encoder: Encoder[Comment] = Encoder.encodeString.contramap(_.content)
implicit val decoder: Decoder[Comment] = Decoder.decodeString.map(Comment.apply)
}
Original file line number Diff line number Diff line change
@@ -1,5 +1,6 @@
package pl.touk.nussknacker.ui.api

import pl.touk.nussknacker.engine.api.Comment
import pl.touk.nussknacker.engine.api.deployment.ProblemDeploymentStatus
import pl.touk.nussknacker.ui.api.description.DeploymentApiEndpoints
import pl.touk.nussknacker.ui.api.description.DeploymentApiEndpoints.Dtos._
Expand Down Expand Up @@ -32,7 +33,7 @@ class DeploymentApiHttpService(
nodesDeploymentData = request.nodesDeploymentData,
user = loggedUser
),
request.comment
request.comment.flatMap(Comment.from)
)
.map(_.left.map {
case UnderlyingServiceError(err) =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -130,7 +130,7 @@ class ManagementResources(
.processCommand(
RunDeploymentCommand(
// adminProcessManagement endpoint is not used by the designer client. It is a part of API for tooling purpose
commonData = CommonCommandData(processIdWithName, comment.map(Comment.apply), user),
commonData = CommonCommandData(processIdWithName, comment.flatMap(Comment.from), user),
nodesDeploymentData = NodesDeploymentData.empty,
stateRestoringStrategy = StateRestoringStrategy.RestoreStateFromCustomSavepoint(savepointPath)
)
Expand All @@ -152,7 +152,7 @@ class ManagementResources(
deploymentService
.processCommand(
RunDeploymentCommand(
commonData = CommonCommandData(processIdWithName, comment.map(Comment.apply), user),
commonData = CommonCommandData(processIdWithName, comment.flatMap(Comment.from), user),
nodesDeploymentData = NodesDeploymentData.empty,
stateRestoringStrategy = StateRestoringStrategy.RestoreStateFromReplacedJobSavepoint
)
Expand All @@ -170,7 +170,7 @@ class ManagementResources(
measureTime("cancel", metricRegistry) {
deploymentService.processCommand(
CancelScenarioCommand(commonData =
CommonCommandData(processIdWithName, comment.map(Comment.apply), user)
CommonCommandData(processIdWithName, comment.flatMap(Comment.from), user)
)
)
}
Expand Down Expand Up @@ -268,7 +268,7 @@ class ManagementResources(
deploymentService
.processCommand(
CustomActionCommand(
commonData = CommonCommandData(processIdWithName, req.comment.map(Comment.apply), user),
commonData = CommonCommandData(processIdWithName, req.comment.flatMap(Comment.from), user),
actionName = req.actionName,
params = req.params
)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -201,7 +201,7 @@ object DeploymentApiEndpoints {
final case class RunDeploymentRequest(
scenarioName: ProcessName,
nodesDeploymentData: NodesDeploymentData,
comment: Option[Comment]
comment: Option[String]
)

implicit val deploymentStatusNameCodec: Schema[DeploymentStatusName] = Schema.string[DeploymentStatusName]
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,7 @@ import db.util.DBIOActionInstances.DB
import pl.touk.nussknacker.engine.api.component.ProcessingMode
import pl.touk.nussknacker.engine.api.deployment.ScenarioAttachment.{AttachmentFilename, AttachmentId}
import pl.touk.nussknacker.engine.api.deployment._
import pl.touk.nussknacker.engine.api.process.{ProcessId, VersionId}
import pl.touk.nussknacker.engine.api.process.ProcessId
import pl.touk.nussknacker.ui.api.description.scenarioActivity.Dtos.Legacy
import pl.touk.nussknacker.ui.db.entity.{
AdditionalProperties,
Expand Down Expand Up @@ -63,8 +63,12 @@ class DbScenarioActivityRepository private (override protected val dbRef: DbRef,
modifyActivityByActivityId(
activityId = activityId,
activityDoesNotExistError = ModifyCommentError.ActivityDoesNotExist,
validateCurrentValue = entity => validateThatActivityIsAssignedToScenario(scenarioId)(entity),
modify = doEditComment(validate, comment),
validateCurrentValue = entity =>
for {
_ <- validateThatActivityIsAssignedToScenario(scenarioId, entity)
_ <- validate(commentModificationMetadata(entity))
} yield (),
modify = doEditComment(comment),
couldNotModifyError = ModifyCommentError.CouldNotModifyComment,
)
}
Expand All @@ -77,8 +81,12 @@ class DbScenarioActivityRepository private (override protected val dbRef: DbRef,
modifyActivityByRowId(
rowId = rowId,
activityDoesNotExistError = ModifyCommentError.ActivityDoesNotExist,
validateCurrentValue = validateThatActivityIsAssignedToScenario(scenarioId),
modify = doDeleteComment(validate),
validateCurrentValue = entity =>
for {
_ <- validateThatActivityIsAssignedToScenario(scenarioId, entity)
_ <- validate(commentModificationMetadata(entity))
} yield (),
modify = doDeleteComment,
couldNotModifyError = ModifyCommentError.CouldNotModifyComment,
)
}
Expand All @@ -91,8 +99,12 @@ class DbScenarioActivityRepository private (override protected val dbRef: DbRef,
modifyActivityByActivityId(
activityId = activityId,
activityDoesNotExistError = ModifyCommentError.ActivityDoesNotExist,
validateCurrentValue = validateThatActivityIsAssignedToScenario(scenarioId),
modify = doDeleteComment(validate),
validateCurrentValue = entity =>
for {
_ <- validateThatActivityIsAssignedToScenario(scenarioId, entity)
_ <- validate(commentModificationMetadata(entity))
} yield (),
modify = doDeleteComment,
couldNotModifyError = ModifyCommentError.CouldNotModifyComment,
)
}
Expand Down Expand Up @@ -230,10 +242,13 @@ class DbScenarioActivityRepository private (override protected val dbRef: DbRef,
}
}

private def validateThatActivityIsAssignedToScenario(scenarioId: ProcessId)(entity: ScenarioActivityEntityData) = {
private def validateThatActivityIsAssignedToScenario(
scenarioId: ProcessId,
entity: ScenarioActivityEntityData
): Either[ModifyCommentError, Unit] = {
for {
_ <- Either.cond(entity.scenarioId == scenarioId, (), ModifyCommentError.CommentDoesNotExist)
} yield entity
} yield ()
}

private def toComment(
Expand Down Expand Up @@ -335,14 +350,14 @@ class DbScenarioActivityRepository private (override protected val dbRef: DbRef,
fromEntity(entity).left.map(_ => ModifyActivityError.CouldNotModifyActivity).map(_._2)
}

private def modifyActivityByActivityId[ERROR, T](
private def modifyActivityByActivityId[ERROR](
activityId: ScenarioActivityId,
activityDoesNotExistError: ERROR,
validateCurrentValue: ScenarioActivityEntityData => Either[ERROR, T],
modify: T => Either[ERROR, ScenarioActivityEntityData],
validateCurrentValue: ScenarioActivityEntityData => Either[ERROR, Unit],
modify: ScenarioActivityEntityData => ScenarioActivityEntityData,
couldNotModifyError: ERROR,
): DB[Either[ERROR, ScenarioActivityId]] = {
doModifyActivity[ScenarioActivityId, ERROR, T](
doModifyActivity[ScenarioActivityId, ERROR](
key = activityId,
fetchActivity = activityByIdCompiled(_).result.headOption,
updateRow = (id: ScenarioActivityId, updatedEntity) => activityByIdCompiled(id).update(updatedEntity),
Expand All @@ -356,11 +371,11 @@ class DbScenarioActivityRepository private (override protected val dbRef: DbRef,
private def modifyActivityByRowId[ERROR](
rowId: Long,
activityDoesNotExistError: ERROR,
validateCurrentValue: ScenarioActivityEntityData => Either[ERROR, ScenarioActivityEntityData],
modify: ScenarioActivityEntityData => Either[ERROR, ScenarioActivityEntityData],
validateCurrentValue: ScenarioActivityEntityData => Either[ERROR, Unit],
modify: ScenarioActivityEntityData => ScenarioActivityEntityData,
couldNotModifyError: ERROR,
): DB[Either[ERROR, ScenarioActivityId]] = {
doModifyActivity[Long, ERROR, ScenarioActivityEntityData](
doModifyActivity[Long, ERROR](
key = rowId,
fetchActivity = activityByRowIdCompiled(_).result.headOption,
updateRow = (id: Long, updatedEntity) => activityByRowIdCompiled(id).update(updatedEntity),
Expand All @@ -371,22 +386,22 @@ class DbScenarioActivityRepository private (override protected val dbRef: DbRef,
)
}

private def doModifyActivity[KEY, ERROR, VALIDATED](
private def doModifyActivity[KEY, ERROR](
key: KEY,
fetchActivity: KEY => DB[Option[ScenarioActivityEntityData]],
updateRow: (KEY, ScenarioActivityEntityData) => DB[Int],
activityDoesNotExistError: ERROR,
validateCurrentValue: ScenarioActivityEntityData => Either[ERROR, VALIDATED],
modify: VALIDATED => Either[ERROR, ScenarioActivityEntityData],
validateCurrentValue: ScenarioActivityEntityData => Either[ERROR, Unit],
modify: ScenarioActivityEntityData => ScenarioActivityEntityData,
couldNotModifyError: ERROR,
): DB[Either[ERROR, ScenarioActivityId]] = {
val action = for {
fetchedActivity <- fetchActivity(key)
result <- {
val modifiedEntity = for {
entity <- fetchedActivity.toRight(activityDoesNotExistError)
validated <- validateCurrentValue(entity)
modifiedEntity <- modify(validated)
entity <- fetchedActivity.toRight(activityDoesNotExistError)
_ <- validateCurrentValue(entity)
modifiedEntity = modify(entity)
} yield modifiedEntity

modifiedEntity match {
Expand Down Expand Up @@ -416,12 +431,6 @@ class DbScenarioActivityRepository private (override protected val dbRef: DbRef,
}
}

private def doEditComment(validate: CommentModificationMetadata => Either[ModifyCommentError, Unit], comment: String)(
entity: ScenarioActivityEntityData
)(implicit user: LoggedUser): Either[ModifyCommentError, ScenarioActivityEntityData] = {
validate(commentModificationMetadata(entity)).map(_ => doEditComment(comment)(entity))
}

private def doEditComment(comment: String)(
entity: ScenarioActivityEntityData
)(implicit user: LoggedUser): ScenarioActivityEntityData = {
Expand All @@ -435,12 +444,6 @@ class DbScenarioActivityRepository private (override protected val dbRef: DbRef,
)
}

private def doDeleteComment(validate: CommentModificationMetadata => Either[ModifyCommentError, Unit])(
entity: ScenarioActivityEntityData
)(implicit user: LoggedUser): Either[ModifyCommentError, ScenarioActivityEntityData] = {
validate(commentModificationMetadata(entity)).map(_ => doDeleteComment(entity))
}

private def doDeleteComment(
entity: ScenarioActivityEntityData
)(implicit user: LoggedUser): ScenarioActivityEntityData = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -137,31 +137,31 @@ private[test] class ScenarioHelper(dbRef: DbRef, clock: Clock, designerConfig: C

private def prepareDeploy(scenarioId: ProcessId, processingType: String): Future[_] = {
val actionName = ScenarioActionName.Deploy
val comment = Comment("Deploy comment")
val comment = Comment.from("Deploy comment")
dbioRunner.run(
actionRepository.addInstantAction(
scenarioId,
VersionId.initialVersionId,
actionName,
Some(comment),
comment,
Some(processingType)
)
)
}

private def prepareCancel(scenarioId: ProcessId): Future[_] = {
val actionName = ScenarioActionName.Cancel
val comment = Comment("Cancel comment")
val comment = Comment.from("Cancel comment")
dbioRunner.run(
actionRepository.addInstantAction(scenarioId, VersionId.initialVersionId, actionName, Some(comment), None)
actionRepository.addInstantAction(scenarioId, VersionId.initialVersionId, actionName, comment, None)
)
}

private def prepareCustomAction(scenarioId: ProcessId): Future[_] = {
val actionName = ScenarioActionName("Custom")
val comment = Comment("Execute custom action")
val comment = Comment.from("Execute custom action")
dbioRunner.run(
actionRepository.addInstantAction(scenarioId, VersionId.initialVersionId, actionName, Some(comment), None)
actionRepository.addInstantAction(scenarioId, VersionId.initialVersionId, actionName, comment, None)
)
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,8 @@ class DeploymentCommentSpec extends AnyFunSuite with Matchers {
exampleComment = None
)

private val validComment = Comment("issues/123123")
private val invalidComment = Comment("invalid_comment")
private val validComment = Comment.unsafeFrom("issues/123123")
private val invalidComment = Comment.unsafeFrom("invalid_comment")
private val emptyComment = None

private val emptyValidationPattern = ""
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -192,7 +192,7 @@ class DeploymentServiceSpec

deploymentServiceWithCommentSettings.processCommand(
RunDeploymentCommand(
CommonCommandData(processIdWithName, Some(Comment("samplePattern")), user),
CommonCommandData(processIdWithName, Comment.from("samplePattern"), user),
StateRestoringStrategy.RestoreStateFromReplacedJobSavepoint,
NodesDeploymentData.empty
)
Expand Down Expand Up @@ -943,12 +943,12 @@ class DeploymentServiceSpec
val processName: ProcessName = generateProcessName
val processIdWithName = prepareProcess(processName).dbioActionValues
val actionName = ScenarioActionName("hello")
val comment = Comment("not empty comment")
val comment = Comment.from("not empty comment")

val result =
deploymentService
.processCommand(
CustomActionCommand(CommonCommandData(processIdWithName, Some(comment), user), actionName, Map.empty)
CustomActionCommand(CommonCommandData(processIdWithName, comment, user), actionName, Map.empty)
)
.futureValue

Expand All @@ -958,7 +958,7 @@ class DeploymentServiceSpec
actionRepository.getFinishedProcessActions(processIdWithName.id, Some(Set(actionName))).dbioActionValues

action.loneElement.state shouldBe ProcessActionState.Finished
action.loneElement.comment shouldBe Some(comment.content)
action.loneElement.comment shouldBe comment.map(_.content)
listener.events.toArray.filter(_.isInstanceOf[OnActionSuccess]) should have length 1
}
}
Expand Down Expand Up @@ -1055,7 +1055,7 @@ class DeploymentServiceSpec
}

private def prepareAction(processId: ProcessId, actionName: ScenarioActionName) = {
val comment = Some(Comment(actionName.toString.capitalize))
val comment = Comment.from(actionName.toString.capitalize)
actionRepository.addInstantAction(processId, initialVersionId, actionName, comment, None).map(_.id)
}

Expand Down

0 comments on commit 329bdb3

Please sign in to comment.