2
0
mirror of https://github.com/VinylDNS/vinyldns synced 2025-08-22 10:10:12 +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:
Arpit Shah 2024-10-15 10:28:43 -04:00 committed by GitHub
commit 85bd74ac1a
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
16 changed files with 260 additions and 45 deletions

View File

@ -34,6 +34,7 @@ import vinyldns.core.domain.auth.AuthPrincipal
import vinyldns.core.domain.batch.BatchChangeApprovalStatus.BatchChangeApprovalStatus import vinyldns.core.domain.batch.BatchChangeApprovalStatus.BatchChangeApprovalStatus
import vinyldns.core.domain.batch._ import vinyldns.core.domain.batch._
import vinyldns.core.domain.batch.BatchChangeApprovalStatus._ 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.{CnameAtZoneApexError, SingleChangeError, UserIsNotAuthorizedError, ZoneDiscoveryError}
import vinyldns.core.domain.membership.{Group, GroupRepository, ListUsersResults, User, UserRepository} import vinyldns.core.domain.membership.{Group, GroupRepository, ListUsersResults, User, UserRepository}
import vinyldns.core.domain.record.RecordType._ import vinyldns.core.domain.record.RecordType._
@ -442,6 +443,7 @@ class BatchChangeService(
changes, changes,
batchChangeInput.ownerGroupId, batchChangeInput.ownerGroupId,
BatchChangeApprovalStatus.PendingReview, BatchChangeApprovalStatus.PendingReview,
BatchChangeStatus.PendingReview,
scheduledTime = batchChangeInput.scheduledTime scheduledTime = batchChangeInput.scheduledTime
).asRight ).asRight
} }
@ -457,6 +459,7 @@ class BatchChangeService(
changes, changes,
batchChangeInput.ownerGroupId, batchChangeInput.ownerGroupId,
BatchChangeApprovalStatus.AutoApproved, BatchChangeApprovalStatus.AutoApproved,
BatchChangeStatus.PendingProcessing,
scheduledTime = batchChangeInput.scheduledTime scheduledTime = batchChangeInput.scheduledTime
).asRight ).asRight
} }
@ -587,6 +590,7 @@ class BatchChangeService(
startFrom: Option[Int] = None, startFrom: Option[Int] = None,
maxItems: Int = 100, maxItems: Int = 100,
ignoreAccess: Boolean = false, ignoreAccess: Boolean = false,
batchStatus: Option[BatchChangeStatus] = None,
approvalStatus: Option[BatchChangeApprovalStatus] = None approvalStatus: Option[BatchChangeApprovalStatus] = None
): BatchResult[BatchChangeSummaryList] = { ): BatchResult[BatchChangeSummaryList] = {
val userId = if (ignoreAccess && auth.isSystemAdmin) None else Some(auth.userId) 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 val endDateTime = if(dateTimeEndRange.isDefined && dateTimeEndRange.get.isEmpty) None else dateTimeEndRange
for { for {
listResults <- batchChangeRepo listResults <- batchChangeRepo
.getBatchChangeSummaries(userId, submitterUserName, startDateTime, endDateTime, startFrom, maxItems, approvalStatus) .getBatchChangeSummaries(userId, submitterUserName, startDateTime, endDateTime, startFrom, maxItems, batchStatus, approvalStatus)
.toBatchResult .toBatchResult
rsOwnerGroupIds = listResults.batchChanges.flatMap(_.ownerGroupId).toSet rsOwnerGroupIds = listResults.batchChanges.flatMap(_.ownerGroupId).toSet
rsOwnerGroups <- groupRepository.getGroups(rsOwnerGroupIds).toBatchResult rsOwnerGroups <- groupRepository.getGroups(rsOwnerGroupIds).toBatchResult

View File

@ -19,6 +19,7 @@ package vinyldns.api.domain.batch
import vinyldns.api.domain.batch.BatchChangeInterfaces.BatchResult import vinyldns.api.domain.batch.BatchChangeInterfaces.BatchResult
import vinyldns.core.domain.auth.AuthPrincipal import vinyldns.core.domain.auth.AuthPrincipal
import vinyldns.core.domain.batch.BatchChangeApprovalStatus.BatchChangeApprovalStatus import vinyldns.core.domain.batch.BatchChangeApprovalStatus.BatchChangeApprovalStatus
import vinyldns.core.domain.batch.BatchChangeStatus.BatchChangeStatus
import vinyldns.core.domain.batch.{BatchChange, BatchChangeInfo, BatchChangeSummaryList} import vinyldns.core.domain.batch.{BatchChange, BatchChangeInfo, BatchChangeSummaryList}
// $COVERAGE-OFF$ // $COVERAGE-OFF$
@ -39,6 +40,7 @@ trait BatchChangeServiceAlgebra {
startFrom: Option[Int], startFrom: Option[Int],
maxItems: Int, maxItems: Int,
ignoreAccess: Boolean, ignoreAccess: Boolean,
batchStatus: Option[BatchChangeStatus],
approvalStatus: Option[BatchChangeApprovalStatus] approvalStatus: Option[BatchChangeApprovalStatus]
): BatchResult[BatchChangeSummaryList] ): BatchResult[BatchChangeSummaryList]

View File

@ -106,6 +106,8 @@ class BatchChangeRoute(
startFrom, startFrom,
maxItems, maxItems,
ignoreAccess, ignoreAccess,
// TODO: Update batch status from None to its actual value when the feature is ready for release
None,
convertApprovalStatus convertApprovalStatus
) )
) { summaries => ) { summaries =>

View File

@ -1041,6 +1041,7 @@ class BatchChangeServiceSpec
List(), List(),
ownerGroupId = Some(okGroup.id), ownerGroupId = Some(okGroup.id),
BatchChangeApprovalStatus.ManuallyApproved, BatchChangeApprovalStatus.ManuallyApproved,
BatchChangeStatus.PendingProcessing,
Some(superUser.id), Some(superUser.id),
None, None,
Some(Instant.now.truncatedTo(ChronoUnit.MILLIS)) Some(Instant.now.truncatedTo(ChronoUnit.MILLIS))
@ -2538,6 +2539,48 @@ class BatchChangeServiceSpec
result.batchChanges(0).ownerGroupId shouldBe Some("no-existo") result.batchChanges(0).ownerGroupId shouldBe Some("no-existo")
result.batchChanges(0).ownerGroupName shouldBe None 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 { "getOwnerGroup" should {
@ -2676,6 +2719,7 @@ class BatchChangeServiceSpec
List(singleChangeGood, singleChangeNR), List(singleChangeGood, singleChangeNR),
Some(authGrp.id), Some(authGrp.id),
BatchChangeApprovalStatus.ManuallyApproved, BatchChangeApprovalStatus.ManuallyApproved,
BatchChangeStatus.PendingProcessing,
Some("reviewer_id"), Some("reviewer_id"),
Some("approved"), Some("approved"),
Some(Instant.now.truncatedTo(ChronoUnit.MILLIS)) Some(Instant.now.truncatedTo(ChronoUnit.MILLIS))

View File

@ -100,6 +100,7 @@ class EmailNotifierSpec
changes, changes,
None, None,
BatchChangeApprovalStatus.AutoApproved, BatchChangeApprovalStatus.AutoApproved,
BatchChangeStatus.PendingProcessing,
None, None,
None, None,
None, None,

View File

@ -27,15 +27,10 @@ import org.mockito.Matchers._
import org.mockito.Mockito._ import org.mockito.Mockito._
import org.mockito.ArgumentCaptor import org.mockito.ArgumentCaptor
import cats.effect.IO 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 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.RecordType
import vinyldns.core.domain.record.AData import vinyldns.core.domain.record.AData
import _root_.vinyldns.core.domain.batch.SingleChangeStatus
import com.typesafe.config.Config import com.typesafe.config.Config
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
@ -70,6 +65,7 @@ class SnsNotifierSpec
changes, changes,
None, None,
BatchChangeApprovalStatus.AutoApproved, BatchChangeApprovalStatus.AutoApproved,
BatchChangeStatus.PendingProcessing,
None, None,
None, None,
None, None,

View File

@ -24,6 +24,7 @@ import scala.collection.concurrent
import cats.effect._ import cats.effect._
import cats.implicits._ import cats.implicits._
import vinyldns.core.domain.batch.BatchChangeApprovalStatus.BatchChangeApprovalStatus import vinyldns.core.domain.batch.BatchChangeApprovalStatus.BatchChangeApprovalStatus
import vinyldns.core.domain.batch.BatchChangeStatus.BatchChangeStatus
class InMemoryBatchChangeRepository extends BatchChangeRepository { class InMemoryBatchChangeRepository extends BatchChangeRepository {
@ -38,6 +39,7 @@ class InMemoryBatchChangeRepository extends BatchChangeRepository {
ownerGroupId: Option[String], ownerGroupId: Option[String],
id: String, id: String,
approvalStatus: BatchChangeApprovalStatus, approvalStatus: BatchChangeApprovalStatus,
batchStatus: BatchChangeStatus,
reviewerId: Option[String], reviewerId: Option[String],
reviewComment: Option[String], reviewComment: Option[String],
reviewTimestamp: Option[Instant] reviewTimestamp: Option[Instant]
@ -53,6 +55,7 @@ class InMemoryBatchChangeRepository extends BatchChangeRepository {
batchChange.ownerGroupId, batchChange.ownerGroupId,
batchChange.id, batchChange.id,
batchChange.approvalStatus, batchChange.approvalStatus,
batchChange.batchStatus,
batchChange.reviewerId, batchChange.reviewerId,
batchChange.reviewComment, batchChange.reviewComment,
batchChange.reviewTimestamp batchChange.reviewTimestamp
@ -87,6 +90,7 @@ class InMemoryBatchChangeRepository extends BatchChangeRepository {
singleChangesFromRepo, singleChangesFromRepo,
sc.ownerGroupId, sc.ownerGroupId,
sc.approvalStatus, sc.approvalStatus,
sc.batchStatus,
sc.reviewerId, sc.reviewerId,
sc.reviewComment, sc.reviewComment,
sc.reviewTimestamp, sc.reviewTimestamp,
@ -118,6 +122,7 @@ class InMemoryBatchChangeRepository extends BatchChangeRepository {
dateTimeEndRange: Option[String] = None, dateTimeEndRange: Option[String] = None,
startFrom: Option[Int] = None, startFrom: Option[Int] = None,
maxItems: Int = 100, maxItems: Int = 100,
batchStatus: Option[BatchChangeStatus] = None,
approvalStatus: Option[BatchChangeApprovalStatus] = None approvalStatus: Option[BatchChangeApprovalStatus] = None
): IO[BatchChangeSummaryList] = { ): IO[BatchChangeSummaryList] = {
@ -130,6 +135,7 @@ class InMemoryBatchChangeRepository extends BatchChangeRepository {
val userBatchChanges = batches.values.toList val userBatchChanges = batches.values.toList
.filter(b => userId.forall(_ == b.userId)) .filter(b => userId.forall(_ == b.userId))
.filter(bs => batchStatus.forall(_ == bs.batchStatus))
.filter(bu => userName.forall(_ == bu.userName)) .filter(bu => userName.forall(_ == bu.userName))
.filter(bdtsi => startInstant.forall(_.isBefore(bdtsi.createdTimestamp))) .filter(bdtsi => startInstant.forall(_.isBefore(bdtsi.createdTimestamp)))
.filter(bdtei => endInstant.forall(_.isAfter(bdtei.createdTimestamp))) .filter(bdtei => endInstant.forall(_.isAfter(bdtei.createdTimestamp)))
@ -146,6 +152,7 @@ class InMemoryBatchChangeRepository extends BatchChangeRepository {
changes, changes,
sc.ownerGroupId, sc.ownerGroupId,
sc.approvalStatus, sc.approvalStatus,
sc.batchStatus,
sc.reviewerId, sc.reviewerId,
sc.reviewComment, sc.reviewComment,
sc.reviewTimestamp, sc.reviewTimestamp,
@ -165,6 +172,7 @@ class InMemoryBatchChangeRepository extends BatchChangeRepository {
nextId = nextId, nextId = nextId,
maxItems = maxItems, maxItems = maxItems,
ignoreAccess = ignoreAccess, ignoreAccess = ignoreAccess,
batchStatus = batchStatus,
approvalStatus = approvalStatus, approvalStatus = approvalStatus,
userName = userName, userName = userName,
dateTimeStartRange = dateTimeStartRange, dateTimeStartRange = dateTimeStartRange,

View File

@ -482,6 +482,7 @@ class BatchChangeJsonProtocolSpec
List(add, delete), List(add, delete),
None, None,
BatchChangeApprovalStatus.PendingReview, BatchChangeApprovalStatus.PendingReview,
BatchChangeStatus.PendingReview,
None, None,
None, None,
None, None,
@ -616,6 +617,7 @@ class BatchChangeJsonProtocolSpec
List(add, delete), List(add, delete),
None, None,
BatchChangeApprovalStatus.PendingReview, BatchChangeApprovalStatus.PendingReview,
BatchChangeStatus.PendingReview,
None, None,
None, None,
None, None,

View File

@ -38,6 +38,7 @@ import vinyldns.core.TestMembershipData._
import vinyldns.core.domain.BatchChangeIsEmpty import vinyldns.core.domain.BatchChangeIsEmpty
import vinyldns.core.domain.auth.AuthPrincipal import vinyldns.core.domain.auth.AuthPrincipal
import vinyldns.core.domain.batch.BatchChangeApprovalStatus.BatchChangeApprovalStatus import vinyldns.core.domain.batch.BatchChangeApprovalStatus.BatchChangeApprovalStatus
import vinyldns.core.domain.batch.BatchChangeStatus.BatchChangeStatus
import vinyldns.core.domain.batch._ import vinyldns.core.domain.batch._
import vinyldns.core.domain.record.RecordType._ import vinyldns.core.domain.record.RecordType._
import vinyldns.core.domain.record._ import vinyldns.core.domain.record._
@ -132,6 +133,7 @@ class BatchChangeRoutingSpec()
), ),
ownerGroupId, ownerGroupId,
approvalStatus, approvalStatus,
BatchChangeStatus.PendingProcessing,
None, None,
None, None,
None, None,
@ -316,6 +318,7 @@ class BatchChangeRoutingSpec()
startFrom: Option[Int], startFrom: Option[Int],
maxItems: Int, maxItems: Int,
ignoreAccess: Boolean = false, ignoreAccess: Boolean = false,
batchStatus: Option[BatchChangeStatus] = None,
approvalStatus: Option[BatchChangeApprovalStatus] = None approvalStatus: Option[BatchChangeApprovalStatus] = None
): EitherT[IO, BatchChangeErrorResponse, BatchChangeSummaryList] = ): EitherT[IO, BatchChangeErrorResponse, BatchChangeSummaryList] =
if (auth.userId == okAuth.userId) if (auth.userId == okAuth.userId)

View File

@ -31,6 +31,7 @@ case class BatchChange(
changes: List[SingleChange], changes: List[SingleChange],
ownerGroupId: Option[String] = None, ownerGroupId: Option[String] = None,
approvalStatus: BatchChangeApprovalStatus, approvalStatus: BatchChangeApprovalStatus,
batchStatus: BatchChangeStatus = BatchChangeStatus.PendingProcessing,
reviewerId: Option[String] = None, reviewerId: Option[String] = None,
reviewComment: Option[String] = None, reviewComment: Option[String] = None,
reviewTimestamp: Option[Instant] = None, reviewTimestamp: Option[Instant] = None,
@ -92,6 +93,12 @@ object BatchChangeStatus extends Enumeration {
case _ => BatchChangeStatus.Complete 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 { object BatchChangeApprovalStatus extends Enumeration {

View File

@ -18,6 +18,7 @@ package vinyldns.core.domain.batch
import cats.effect.IO import cats.effect.IO
import vinyldns.core.domain.batch.BatchChangeApprovalStatus.BatchChangeApprovalStatus import vinyldns.core.domain.batch.BatchChangeApprovalStatus.BatchChangeApprovalStatus
import vinyldns.core.domain.batch.BatchChangeStatus.BatchChangeStatus
import vinyldns.core.repository.Repository import vinyldns.core.repository.Repository
// $COVERAGE-OFF$ // $COVERAGE-OFF$
@ -34,6 +35,7 @@ trait BatchChangeRepository extends Repository {
dateTimeEndRange: Option[String] = None, dateTimeEndRange: Option[String] = None,
startFrom: Option[Int] = None, startFrom: Option[Int] = None,
maxItems: Int = 100, maxItems: Int = 100,
batchStatus: Option[BatchChangeStatus] = None,
approvalStatus: Option[BatchChangeApprovalStatus] = None approvalStatus: Option[BatchChangeApprovalStatus] = None
): IO[BatchChangeSummaryList] ): IO[BatchChangeSummaryList]

View File

@ -91,6 +91,7 @@ case class BatchChangeSummaryList(
nextId: Option[Int] = None, nextId: Option[Int] = None,
maxItems: Int = 100, maxItems: Int = 100,
ignoreAccess: Boolean = false, ignoreAccess: Boolean = false,
batchStatus: Option[BatchChangeStatus] = None,
approvalStatus: Option[BatchChangeApprovalStatus] = None, approvalStatus: Option[BatchChangeApprovalStatus] = None,
userName: Option[String] = None, userName: Option[String] = None,
dateTimeStartRange: Option[String] = None, dateTimeStartRange: Option[String] = None,

View File

@ -47,6 +47,7 @@ class BatchChangeSummarySpec extends AnyWordSpec with Matchers {
List(pendingChange, failedChange, completeChange), List(pendingChange, failedChange, completeChange),
Some("groupId"), Some("groupId"),
BatchChangeApprovalStatus.AutoApproved, BatchChangeApprovalStatus.AutoApproved,
BatchChangeStatus.PendingProcessing,
None, None,
None, None,
None, None,

View File

@ -103,6 +103,7 @@ class MySqlBatchChangeRepositoryIntegrationSpec
changes, changes,
Some(UUID.randomUUID().toString), Some(UUID.randomUUID().toString),
BatchChangeApprovalStatus.AutoApproved, BatchChangeApprovalStatus.AutoApproved,
BatchChangeStatus.PendingProcessing,
Some(UUID.randomUUID().toString), Some(UUID.randomUUID().toString),
Some("review comment"), Some("review comment"),
Some(Instant.now.truncatedTo(ChronoUnit.MILLIS).plusSeconds(2)) Some(Instant.now.truncatedTo(ChronoUnit.MILLIS).plusSeconds(2))
@ -150,6 +151,8 @@ class MySqlBatchChangeRepositoryIntegrationSpec
createdTimestamp = timeBase.plusMillis(10000000), createdTimestamp = timeBase.plusMillis(10000000),
approvalStatus = BatchChangeApprovalStatus.ManuallyRejected approvalStatus = BatchChangeApprovalStatus.ManuallyRejected
) )
val change_six: BatchChange =
completeBatchChange.copy(createdTimestamp = timeBase.plusMillis(2000), ownerGroupId = None)
} }
import TestData._ import TestData._
@ -453,7 +456,8 @@ class MySqlBatchChangeRepositoryIntegrationSpec
} yield (updated, saved) } yield (updated, saved)
val (retrieved, saved) = f.unsafeRunSync 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 { "return no batch when single changes list is empty" in {
@ -894,6 +898,59 @@ class MySqlBatchChangeRepositoryIntegrationSpec
batchChangeSummaries.batchChanges shouldBe empty 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 { "properly status check (pending)" in {
val chg = randomBatchChange( val chg = randomBatchChange(
List( List(

View File

@ -0,0 +1,5 @@
CREATE SCHEMA IF NOT EXISTS ${dbName};
USE ${dbName};
ALTER TABLE batch_change ADD batch_status VARCHAR(25) NOT NULL;

View File

@ -24,6 +24,7 @@ import java.time.Instant
import org.slf4j.LoggerFactory import org.slf4j.LoggerFactory
import scalikejdbc._ import scalikejdbc._
import vinyldns.core.domain.batch.BatchChangeApprovalStatus.BatchChangeApprovalStatus import vinyldns.core.domain.batch.BatchChangeApprovalStatus.BatchChangeApprovalStatus
import vinyldns.core.domain.batch.BatchChangeStatus.BatchChangeStatus
import vinyldns.core.domain.batch._ import vinyldns.core.domain.batch._
import vinyldns.core.protobuf.{BatchChangeProtobufConversions, SingleChangeType} import vinyldns.core.protobuf.{BatchChangeProtobufConversions, SingleChangeType}
import vinyldns.core.route.Monitored import vinyldns.core.route.Monitored
@ -48,11 +49,11 @@ class MySqlBatchChangeRepository
private final val PUT_BATCH_CHANGE = private final val PUT_BATCH_CHANGE =
sql""" sql"""
| INSERT INTO batch_change(id, user_id, user_name, created_time, comments, owner_group_id, | 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) | scheduled_time, cancelled_timestamp)
| VALUES ({id}, {userId}, {userName}, {createdTime}, {comments}, {ownerGroupId}, {approvalStatus}, | VALUES ({id}, {userId}, {userName}, {createdTime}, {comments}, {ownerGroupId}, {approvalStatus},
| {reviewerId}, {reviewComment}, {reviewTimestamp}, {scheduledTime}, {cancelledTimestamp}) | {batchStatus}, {reviewerId}, {reviewComment}, {reviewTimestamp}, {scheduledTime}, {cancelledTimestamp})
|ON DUPLICATE KEY UPDATE comments={comments}, owner_group_id={ownerGroupId}, approval_status={approvalStatus}, |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}, | reviewer_id={reviewerId}, review_comment={reviewComment}, review_timestamp={reviewTimestamp},
| scheduled_time={scheduledTime}, cancelled_timestamp={cancelledTimestamp} | scheduled_time={scheduledTime}, cancelled_timestamp={cancelledTimestamp}
""".stripMargin """.stripMargin
@ -70,14 +71,14 @@ class MySqlBatchChangeRepository
private final val GET_BATCH_CHANGE_METADATA = private final val GET_BATCH_CHANGE_METADATA =
sql""" sql"""
|SELECT user_id, user_name, created_time, comments, owner_group_id, |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 | FROM batch_change bc
| WHERE bc.id = ? | WHERE bc.id = ?
""".stripMargin """.stripMargin
private final val GET_BATCH_CHANGE_METADATA_FROM_SINGLE_CHANGE = private final val GET_BATCH_CHANGE_METADATA_FROM_SINGLE_CHANGE =
sql""" 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 | bc.reviewer_id, bc.review_comment, bc.review_timestamp, bc.scheduled_time, bc.cancelled_timestamp
| FROM batch_change bc | FROM batch_change bc
| JOIN (SELECT id, batch_change_id from single_change where id = ?) sc | 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 = 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, | 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 '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 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 'Complete' THEN 1 ELSE 0 END) AS complete_count,
| SUM(CASE sc.status WHEN 'Cancelled' THEN 1 ELSE 0 END) AS cancelled_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 | bc.reviewer_id, bc.review_comment, bc.review_timestamp, bc.scheduled_time, bc.cancelled_timestamp
| FROM batch_change bc | FROM batch_change bc
""".stripMargin """.stripMargin
@ -124,6 +125,13 @@ class MySqlBatchChangeRepository
| WHERE id={id} | WHERE id={id}
""".stripMargin """.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] = def save(batch: BatchChange): IO[BatchChange] =
monitor("repo.BatchChangeJDBC.save") { monitor("repo.BatchChangeJDBC.save") {
IO { IO {
@ -181,6 +189,24 @@ class MySqlBatchChangeRepository
.apply() .apply()
batchMeta.copy(changes = changes) 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") { monitor("repo.BatchChangeJDBC.updateSingleChanges") {
IO { IO {
logger.info(s"Updating single change status: ${singleChanges.map(ch => (ch.id, ch.status))}") logger.info(s"Updating single change status: ${singleChanges.map(ch => (ch.id, ch.status))}")
@ -190,6 +216,8 @@ class MySqlBatchChangeRepository
batchParams = singleChanges.map(convertSingleChangeToParams) batchParams = singleChanges.map(convertSingleChangeToParams)
_ = UPDATE_SINGLE_CHANGE.batchByName(batchParams: _*).apply() _ = UPDATE_SINGLE_CHANGE.batchByName(batchParams: _*).apply()
batchChange <- getBatchFromSingleChangeId(headChange.id) 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 } yield batchChange
}} }}
} }
@ -234,6 +262,7 @@ class MySqlBatchChangeRepository
dateTimeEndRange: Option[String] = None, dateTimeEndRange: Option[String] = None,
startFrom: Option[Int] = None, startFrom: Option[Int] = None,
maxItems: Int = 100, maxItems: Int = 100,
batchStatus: Option[BatchChangeStatus],
approvalStatus: Option[BatchChangeApprovalStatus] approvalStatus: Option[BatchChangeApprovalStatus]
): IO[BatchChangeSummaryList] = ): IO[BatchChangeSummaryList] =
monitor("repo.BatchChangeJDBC.getBatchChangeSummaries") { monitor("repo.BatchChangeJDBC.getBatchChangeSummaries") {
@ -245,13 +274,14 @@ class MySqlBatchChangeRepository
val uid = userId.map(u => s"bc.user_id = '$u'") val uid = userId.map(u => s"bc.user_id = '$u'")
val as = approvalStatus.map(a => s"bc.approval_status = '${fromApprovalStatus(a)}'") 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 uname = userName.map(uname => s"bc.user_name = '$uname'")
val dtRange = if(dateTimeStartRange.isDefined && dateTimeEndRange.isDefined) { val dtRange = if(dateTimeStartRange.isDefined && dateTimeEndRange.isDefined) {
Some(s"(bc.created_time >= '${dateTimeStartRange.get}' AND bc.created_time <= '${dateTimeEndRange.get}')") Some(s"(bc.created_time >= '${dateTimeStartRange.get}' AND bc.created_time <= '${dateTimeEndRange.get}')")
} else { } else {
None None
} }
val opts = uid ++ as ++ uname ++ dtRange val opts = uid ++ as ++ bs ++ uname ++ dtRange
if (opts.nonEmpty) sb.append("WHERE ").append(opts.mkString(" AND ")) if (opts.nonEmpty) sb.append("WHERE ").append(opts.mkString(" AND "))
@ -267,6 +297,7 @@ class MySqlBatchChangeRepository
val complete = res.int("complete_count") val complete = res.int("complete_count")
val cancelled = res.int("cancelled_count") val cancelled = res.int("cancelled_count")
val approvalStatus = toApprovalStatus(res.intOpt("approval_status")) val approvalStatus = toApprovalStatus(res.intOpt("approval_status"))
val batchChangeStatus = toBatchChangeStatus(res.stringOpt("batch_status"))
val schedTime = val schedTime =
res.timestampOpt("scheduled_time").map(st => st.toInstant) res.timestampOpt("scheduled_time").map(st => st.toInstant)
val cancelledTimestamp = val cancelledTimestamp =
@ -277,14 +308,7 @@ class MySqlBatchChangeRepository
Option(res.string("comments")), Option(res.string("comments")),
res.timestamp("created_time").toInstant, res.timestamp("created_time").toInstant,
pending + failed + complete + cancelled, pending + failed + complete + cancelled,
BatchChangeStatus batchChangeStatus,
.calculateBatchStatus(
approvalStatus,
pending > 0,
failed > 0,
complete > 0,
schedTime.isDefined
),
Option(res.string("owner_group_id")), Option(res.string("owner_group_id")),
res.string("id"), res.string("id"),
None, None,
@ -308,6 +332,7 @@ class MySqlBatchChangeRepository
nextId, nextId,
maxItems, maxItems,
ignoreAccess, ignoreAccess,
batchStatus,
approvalStatus approvalStatus
) )
} }
@ -338,6 +363,7 @@ class MySqlBatchChangeRepository
Nil, Nil,
result.stringOpt("owner_group_id"), result.stringOpt("owner_group_id"),
toApprovalStatus(result.intOpt("approval_status")), toApprovalStatus(result.intOpt("approval_status")),
toBatchChangeStatus(result.stringOpt("batch_status")),
result.stringOpt("reviewer_id"), result.stringOpt("reviewer_id"),
result.stringOpt("review_comment"), result.stringOpt("review_comment"),
result.timestampOpt("review_timestamp").map(toDateTime), result.timestampOpt("review_timestamp").map(toDateTime),
@ -350,6 +376,33 @@ class MySqlBatchChangeRepository
private def saveBatchChange( private def saveBatchChange(
batchChange: BatchChange batchChange: BatchChange
)(implicit session: DBSession): 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 PUT_BATCH_CHANGE
.bindByName( .bindByName(
Seq( Seq(
@ -360,6 +413,7 @@ class MySqlBatchChangeRepository
'comments -> batchChange.comments, 'comments -> batchChange.comments,
'ownerGroupId -> batchChange.ownerGroupId, 'ownerGroupId -> batchChange.ownerGroupId,
'approvalStatus -> fromApprovalStatus(batchChange.approvalStatus), 'approvalStatus -> fromApprovalStatus(batchChange.approvalStatus),
'batchStatus -> fromBatchStatus(batchStatus),
'reviewerId -> batchChange.reviewerId, 'reviewerId -> batchChange.reviewerId,
'reviewComment -> batchChange.reviewComment, 'reviewComment -> batchChange.reviewComment,
'reviewTimestamp -> batchChange.reviewTimestamp, 'reviewTimestamp -> batchChange.reviewTimestamp,
@ -424,6 +478,18 @@ class MySqlBatchChangeRepository
case BatchChangeApprovalStatus.Cancelled => 5 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 = def toApprovalStatus(key: Option[Int]): BatchChangeApprovalStatus =
key match { key match {
case Some(1) => BatchChangeApprovalStatus.AutoApproved case Some(1) => BatchChangeApprovalStatus.AutoApproved
@ -434,5 +500,19 @@ class MySqlBatchChangeRepository
case _ => BatchChangeApprovalStatus.AutoApproved 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 def toDateTime(ts: Timestamp): Instant = ts.toInstant
} }