mirror of
https://github.com/VinylDNS/vinyldns
synced 2025-08-22 02:02:14 +00:00
Merge pull request #1335 from Aravindh-Raju/aravindhr/add-batch-status-filter
Save batch changes with their status
This commit is contained in:
commit
85bd74ac1a
@ -34,6 +34,7 @@ import vinyldns.core.domain.auth.AuthPrincipal
|
||||
import vinyldns.core.domain.batch.BatchChangeApprovalStatus.BatchChangeApprovalStatus
|
||||
import vinyldns.core.domain.batch._
|
||||
import vinyldns.core.domain.batch.BatchChangeApprovalStatus._
|
||||
import vinyldns.core.domain.batch.BatchChangeStatus.BatchChangeStatus
|
||||
import vinyldns.core.domain.{CnameAtZoneApexError, SingleChangeError, UserIsNotAuthorizedError, ZoneDiscoveryError}
|
||||
import vinyldns.core.domain.membership.{Group, GroupRepository, ListUsersResults, User, UserRepository}
|
||||
import vinyldns.core.domain.record.RecordType._
|
||||
@ -442,6 +443,7 @@ class BatchChangeService(
|
||||
changes,
|
||||
batchChangeInput.ownerGroupId,
|
||||
BatchChangeApprovalStatus.PendingReview,
|
||||
BatchChangeStatus.PendingReview,
|
||||
scheduledTime = batchChangeInput.scheduledTime
|
||||
).asRight
|
||||
}
|
||||
@ -457,6 +459,7 @@ class BatchChangeService(
|
||||
changes,
|
||||
batchChangeInput.ownerGroupId,
|
||||
BatchChangeApprovalStatus.AutoApproved,
|
||||
BatchChangeStatus.PendingProcessing,
|
||||
scheduledTime = batchChangeInput.scheduledTime
|
||||
).asRight
|
||||
}
|
||||
@ -587,6 +590,7 @@ class BatchChangeService(
|
||||
startFrom: Option[Int] = None,
|
||||
maxItems: Int = 100,
|
||||
ignoreAccess: Boolean = false,
|
||||
batchStatus: Option[BatchChangeStatus] = None,
|
||||
approvalStatus: Option[BatchChangeApprovalStatus] = None
|
||||
): BatchResult[BatchChangeSummaryList] = {
|
||||
val userId = if (ignoreAccess && auth.isSystemAdmin) None else Some(auth.userId)
|
||||
@ -595,7 +599,7 @@ class BatchChangeService(
|
||||
val endDateTime = if(dateTimeEndRange.isDefined && dateTimeEndRange.get.isEmpty) None else dateTimeEndRange
|
||||
for {
|
||||
listResults <- batchChangeRepo
|
||||
.getBatchChangeSummaries(userId, submitterUserName, startDateTime, endDateTime, startFrom, maxItems, approvalStatus)
|
||||
.getBatchChangeSummaries(userId, submitterUserName, startDateTime, endDateTime, startFrom, maxItems, batchStatus, approvalStatus)
|
||||
.toBatchResult
|
||||
rsOwnerGroupIds = listResults.batchChanges.flatMap(_.ownerGroupId).toSet
|
||||
rsOwnerGroups <- groupRepository.getGroups(rsOwnerGroupIds).toBatchResult
|
||||
|
@ -19,6 +19,7 @@ package vinyldns.api.domain.batch
|
||||
import vinyldns.api.domain.batch.BatchChangeInterfaces.BatchResult
|
||||
import vinyldns.core.domain.auth.AuthPrincipal
|
||||
import vinyldns.core.domain.batch.BatchChangeApprovalStatus.BatchChangeApprovalStatus
|
||||
import vinyldns.core.domain.batch.BatchChangeStatus.BatchChangeStatus
|
||||
import vinyldns.core.domain.batch.{BatchChange, BatchChangeInfo, BatchChangeSummaryList}
|
||||
|
||||
// $COVERAGE-OFF$
|
||||
@ -39,6 +40,7 @@ trait BatchChangeServiceAlgebra {
|
||||
startFrom: Option[Int],
|
||||
maxItems: Int,
|
||||
ignoreAccess: Boolean,
|
||||
batchStatus: Option[BatchChangeStatus],
|
||||
approvalStatus: Option[BatchChangeApprovalStatus]
|
||||
): BatchResult[BatchChangeSummaryList]
|
||||
|
||||
|
@ -91,32 +91,34 @@ class BatchChangeRoute(
|
||||
) =>
|
||||
{
|
||||
val convertApprovalStatus = approvalStatus.flatMap(BatchChangeApprovalStatus.find)
|
||||
|
||||
handleRejections(invalidQueryHandler) {
|
||||
validate(
|
||||
0 < maxItems && maxItems <= MAX_ITEMS_LIMIT,
|
||||
s"maxItems was $maxItems, maxItems must be between 1 and $MAX_ITEMS_LIMIT, inclusive."
|
||||
) {
|
||||
authenticateAndExecute(
|
||||
batchChangeService.listBatchChangeSummaries(
|
||||
_,
|
||||
userName,
|
||||
dateTimeRangeStart,
|
||||
dateTimeRangeEnd,
|
||||
startFrom,
|
||||
maxItems,
|
||||
ignoreAccess,
|
||||
convertApprovalStatus
|
||||
)
|
||||
) { summaries =>
|
||||
complete(StatusCodes.OK, summaries)
|
||||
|
||||
handleRejections(invalidQueryHandler) {
|
||||
validate(
|
||||
0 < maxItems && maxItems <= MAX_ITEMS_LIMIT,
|
||||
s"maxItems was $maxItems, maxItems must be between 1 and $MAX_ITEMS_LIMIT, inclusive."
|
||||
) {
|
||||
authenticateAndExecute(
|
||||
batchChangeService.listBatchChangeSummaries(
|
||||
_,
|
||||
userName,
|
||||
dateTimeRangeStart,
|
||||
dateTimeRangeEnd,
|
||||
startFrom,
|
||||
maxItems,
|
||||
ignoreAccess,
|
||||
// TODO: Update batch status from None to its actual value when the feature is ready for release
|
||||
None,
|
||||
convertApprovalStatus
|
||||
)
|
||||
) { summaries =>
|
||||
complete(StatusCodes.OK, summaries)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
} ~
|
||||
} ~
|
||||
path("zones" / "batchrecordchanges" / Segment) { id =>
|
||||
(get & monitor("Endpoint.getBatchChange")) {
|
||||
authenticateAndExecute(batchChangeService.getBatchChange(id, _)) { chg =>
|
||||
|
@ -1041,6 +1041,7 @@ class BatchChangeServiceSpec
|
||||
List(),
|
||||
ownerGroupId = Some(okGroup.id),
|
||||
BatchChangeApprovalStatus.ManuallyApproved,
|
||||
BatchChangeStatus.PendingProcessing,
|
||||
Some(superUser.id),
|
||||
None,
|
||||
Some(Instant.now.truncatedTo(ChronoUnit.MILLIS))
|
||||
@ -2538,6 +2539,48 @@ class BatchChangeServiceSpec
|
||||
result.batchChanges(0).ownerGroupId shouldBe Some("no-existo")
|
||||
result.batchChanges(0).ownerGroupName shouldBe None
|
||||
}
|
||||
|
||||
"return list of batchChangeSummaries filtered by batch change status" in {
|
||||
val batchChangeOne =
|
||||
BatchChange(
|
||||
auth.userId,
|
||||
auth.signedInUser.userName,
|
||||
None,
|
||||
Instant.now.truncatedTo(ChronoUnit.MILLIS),
|
||||
List(),
|
||||
approvalStatus = BatchChangeApprovalStatus.PendingReview,
|
||||
batchStatus = BatchChangeStatus.PendingReview
|
||||
)
|
||||
batchChangeRepo.save(batchChangeOne)
|
||||
|
||||
val batchChangeTwo = BatchChange(
|
||||
auth.userId,
|
||||
auth.signedInUser.userName,
|
||||
None,
|
||||
Instant.ofEpochMilli(Instant.now.truncatedTo(ChronoUnit.MILLIS).toEpochMilli + 1000),
|
||||
List(),
|
||||
approvalStatus = BatchChangeApprovalStatus.AutoApproved,
|
||||
batchStatus = BatchChangeStatus.PendingProcessing
|
||||
)
|
||||
batchChangeRepo.save(batchChangeTwo)
|
||||
|
||||
val result =
|
||||
underTest
|
||||
.listBatchChangeSummaries(
|
||||
auth,
|
||||
batchStatus = Some(BatchChangeStatus.PendingReview)
|
||||
)
|
||||
.value.unsafeRunSync().toOption.get
|
||||
|
||||
result.maxItems shouldBe 100
|
||||
result.nextId shouldBe None
|
||||
result.startFrom shouldBe None
|
||||
result.ignoreAccess shouldBe false
|
||||
result.batchStatus shouldBe Some(BatchChangeStatus.PendingReview)
|
||||
|
||||
result.batchChanges.length shouldBe 1
|
||||
result.batchChanges(0).status shouldBe batchChangeOne.batchStatus
|
||||
}
|
||||
}
|
||||
|
||||
"getOwnerGroup" should {
|
||||
@ -2676,6 +2719,7 @@ class BatchChangeServiceSpec
|
||||
List(singleChangeGood, singleChangeNR),
|
||||
Some(authGrp.id),
|
||||
BatchChangeApprovalStatus.ManuallyApproved,
|
||||
BatchChangeStatus.PendingProcessing,
|
||||
Some("reviewer_id"),
|
||||
Some("approved"),
|
||||
Some(Instant.now.truncatedTo(ChronoUnit.MILLIS))
|
||||
|
@ -100,6 +100,7 @@ class EmailNotifierSpec
|
||||
changes,
|
||||
None,
|
||||
BatchChangeApprovalStatus.AutoApproved,
|
||||
BatchChangeStatus.PendingProcessing,
|
||||
None,
|
||||
None,
|
||||
None,
|
||||
|
@ -27,15 +27,10 @@ import org.mockito.Matchers._
|
||||
import org.mockito.Mockito._
|
||||
import org.mockito.ArgumentCaptor
|
||||
import cats.effect.IO
|
||||
import vinyldns.core.domain.batch.BatchChange
|
||||
import _root_.vinyldns.core.domain.batch.{BatchChange, BatchChangeApprovalStatus, BatchChangeStatus, SingleAddChange, SingleChange, SingleChangeStatus, SingleDeleteRRSetChange}
|
||||
import java.time.Instant
|
||||
import vinyldns.core.domain.batch.BatchChangeApprovalStatus
|
||||
import vinyldns.core.domain.batch.SingleChange
|
||||
import vinyldns.core.domain.batch.SingleAddChange
|
||||
import vinyldns.core.domain.batch.SingleDeleteRRSetChange
|
||||
import vinyldns.core.domain.record.RecordType
|
||||
import vinyldns.core.domain.record.AData
|
||||
import _root_.vinyldns.core.domain.batch.SingleChangeStatus
|
||||
import com.typesafe.config.Config
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import scala.collection.JavaConverters._
|
||||
@ -70,6 +65,7 @@ class SnsNotifierSpec
|
||||
changes,
|
||||
None,
|
||||
BatchChangeApprovalStatus.AutoApproved,
|
||||
BatchChangeStatus.PendingProcessing,
|
||||
None,
|
||||
None,
|
||||
None,
|
||||
|
@ -24,6 +24,7 @@ import scala.collection.concurrent
|
||||
import cats.effect._
|
||||
import cats.implicits._
|
||||
import vinyldns.core.domain.batch.BatchChangeApprovalStatus.BatchChangeApprovalStatus
|
||||
import vinyldns.core.domain.batch.BatchChangeStatus.BatchChangeStatus
|
||||
|
||||
class InMemoryBatchChangeRepository extends BatchChangeRepository {
|
||||
|
||||
@ -38,6 +39,7 @@ class InMemoryBatchChangeRepository extends BatchChangeRepository {
|
||||
ownerGroupId: Option[String],
|
||||
id: String,
|
||||
approvalStatus: BatchChangeApprovalStatus,
|
||||
batchStatus: BatchChangeStatus,
|
||||
reviewerId: Option[String],
|
||||
reviewComment: Option[String],
|
||||
reviewTimestamp: Option[Instant]
|
||||
@ -53,6 +55,7 @@ class InMemoryBatchChangeRepository extends BatchChangeRepository {
|
||||
batchChange.ownerGroupId,
|
||||
batchChange.id,
|
||||
batchChange.approvalStatus,
|
||||
batchChange.batchStatus,
|
||||
batchChange.reviewerId,
|
||||
batchChange.reviewComment,
|
||||
batchChange.reviewTimestamp
|
||||
@ -87,6 +90,7 @@ class InMemoryBatchChangeRepository extends BatchChangeRepository {
|
||||
singleChangesFromRepo,
|
||||
sc.ownerGroupId,
|
||||
sc.approvalStatus,
|
||||
sc.batchStatus,
|
||||
sc.reviewerId,
|
||||
sc.reviewComment,
|
||||
sc.reviewTimestamp,
|
||||
@ -118,6 +122,7 @@ class InMemoryBatchChangeRepository extends BatchChangeRepository {
|
||||
dateTimeEndRange: Option[String] = None,
|
||||
startFrom: Option[Int] = None,
|
||||
maxItems: Int = 100,
|
||||
batchStatus: Option[BatchChangeStatus] = None,
|
||||
approvalStatus: Option[BatchChangeApprovalStatus] = None
|
||||
): IO[BatchChangeSummaryList] = {
|
||||
|
||||
@ -130,6 +135,7 @@ class InMemoryBatchChangeRepository extends BatchChangeRepository {
|
||||
|
||||
val userBatchChanges = batches.values.toList
|
||||
.filter(b => userId.forall(_ == b.userId))
|
||||
.filter(bs => batchStatus.forall(_ == bs.batchStatus))
|
||||
.filter(bu => userName.forall(_ == bu.userName))
|
||||
.filter(bdtsi => startInstant.forall(_.isBefore(bdtsi.createdTimestamp)))
|
||||
.filter(bdtei => endInstant.forall(_.isAfter(bdtei.createdTimestamp)))
|
||||
@ -146,6 +152,7 @@ class InMemoryBatchChangeRepository extends BatchChangeRepository {
|
||||
changes,
|
||||
sc.ownerGroupId,
|
||||
sc.approvalStatus,
|
||||
sc.batchStatus,
|
||||
sc.reviewerId,
|
||||
sc.reviewComment,
|
||||
sc.reviewTimestamp,
|
||||
@ -165,6 +172,7 @@ class InMemoryBatchChangeRepository extends BatchChangeRepository {
|
||||
nextId = nextId,
|
||||
maxItems = maxItems,
|
||||
ignoreAccess = ignoreAccess,
|
||||
batchStatus = batchStatus,
|
||||
approvalStatus = approvalStatus,
|
||||
userName = userName,
|
||||
dateTimeStartRange = dateTimeStartRange,
|
||||
|
@ -482,6 +482,7 @@ class BatchChangeJsonProtocolSpec
|
||||
List(add, delete),
|
||||
None,
|
||||
BatchChangeApprovalStatus.PendingReview,
|
||||
BatchChangeStatus.PendingReview,
|
||||
None,
|
||||
None,
|
||||
None,
|
||||
@ -616,6 +617,7 @@ class BatchChangeJsonProtocolSpec
|
||||
List(add, delete),
|
||||
None,
|
||||
BatchChangeApprovalStatus.PendingReview,
|
||||
BatchChangeStatus.PendingReview,
|
||||
None,
|
||||
None,
|
||||
None,
|
||||
|
@ -38,6 +38,7 @@ import vinyldns.core.TestMembershipData._
|
||||
import vinyldns.core.domain.BatchChangeIsEmpty
|
||||
import vinyldns.core.domain.auth.AuthPrincipal
|
||||
import vinyldns.core.domain.batch.BatchChangeApprovalStatus.BatchChangeApprovalStatus
|
||||
import vinyldns.core.domain.batch.BatchChangeStatus.BatchChangeStatus
|
||||
import vinyldns.core.domain.batch._
|
||||
import vinyldns.core.domain.record.RecordType._
|
||||
import vinyldns.core.domain.record._
|
||||
@ -132,6 +133,7 @@ class BatchChangeRoutingSpec()
|
||||
),
|
||||
ownerGroupId,
|
||||
approvalStatus,
|
||||
BatchChangeStatus.PendingProcessing,
|
||||
None,
|
||||
None,
|
||||
None,
|
||||
@ -316,6 +318,7 @@ class BatchChangeRoutingSpec()
|
||||
startFrom: Option[Int],
|
||||
maxItems: Int,
|
||||
ignoreAccess: Boolean = false,
|
||||
batchStatus: Option[BatchChangeStatus] = None,
|
||||
approvalStatus: Option[BatchChangeApprovalStatus] = None
|
||||
): EitherT[IO, BatchChangeErrorResponse, BatchChangeSummaryList] =
|
||||
if (auth.userId == okAuth.userId)
|
||||
|
@ -31,6 +31,7 @@ case class BatchChange(
|
||||
changes: List[SingleChange],
|
||||
ownerGroupId: Option[String] = None,
|
||||
approvalStatus: BatchChangeApprovalStatus,
|
||||
batchStatus: BatchChangeStatus = BatchChangeStatus.PendingProcessing,
|
||||
reviewerId: Option[String] = None,
|
||||
reviewComment: Option[String] = None,
|
||||
reviewTimestamp: Option[Instant] = None,
|
||||
@ -92,6 +93,12 @@ object BatchChangeStatus extends Enumeration {
|
||||
case _ => BatchChangeStatus.Complete
|
||||
}
|
||||
}
|
||||
|
||||
private val valueMap =
|
||||
BatchChangeStatus.values.map(v => v.toString.toLowerCase -> v).toMap
|
||||
|
||||
def find(status: String): Option[BatchChangeStatus] =
|
||||
valueMap.get(status.toLowerCase)
|
||||
}
|
||||
|
||||
object BatchChangeApprovalStatus extends Enumeration {
|
||||
|
@ -18,6 +18,7 @@ package vinyldns.core.domain.batch
|
||||
|
||||
import cats.effect.IO
|
||||
import vinyldns.core.domain.batch.BatchChangeApprovalStatus.BatchChangeApprovalStatus
|
||||
import vinyldns.core.domain.batch.BatchChangeStatus.BatchChangeStatus
|
||||
import vinyldns.core.repository.Repository
|
||||
|
||||
// $COVERAGE-OFF$
|
||||
@ -34,6 +35,7 @@ trait BatchChangeRepository extends Repository {
|
||||
dateTimeEndRange: Option[String] = None,
|
||||
startFrom: Option[Int] = None,
|
||||
maxItems: Int = 100,
|
||||
batchStatus: Option[BatchChangeStatus] = None,
|
||||
approvalStatus: Option[BatchChangeApprovalStatus] = None
|
||||
): IO[BatchChangeSummaryList]
|
||||
|
||||
|
@ -91,6 +91,7 @@ case class BatchChangeSummaryList(
|
||||
nextId: Option[Int] = None,
|
||||
maxItems: Int = 100,
|
||||
ignoreAccess: Boolean = false,
|
||||
batchStatus: Option[BatchChangeStatus] = None,
|
||||
approvalStatus: Option[BatchChangeApprovalStatus] = None,
|
||||
userName: Option[String] = None,
|
||||
dateTimeStartRange: Option[String] = None,
|
||||
|
@ -47,6 +47,7 @@ class BatchChangeSummarySpec extends AnyWordSpec with Matchers {
|
||||
List(pendingChange, failedChange, completeChange),
|
||||
Some("groupId"),
|
||||
BatchChangeApprovalStatus.AutoApproved,
|
||||
BatchChangeStatus.PendingProcessing,
|
||||
None,
|
||||
None,
|
||||
None,
|
||||
|
@ -103,6 +103,7 @@ class MySqlBatchChangeRepositoryIntegrationSpec
|
||||
changes,
|
||||
Some(UUID.randomUUID().toString),
|
||||
BatchChangeApprovalStatus.AutoApproved,
|
||||
BatchChangeStatus.PendingProcessing,
|
||||
Some(UUID.randomUUID().toString),
|
||||
Some("review comment"),
|
||||
Some(Instant.now.truncatedTo(ChronoUnit.MILLIS).plusSeconds(2))
|
||||
@ -150,6 +151,8 @@ class MySqlBatchChangeRepositoryIntegrationSpec
|
||||
createdTimestamp = timeBase.plusMillis(10000000),
|
||||
approvalStatus = BatchChangeApprovalStatus.ManuallyRejected
|
||||
)
|
||||
val change_six: BatchChange =
|
||||
completeBatchChange.copy(createdTimestamp = timeBase.plusMillis(2000), ownerGroupId = None)
|
||||
}
|
||||
|
||||
import TestData._
|
||||
@ -453,7 +456,8 @@ class MySqlBatchChangeRepositoryIntegrationSpec
|
||||
} yield (updated, saved)
|
||||
|
||||
val (retrieved, saved) = f.unsafeRunSync
|
||||
retrieved shouldBe saved
|
||||
// Batch Change Status will be updated once saved
|
||||
retrieved.map(x => x.copy(batchStatus = BatchChangeStatus.Complete)) shouldBe saved
|
||||
}
|
||||
|
||||
"return no batch when single changes list is empty" in {
|
||||
@ -894,6 +898,59 @@ class MySqlBatchChangeRepositoryIntegrationSpec
|
||||
batchChangeSummaries.batchChanges shouldBe empty
|
||||
}
|
||||
|
||||
"get batch change summaries by batch change status" in {
|
||||
val f =
|
||||
for {
|
||||
_ <- repo.save(change_one)
|
||||
_ <- repo.save(change_two)
|
||||
_ <- repo.save(change_three)
|
||||
_ <- repo.save(change_four)
|
||||
_ <- repo.save(otherUserBatchChange)
|
||||
|
||||
retrieved <- repo.getBatchChangeSummaries(None, batchStatus = Some(BatchChangeStatus.PartialFailure))
|
||||
} yield retrieved
|
||||
|
||||
// from most recent descending
|
||||
val expectedChanges = BatchChangeSummaryList(
|
||||
List(
|
||||
BatchChangeSummary(change_four)
|
||||
)
|
||||
)
|
||||
|
||||
areSame(f.unsafeRunSync(), expectedChanges)
|
||||
}
|
||||
|
||||
"get batch change summaries by user ID, approval status and batch change status" in {
|
||||
val f =
|
||||
for {
|
||||
_ <- repo.save(change_one)
|
||||
_ <- repo.save(change_two)
|
||||
_ <- repo.save(change_three)
|
||||
_ <- repo.save(change_four)
|
||||
_ <- repo.save(otherUserBatchChange)
|
||||
|
||||
retrieved <- repo.getBatchChangeSummaries(
|
||||
Some(pendingBatchChange.userId),
|
||||
approvalStatus = Some(BatchChangeApprovalStatus.AutoApproved),
|
||||
batchStatus = Some(BatchChangeStatus.Complete)
|
||||
)
|
||||
} yield retrieved
|
||||
|
||||
// from most recent descending
|
||||
val expectedChanges = BatchChangeSummaryList(
|
||||
List(
|
||||
BatchChangeSummary(change_two)
|
||||
)
|
||||
)
|
||||
|
||||
areSame(f.unsafeRunSync(), expectedChanges)
|
||||
}
|
||||
|
||||
"return empty list if a batch change summary is not found for a batch change status" in {
|
||||
val batchChangeSummaries = repo.getBatchChangeSummaries(None, batchStatus = Some(BatchChangeStatus.Scheduled)).unsafeRunSync()
|
||||
batchChangeSummaries.batchChanges shouldBe empty
|
||||
}
|
||||
|
||||
"properly status check (pending)" in {
|
||||
val chg = randomBatchChange(
|
||||
List(
|
||||
|
@ -0,0 +1,5 @@
|
||||
CREATE SCHEMA IF NOT EXISTS ${dbName};
|
||||
|
||||
USE ${dbName};
|
||||
|
||||
ALTER TABLE batch_change ADD batch_status VARCHAR(25) NOT NULL;
|
@ -24,6 +24,7 @@ import java.time.Instant
|
||||
import org.slf4j.LoggerFactory
|
||||
import scalikejdbc._
|
||||
import vinyldns.core.domain.batch.BatchChangeApprovalStatus.BatchChangeApprovalStatus
|
||||
import vinyldns.core.domain.batch.BatchChangeStatus.BatchChangeStatus
|
||||
import vinyldns.core.domain.batch._
|
||||
import vinyldns.core.protobuf.{BatchChangeProtobufConversions, SingleChangeType}
|
||||
import vinyldns.core.route.Monitored
|
||||
@ -48,11 +49,11 @@ class MySqlBatchChangeRepository
|
||||
private final val PUT_BATCH_CHANGE =
|
||||
sql"""
|
||||
| INSERT INTO batch_change(id, user_id, user_name, created_time, comments, owner_group_id,
|
||||
| approval_status, reviewer_id, review_comment, review_timestamp,
|
||||
| approval_status, batch_status, reviewer_id, review_comment, review_timestamp,
|
||||
| scheduled_time, cancelled_timestamp)
|
||||
| VALUES ({id}, {userId}, {userName}, {createdTime}, {comments}, {ownerGroupId}, {approvalStatus},
|
||||
| {reviewerId}, {reviewComment}, {reviewTimestamp}, {scheduledTime}, {cancelledTimestamp})
|
||||
|ON DUPLICATE KEY UPDATE comments={comments}, owner_group_id={ownerGroupId}, approval_status={approvalStatus},
|
||||
| {batchStatus}, {reviewerId}, {reviewComment}, {reviewTimestamp}, {scheduledTime}, {cancelledTimestamp})
|
||||
|ON DUPLICATE KEY UPDATE comments={comments}, owner_group_id={ownerGroupId}, approval_status={approvalStatus}, batch_status={batchStatus},
|
||||
| reviewer_id={reviewerId}, review_comment={reviewComment}, review_timestamp={reviewTimestamp},
|
||||
| scheduled_time={scheduledTime}, cancelled_timestamp={cancelledTimestamp}
|
||||
""".stripMargin
|
||||
@ -70,14 +71,14 @@ class MySqlBatchChangeRepository
|
||||
private final val GET_BATCH_CHANGE_METADATA =
|
||||
sql"""
|
||||
|SELECT user_id, user_name, created_time, comments, owner_group_id,
|
||||
| approval_status, reviewer_id, review_comment, review_timestamp, scheduled_time, cancelled_timestamp
|
||||
| approval_status, batch_status, reviewer_id, review_comment, review_timestamp, scheduled_time, cancelled_timestamp
|
||||
| FROM batch_change bc
|
||||
| WHERE bc.id = ?
|
||||
""".stripMargin
|
||||
|
||||
private final val GET_BATCH_CHANGE_METADATA_FROM_SINGLE_CHANGE =
|
||||
sql"""
|
||||
|SELECT bc.id, bc.user_id, bc.user_name, bc.created_time, bc.comments, bc.owner_group_id, bc.approval_status,
|
||||
|SELECT bc.id, bc.user_id, bc.user_name, bc.created_time, bc.comments, bc.owner_group_id, bc.approval_status, bc.batch_status,
|
||||
| bc.reviewer_id, bc.review_comment, bc.review_timestamp, bc.scheduled_time, bc.cancelled_timestamp
|
||||
| FROM batch_change bc
|
||||
| JOIN (SELECT id, batch_change_id from single_change where id = ?) sc
|
||||
@ -86,13 +87,13 @@ class MySqlBatchChangeRepository
|
||||
|
||||
private final val GET_BATCH_CHANGE_SUMMARY_BASE =
|
||||
"""
|
||||
|SELECT batch_change_page.id, user_id, user_name, created_time, comments, owner_group_id, approval_status, reviewer_id,
|
||||
|SELECT batch_change_page.id, user_id, user_name, created_time, comments, owner_group_id, approval_status, batch_status, reviewer_id,
|
||||
| review_comment, review_timestamp, scheduled_time, cancelled_timestamp,
|
||||
| SUM(CASE WHEN sc.status LIKE 'Failed' OR sc.status LIKE 'Rejected' THEN 1 ELSE 0 END) AS fail_count,
|
||||
| SUM(CASE WHEN sc.status LIKE 'Pending' OR sc.status LIKE 'NeedsReview' THEN 1 ELSE 0 END) AS pending_count,
|
||||
| SUM(CASE sc.status WHEN 'Complete' THEN 1 ELSE 0 END) AS complete_count,
|
||||
| SUM(CASE sc.status WHEN 'Cancelled' THEN 1 ELSE 0 END) AS cancelled_count
|
||||
| FROM (SELECT bc.id, bc.user_id, bc.user_name, bc.created_time, bc.comments, bc.owner_group_id, bc.approval_status,
|
||||
| FROM (SELECT bc.id, bc.user_id, bc.user_name, bc.created_time, bc.comments, bc.owner_group_id, bc.approval_status, bc.batch_status,
|
||||
| bc.reviewer_id, bc.review_comment, bc.review_timestamp, bc.scheduled_time, bc.cancelled_timestamp
|
||||
| FROM batch_change bc
|
||||
""".stripMargin
|
||||
@ -124,6 +125,13 @@ class MySqlBatchChangeRepository
|
||||
| WHERE id={id}
|
||||
""".stripMargin
|
||||
|
||||
private final val UPDATE_BATCH_CHANGE =
|
||||
sql"""
|
||||
|UPDATE batch_change
|
||||
| SET batch_status={batchStatus}
|
||||
| WHERE id={id}
|
||||
""".stripMargin
|
||||
|
||||
def save(batch: BatchChange): IO[BatchChange] =
|
||||
monitor("repo.BatchChangeJDBC.save") {
|
||||
IO {
|
||||
@ -181,6 +189,24 @@ class MySqlBatchChangeRepository
|
||||
.apply()
|
||||
batchMeta.copy(changes = changes)
|
||||
}
|
||||
|
||||
var failCount = 0
|
||||
var pendingCount = 0
|
||||
var completeCount = 0
|
||||
var cancelledCount = 0
|
||||
|
||||
singleChanges.foreach { sc =>
|
||||
if (sc.status.toString == "Failed" || sc.status.toString == "Rejected") {
|
||||
failCount += 1
|
||||
} else if (sc.status.toString == "Pending" || sc.status.toString == "NeedsReview") {
|
||||
pendingCount += 1
|
||||
} else if (sc.status.toString == "Complete") {
|
||||
completeCount += 1
|
||||
} else {
|
||||
cancelledCount += 1
|
||||
}
|
||||
}
|
||||
|
||||
monitor("repo.BatchChangeJDBC.updateSingleChanges") {
|
||||
IO {
|
||||
logger.info(s"Updating single change status: ${singleChanges.map(ch => (ch.id, ch.status))}")
|
||||
@ -190,6 +216,8 @@ class MySqlBatchChangeRepository
|
||||
batchParams = singleChanges.map(convertSingleChangeToParams)
|
||||
_ = UPDATE_SINGLE_CHANGE.batchByName(batchParams: _*).apply()
|
||||
batchChange <- getBatchFromSingleChangeId(headChange.id)
|
||||
batchStatus = BatchChangeStatus.calculateBatchStatus(batchChange.approvalStatus, pendingCount > 0, failCount > 0, completeCount > 0, batchChange.scheduledTime.isDefined)
|
||||
_ = UPDATE_BATCH_CHANGE.bindByName('batchStatus -> batchStatus.toString, 'id -> batchChange.id).update().apply()
|
||||
} yield batchChange
|
||||
}}
|
||||
}
|
||||
@ -234,6 +262,7 @@ class MySqlBatchChangeRepository
|
||||
dateTimeEndRange: Option[String] = None,
|
||||
startFrom: Option[Int] = None,
|
||||
maxItems: Int = 100,
|
||||
batchStatus: Option[BatchChangeStatus],
|
||||
approvalStatus: Option[BatchChangeApprovalStatus]
|
||||
): IO[BatchChangeSummaryList] =
|
||||
monitor("repo.BatchChangeJDBC.getBatchChangeSummaries") {
|
||||
@ -245,13 +274,14 @@ class MySqlBatchChangeRepository
|
||||
|
||||
val uid = userId.map(u => s"bc.user_id = '$u'")
|
||||
val as = approvalStatus.map(a => s"bc.approval_status = '${fromApprovalStatus(a)}'")
|
||||
val bs = batchStatus.map(b => s"bc.batch_status = '${fromBatchStatus(b)}'")
|
||||
val uname = userName.map(uname => s"bc.user_name = '$uname'")
|
||||
val dtRange = if(dateTimeStartRange.isDefined && dateTimeEndRange.isDefined) {
|
||||
Some(s"(bc.created_time >= '${dateTimeStartRange.get}' AND bc.created_time <= '${dateTimeEndRange.get}')")
|
||||
} else {
|
||||
None
|
||||
}
|
||||
val opts = uid ++ as ++ uname ++ dtRange
|
||||
val opts = uid ++ as ++ bs ++ uname ++ dtRange
|
||||
|
||||
if (opts.nonEmpty) sb.append("WHERE ").append(opts.mkString(" AND "))
|
||||
|
||||
@ -267,6 +297,7 @@ class MySqlBatchChangeRepository
|
||||
val complete = res.int("complete_count")
|
||||
val cancelled = res.int("cancelled_count")
|
||||
val approvalStatus = toApprovalStatus(res.intOpt("approval_status"))
|
||||
val batchChangeStatus = toBatchChangeStatus(res.stringOpt("batch_status"))
|
||||
val schedTime =
|
||||
res.timestampOpt("scheduled_time").map(st => st.toInstant)
|
||||
val cancelledTimestamp =
|
||||
@ -277,14 +308,7 @@ class MySqlBatchChangeRepository
|
||||
Option(res.string("comments")),
|
||||
res.timestamp("created_time").toInstant,
|
||||
pending + failed + complete + cancelled,
|
||||
BatchChangeStatus
|
||||
.calculateBatchStatus(
|
||||
approvalStatus,
|
||||
pending > 0,
|
||||
failed > 0,
|
||||
complete > 0,
|
||||
schedTime.isDefined
|
||||
),
|
||||
batchChangeStatus,
|
||||
Option(res.string("owner_group_id")),
|
||||
res.string("id"),
|
||||
None,
|
||||
@ -308,6 +332,7 @@ class MySqlBatchChangeRepository
|
||||
nextId,
|
||||
maxItems,
|
||||
ignoreAccess,
|
||||
batchStatus,
|
||||
approvalStatus
|
||||
)
|
||||
}
|
||||
@ -338,6 +363,7 @@ class MySqlBatchChangeRepository
|
||||
Nil,
|
||||
result.stringOpt("owner_group_id"),
|
||||
toApprovalStatus(result.intOpt("approval_status")),
|
||||
toBatchChangeStatus(result.stringOpt("batch_status")),
|
||||
result.stringOpt("reviewer_id"),
|
||||
result.stringOpt("review_comment"),
|
||||
result.timestampOpt("review_timestamp").map(toDateTime),
|
||||
@ -350,6 +376,33 @@ class MySqlBatchChangeRepository
|
||||
private def saveBatchChange(
|
||||
batchChange: BatchChange
|
||||
)(implicit session: DBSession): BatchChange = {
|
||||
|
||||
var failCount = 0
|
||||
var pendingCount = 0
|
||||
var completeCount = 0
|
||||
var cancelledCount = 0
|
||||
|
||||
batchChange.changes.foreach { sc =>
|
||||
if (sc.status.toString == "Failed" || sc.status.toString == "Rejected") {
|
||||
failCount += 1
|
||||
} else if (sc.status.toString == "Pending" || sc.status.toString == "NeedsReview") {
|
||||
pendingCount += 1
|
||||
} else if (sc.status.toString == "Complete") {
|
||||
completeCount += 1
|
||||
} else {
|
||||
cancelledCount += 1
|
||||
}
|
||||
}
|
||||
|
||||
val batchStatus = BatchChangeStatus
|
||||
.calculateBatchStatus(
|
||||
batchChange.approvalStatus,
|
||||
pendingCount > 0,
|
||||
failCount > 0,
|
||||
completeCount > 0,
|
||||
batchChange.scheduledTime.isDefined
|
||||
)
|
||||
|
||||
PUT_BATCH_CHANGE
|
||||
.bindByName(
|
||||
Seq(
|
||||
@ -360,6 +413,7 @@ class MySqlBatchChangeRepository
|
||||
'comments -> batchChange.comments,
|
||||
'ownerGroupId -> batchChange.ownerGroupId,
|
||||
'approvalStatus -> fromApprovalStatus(batchChange.approvalStatus),
|
||||
'batchStatus -> fromBatchStatus(batchStatus),
|
||||
'reviewerId -> batchChange.reviewerId,
|
||||
'reviewComment -> batchChange.reviewComment,
|
||||
'reviewTimestamp -> batchChange.reviewTimestamp,
|
||||
@ -424,6 +478,18 @@ class MySqlBatchChangeRepository
|
||||
case BatchChangeApprovalStatus.Cancelled => 5
|
||||
}
|
||||
|
||||
def fromBatchStatus(typ: BatchChangeStatus): String =
|
||||
typ match {
|
||||
case BatchChangeStatus.Cancelled => "Cancelled"
|
||||
case BatchChangeStatus.Complete => "Complete"
|
||||
case BatchChangeStatus.Failed => "Failed"
|
||||
case BatchChangeStatus.PartialFailure => "PartialFailure"
|
||||
case BatchChangeStatus.PendingProcessing => "PendingProcessing"
|
||||
case BatchChangeStatus.PendingReview => "PendingReview"
|
||||
case BatchChangeStatus.Rejected => "Rejected"
|
||||
case BatchChangeStatus.Scheduled => "Scheduled"
|
||||
}
|
||||
|
||||
def toApprovalStatus(key: Option[Int]): BatchChangeApprovalStatus =
|
||||
key match {
|
||||
case Some(1) => BatchChangeApprovalStatus.AutoApproved
|
||||
@ -434,5 +500,19 @@ class MySqlBatchChangeRepository
|
||||
case _ => BatchChangeApprovalStatus.AutoApproved
|
||||
}
|
||||
|
||||
def toBatchChangeStatus(key: Option[String]): BatchChangeStatus = {
|
||||
key match {
|
||||
case Some("Cancelled") => BatchChangeStatus.Cancelled
|
||||
case Some("Complete") => BatchChangeStatus.Complete
|
||||
case Some("Failed") => BatchChangeStatus.Failed
|
||||
case Some("PartialFailure") => BatchChangeStatus.PartialFailure
|
||||
case Some("PendingProcessing") => BatchChangeStatus.PendingProcessing
|
||||
case Some("PendingReview") => BatchChangeStatus.PendingReview
|
||||
case Some("Rejected") => BatchChangeStatus.Rejected
|
||||
case Some("Scheduled") => BatchChangeStatus.Scheduled
|
||||
case _ => BatchChangeStatus.Complete
|
||||
}
|
||||
}
|
||||
|
||||
def toDateTime(ts: Timestamp): Instant = ts.toInstant
|
||||
}
|
||||
|
Loading…
x
Reference in New Issue
Block a user