2
0
mirror of https://github.com/VinylDNS/vinyldns synced 2025-09-04 16:25:32 +00:00

Add approvalStatus query param to list batch changes (#715)

This commit is contained in:
Britney Wright
2019-07-03 13:37:26 -04:00
committed by GitHub
parent 81df1aff1b
commit 6b84bb18ca
15 changed files with 287 additions and 49 deletions

View File

@@ -63,7 +63,8 @@ class ListBatchChangeSummariesFixture():
shared_zone_test_context.ok_vinyldns_client.wait_until_recordset_change_status(delete_result, 'Complete')
clear_ok_acl_rules(shared_zone_test_context)
def check_batch_change_summaries_page_accuracy(self, summaries_page, size, next_id=False, start_from=False, max_items=100):
def check_batch_change_summaries_page_accuracy(self, summaries_page, size, next_id=False, start_from=False,
max_items=100, approval_status=False):
# validate fields
if next_id:
assert_that(summaries_page, has_key('nextId'))
@@ -73,6 +74,10 @@ class ListBatchChangeSummariesFixture():
assert_that(summaries_page['startFrom'], is_(start_from))
else:
assert_that(summaries_page, is_not(has_key('startFrom')))
if approval_status:
assert_that(summaries_page, has_key('approvalStatus'))
else:
assert_that(summaries_page, is_not(has_key('approvalStatus')))
assert_that(summaries_page['maxItems'], is_(max_items))
@@ -145,6 +150,15 @@ def test_list_batch_change_summaries_with_next_id(list_fixture):
list_fixture.check_batch_change_summaries_page_accuracy(next_page_result, size=1, start_from=batch_change_summaries_result['nextId'])
def test_list_batch_change_summaries_with_pending_status(list_fixture):
"""
Test listing a limited number of user's batch change summaries with maxItems parameter
"""
client = list_fixture.client
batch_change_summaries_result = client.list_batch_change_summaries(status=200, approval_status="PendingApproval")
list_fixture.check_batch_change_summaries_page_accuracy(batch_change_summaries_result, size=0, approval_status="PendingApproval")
def test_list_batch_change_summaries_with_list_batch_change_summaries_with_no_changes_passes():
"""
Test successfully getting an empty list of summaries when user has no batch changes
@@ -252,7 +266,7 @@ def test_list_batch_change_summaries_with_deleted_record_owner_group_passes(shar
client.wait_until_recordset_change_status(delete_result, 'Complete')
def test_list_batch_change_summaries_with_list_all_true_only_shows_requesting_users_records(shared_zone_test_context):
def test_list_batch_change_summaries_with_ignore_access_true_only_shows_requesting_users_records(shared_zone_test_context):
"""
Test that getting a batch change summary with list all set to true only returns the requesting user's batch changes
if they are not a super user
@@ -287,7 +301,7 @@ def test_list_batch_change_summaries_with_list_all_true_only_shows_requesting_us
record_set_list = [(change['zoneId'], change['recordSetId']) for change in completed_batch['changes']]
record_to_delete = set(record_set_list)
batch_change_summaries_result = client.list_batch_change_summaries(list_all=True, status=200)["batchChanges"]
batch_change_summaries_result = client.list_batch_change_summaries(ignore_access=True, status=200)["batchChanges"]
under_test = [item for item in batch_change_summaries_result if item['id'] == completed_batch['id']]
assert_that(under_test, has_length(1))
@@ -299,7 +313,7 @@ def test_list_batch_change_summaries_with_list_all_true_only_shows_requesting_us
ok_record_set_list = [(change['zoneId'], change['recordSetId']) for change in ok_completed_batch['changes']]
ok_record_to_delete = set(ok_record_set_list)
ok_batch_change_summaries_result = ok_client.list_batch_change_summaries(list_all=True, status=200)["batchChanges"]
ok_batch_change_summaries_result = ok_client.list_batch_change_summaries(ignore_access=True, status=200)["batchChanges"]
ok_under_test = [item for item in ok_batch_change_summaries_result if (item['id'] == ok_completed_batch['id'] or item['id'] == completed_batch['id']) ]
assert_that(ok_under_test, has_length(1))
@@ -311,3 +325,4 @@ def test_list_batch_change_summaries_with_list_all_true_only_shows_requesting_us
for result_rs in ok_record_to_delete:
delete_result = client.delete_recordset(result_rs[0], result_rs[1], status=202)
client.wait_until_recordset_change_status(delete_result, 'Complete')

View File

@@ -127,9 +127,9 @@ def test_list_zones_max_items_100(list_zones_context):
result = list_zones_context.client.list_zones(status=200)
assert_that(result['maxItems'], is_(100))
def test_list_zones_list_all_default_false(list_zones_context):
def test_list_zones_ignore_access_default_false(list_zones_context):
"""
Test that the default list all value for a list zones request is false
Test that the default ignore access value for a list zones request is false
"""
result = list_zones_context.client.list_zones(status=200)
assert_that(result['ignoreAccess'], is_(False))
@@ -248,22 +248,22 @@ def test_list_zones_with_search_last_page(list_zones_context):
assert_that(result['nameFilter'], is_('*test-searched-3'))
assert_that(result['startFrom'], is_('list-zones-test-searched-2.'))
def test_list_zones_list_all_success(list_zones_context):
def test_list_zones_ignore_access_success(list_zones_context):
"""
Test that we can retrieve a list of all zones
Test that we can retrieve a list of zones regardless of zone access
"""
result = list_zones_context.client.list_zones(list_all=True, status=200)
result = list_zones_context.client.list_zones(ignore_access=True, status=200)
retrieved = result['zones']
assert_that(result['ignoreAccess'], is_(True))
assert_that(len(retrieved), greater_than(5))
def test_list_zones_list_all_success_with_name_filter(list_zones_context):
def test_list_zones_ignore_access_success_with_name_filter(list_zones_context):
"""
Test that we can retrieve a list of all zones with a name filter
"""
result = list_zones_context.client.list_zones(name_filter='shared', list_all=True, status=200)
result = list_zones_context.client.list_zones(name_filter='shared', ignore_access=True, status=200)
retrieved = result['zones']
assert_that(result['ignoreAccess'], is_(True))

View File

@@ -444,7 +444,7 @@ class VinylDNSClient(object):
response, data = self.make_request(url, u'GET', self.headers, not_found_ok=True, **kwargs)
return data
def list_zones(self, name_filter=None, start_from=None, max_items=None, list_all=False, **kwargs):
def list_zones(self, name_filter=None, start_from=None, max_items=None, ignore_access=False, **kwargs):
"""
Gets a list of zones that currently exist
:return: a list of zones
@@ -461,8 +461,8 @@ class VinylDNSClient(object):
if max_items:
query.append(u'maxItems=' + str(max_items))
if list_all:
query.append(u'ignoreAccess=' + str(list_all))
if ignore_access:
query.append(u'ignoreAccess=' + str(ignore_access))
if query:
url = url + u'?' + u'&'.join(query)
@@ -595,7 +595,7 @@ class VinylDNSClient(object):
_, data = self.make_request(url, u'POST', self.headers, json.dumps(approve_batch_change_input), **kwargs)
return data
def list_batch_change_summaries(self, start_from=None, max_items=None, list_all=False, **kwargs):
def list_batch_change_summaries(self, start_from=None, max_items=None, ignore_access=False, approval_status=None, **kwargs):
"""
Gets list of user's batch change summaries
:return: the content of the response
@@ -605,8 +605,10 @@ class VinylDNSClient(object):
args.append(u'startFrom={0}'.format(start_from))
if max_items is not None:
args.append(u'maxItems={0}'.format(max_items))
if list_all:
args.append(u'ignoreAccess={0}'.format(list_all))
if ignore_access:
args.append(u'ignoreAccess={0}'.format(ignore_access))
if approval_status:
args.append(u'approvalStatus={0}'.format(approval_status))
url = urljoin(self.index_url, u'/zones/batchrecordchanges') + u'?' + u'&'.join(args)

View File

@@ -29,6 +29,7 @@ import vinyldns.api.domain.dns.DnsConversions._
import vinyldns.api.domain.{RecordAlreadyExists, ZoneDiscoveryError}
import vinyldns.api.repository.ApiDataAccessor
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.membership.{Group, GroupRepository}
@@ -372,11 +373,13 @@ class BatchChangeService(
auth: AuthPrincipal,
startFrom: Option[Int] = None,
maxItems: Int = 100,
ignoreAccess: Boolean = false): BatchResult[BatchChangeSummaryList] = {
ignoreAccess: Boolean = false,
approvalStatus: Option[BatchChangeApprovalStatus] = None)
: BatchResult[BatchChangeSummaryList] = {
val userId = if (ignoreAccess && auth.isSystemAdmin) None else Some(auth.userId)
for {
listResults <- batchChangeRepo
.getBatchChangeSummaries(userId, startFrom, maxItems)
.getBatchChangeSummaries(userId, startFrom, maxItems, approvalStatus)
.toBatchResult
rsOwnerGroupIds = listResults.batchChanges.flatMap(_.ownerGroupId).toSet
rsOwnerGroups <- groupRepository.getGroups(rsOwnerGroupIds).toBatchResult
@@ -385,7 +388,8 @@ class BatchChangeService(
rsOwnerGroups)
listWithGroupNames = listResults.copy(
batchChanges = summariesWithGroupNames,
ignoreAccess = ignoreAccess)
ignoreAccess = ignoreAccess,
approvalStatus = approvalStatus)
} yield listWithGroupNames
}
}

View File

@@ -18,6 +18,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.{BatchChange, BatchChangeInfo, BatchChangeSummaryList}
// $COVERAGE-OFF$
@@ -32,7 +33,8 @@ trait BatchChangeServiceAlgebra {
auth: AuthPrincipal,
startFrom: Option[Int],
maxItems: Int,
ignoreAccess: Boolean): BatchResult[BatchChangeSummaryList]
ignoreAccess: Boolean,
approvalStatus: Option[BatchChangeApprovalStatus]): BatchResult[BatchChangeSummaryList]
def rejectBatchChange(
batchChangeId: String,

View File

@@ -33,6 +33,7 @@ trait BatchChangeJsonProtocol extends JsonValidation {
val batchChangeSerializers = Seq(
JsonEnumV(ChangeInputType),
JsonEnumV(SingleChangeStatus),
JsonEnumV(BatchChangeApprovalStatus),
BatchChangeInputSerializer,
ChangeInputSerializer,
AddChangeInputSerializer,

View File

@@ -24,6 +24,7 @@ import cats.effect._
import vinyldns.api.VinylDNSConfig
import vinyldns.core.domain.auth.AuthPrincipal
import vinyldns.api.domain.batch._
import vinyldns.core.domain.batch.BatchChangeApprovalStatus
trait BatchChangeRoute extends Directives {
this: VinylDNSJsonProtocol with VinylDNSDirectives with JsonValidationRejection =>
@@ -53,16 +54,27 @@ trait BatchChangeRoute extends Directives {
parameters(
"startFrom".as[Int].?,
"maxItems".as[Int].?(MAX_ITEMS_LIMIT),
"ignoreAccess".as[Boolean].?(false)) {
(startFrom: Option[Int], maxItems: Int, ignoreAccess: Boolean) =>
"ignoreAccess".as[Boolean].?(false),
"approvalStatus".as[String].?) {
(
startFrom: Option[Int],
maxItems: Int,
ignoreAccess: Boolean,
approvalStatus: Option[String]) =>
{
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.") {
execute(batchChangeService
.listBatchChangeSummaries(authPrincipal, startFrom, maxItems, ignoreAccess)) {
summaries =>
execute(
batchChangeService.listBatchChangeSummaries(
authPrincipal,
startFrom,
maxItems,
ignoreAccess,
convertApprovalStatus)) { summaries =>
complete(StatusCodes.OK, summaries)
}
}

View File

@@ -1169,6 +1169,43 @@ class BatchChangeServiceSpec
result.batchChanges(0).createdTimestamp shouldBe batchChangeTwo.createdTimestamp
}
"return list of batchChangeSummaries filtered by approvalStatus if some exist" in {
val batchChangeOne =
BatchChange(
auth.userId,
auth.signedInUser.userName,
None,
DateTime.now,
List(),
approvalStatus = BatchChangeApprovalStatus.PendingApproval)
batchChangeRepo.save(batchChangeOne)
val batchChangeTwo = BatchChange(
auth.userId,
auth.signedInUser.userName,
None,
new DateTime(DateTime.now.getMillis + 1000),
List(),
approvalStatus = BatchChangeApprovalStatus.AutoApproved)
batchChangeRepo.save(batchChangeTwo)
val result = rightResultOf(
underTest
.listBatchChangeSummaries(
auth,
approvalStatus = Some(BatchChangeApprovalStatus.PendingApproval))
.value)
result.maxItems shouldBe 100
result.nextId shouldBe None
result.startFrom shouldBe None
result.ignoreAccess shouldBe false
result.approvalStatus shouldBe Some(BatchChangeApprovalStatus.PendingApproval)
result.batchChanges.length shouldBe 1
result.batchChanges(0).createdTimestamp shouldBe batchChangeOne.createdTimestamp
}
"return an offset list of batchChangeSummaries if some exist" in {
val batchChangeOne =
BatchChange(

View File

@@ -112,8 +112,11 @@ class InMemoryBatchChangeRepository extends BatchChangeRepository {
def getBatchChangeSummaries(
userId: Option[String],
startFrom: Option[Int] = None,
maxItems: Int = 100): IO[BatchChangeSummaryList] = {
val userBatchChanges = batches.values.toList.filter(b => userId.forall(_ == b.userId))
maxItems: Int = 100,
approvalStatus: Option[BatchChangeApprovalStatus] = None): IO[BatchChangeSummaryList] = {
val userBatchChanges = batches.values.toList
.filter(b => userId.forall(_ == b.userId))
.filter(as => approvalStatus.forall(_ == as.approvalStatus))
val batchChangeSummaries = for {
sc <- userBatchChanges
ids = sc.changes
@@ -136,8 +139,15 @@ class InMemoryBatchChangeRepository extends BatchChangeRepository {
val until = maxItems + start
val limited = sorted.slice(start, until)
val nextId = if (limited.size < maxItems) None else Some(start + limited.size)
val ignoreAccess = userId.isDefined
IO.pure(
BatchChangeSummaryList(limited, startFrom = startFrom, nextId = nextId, maxItems = maxItems))
BatchChangeSummaryList(
limited,
startFrom = startFrom,
nextId = nextId,
maxItems = maxItems,
ignoreAccess = ignoreAccess,
approvalStatus = approvalStatus))
}
def clear(): Unit = {

View File

@@ -34,6 +34,7 @@ import vinyldns.core.domain.record._
import cats.effect._
import vinyldns.api.domain.BatchChangeIsEmpty
import vinyldns.core.domain.auth.AuthPrincipal
import vinyldns.core.domain.batch.BatchChangeApprovalStatus.BatchChangeApprovalStatus
import vinyldns.core.domain.batch._
class BatchChangeRoutingSpec
@@ -59,7 +60,9 @@ class BatchChangeRoutingSpec
def createBatchChangeResponse(
comments: Option[String] = None,
ownerGroupId: Option[String] = None,
auth: AuthPrincipal = okAuth): BatchChange =
auth: AuthPrincipal = okAuth,
approvalStatus: BatchChangeApprovalStatus = BatchChangeApprovalStatus.AutoApproved)
: BatchChange =
BatchChange(
auth.userId,
auth.signedInUser.userName,
@@ -92,7 +95,7 @@ class BatchChangeRoutingSpec
"singleDeleteChangeId")
),
ownerGroupId,
BatchChangeApprovalStatus.AutoApproved,
approvalStatus,
None,
None,
None,
@@ -139,10 +142,16 @@ class BatchChangeRoutingSpec
compact(render(("comments" -> comments) ~~ changeList))
val batchChangeSummaryInfo1 = BatchChangeSummary(createBatchChangeResponse(Some("first")))
val batchChangeSummaryInfo2 = BatchChangeSummary(createBatchChangeResponse(Some("second")))
val batchChangeSummaryInfo2 = BatchChangeSummary(
createBatchChangeResponse(
Some("second"),
approvalStatus = BatchChangeApprovalStatus.PendingApproval))
val batchChangeSummaryInfo3 = BatchChangeSummary(createBatchChangeResponse(Some("third")))
val batchChangeSummaryInfo4 = BatchChangeSummary(
createBatchChangeResponse(Some("fourth"), auth = dummyAuth))
createBatchChangeResponse(
Some("fourth"),
auth = dummyAuth,
approvalStatus = BatchChangeApprovalStatus.PendingApproval))
val validResponseWithComments: BatchChange = createBatchChangeResponse(
Some("validChangeWithComments"))
@@ -229,11 +238,12 @@ class BatchChangeRoutingSpec
auth: AuthPrincipal,
startFrom: Option[Int],
maxItems: Int,
ignoreAccess: Boolean = false)
ignoreAccess: Boolean = false,
approvalStatus: Option[BatchChangeApprovalStatus] = None)
: EitherT[IO, BatchChangeErrorResponse, BatchChangeSummaryList] =
if (auth.userId == okAuth.userId)
(auth, startFrom, maxItems, ignoreAccess) match {
case (_, None, 100, false) =>
(auth, startFrom, maxItems, ignoreAccess, approvalStatus) match {
case (_, None, 100, _, None) =>
EitherT.rightT(
BatchChangeSummaryList(
batchChanges =
@@ -243,7 +253,7 @@ class BatchChangeRoutingSpec
)
)
case (_, None, 1, false) =>
case (_, None, 1, _, None) =>
EitherT.rightT(
BatchChangeSummaryList(
batchChanges = List(batchChangeSummaryInfo1),
@@ -253,7 +263,7 @@ class BatchChangeRoutingSpec
)
)
case (_, Some(1), 100, false) =>
case (_, Some(1), 100, _, None) =>
EitherT.rightT(
BatchChangeSummaryList(
batchChanges = List(batchChangeSummaryInfo2),
@@ -262,7 +272,7 @@ class BatchChangeRoutingSpec
)
)
case (_, Some(1), 1, false) =>
case (_, Some(1), 1, _, None) =>
EitherT.rightT(
BatchChangeSummaryList(
batchChanges = List(batchChangeSummaryInfo2),
@@ -272,10 +282,20 @@ class BatchChangeRoutingSpec
)
)
case (_, None, 100, _, Some(BatchChangeApprovalStatus.PendingApproval)) =>
EitherT.rightT(
BatchChangeSummaryList(
batchChanges = List(batchChangeSummaryInfo2),
startFrom = None,
nextId = None,
approvalStatus = Some(BatchChangeApprovalStatus.PendingApproval)
)
)
case (_) => EitherT.rightT(BatchChangeSummaryList(List()))
} else if (auth.userId == superUserAuth.userId)
(auth, startFrom, maxItems, ignoreAccess) match {
case (_, None, 100, true) =>
(auth, startFrom, maxItems, ignoreAccess, approvalStatus) match {
case (_, None, 100, true, None) =>
EitherT.rightT(
BatchChangeSummaryList(
batchChanges = List(
@@ -288,6 +308,38 @@ class BatchChangeRoutingSpec
ignoreAccess = true
)
)
case (_, None, 100, true, Some(BatchChangeApprovalStatus.PendingApproval)) => {
EitherT.rightT(
BatchChangeSummaryList(
batchChanges = List(batchChangeSummaryInfo2, batchChangeSummaryInfo4),
startFrom = None,
nextId = None,
ignoreAccess = true,
approvalStatus = Some(BatchChangeApprovalStatus.PendingApproval)
)
)
}
case (_, None, 100, false, None) =>
EitherT.rightT(
BatchChangeSummaryList(
batchChanges = List(),
startFrom = None,
nextId = None
)
)
case (_, None, 100, false, Some(BatchChangeApprovalStatus.PendingApproval)) =>
EitherT.rightT(
BatchChangeSummaryList(
batchChanges = List(),
startFrom = None,
nextId = None,
approvalStatus = Some(BatchChangeApprovalStatus.PendingApproval)
)
)
case (_) => EitherT.rightT(BatchChangeSummaryList(List()))
} else
EitherT.rightT(BatchChangeSummaryList(List()))
@@ -498,6 +550,21 @@ class BatchChangeRoutingSpec
}
}
"return user's Pending batch changes if approval status is `PendingApproval`" in {
Get("/zones/batchrecordchanges?approvalStatus=pendingapproval") ~>
batchChangeRoute(okAuth) ~> check {
status shouldBe OK
val resp = responseAs[BatchChangeSummaryList]
resp.batchChanges.length shouldBe 1
resp.maxItems shouldBe 100
resp.startFrom shouldBe None
resp.nextId shouldBe None
resp.approvalStatus shouldBe Some(BatchChangeApprovalStatus.PendingApproval)
}
}
"return an error if maxItems is out of range" in {
Get("/zones/batchrecordchanges?startFrom=1&maxItems=101") ~> batchChangeRoute(okAuth) ~> check {
status shouldBe BadRequest
@@ -517,7 +584,7 @@ class BatchChangeRoutingSpec
}
}
"return all batch changes if ignoreAccess is true" in {
"return all batch changes if ignoreAccess is true and requester is a super user" in {
Get("/zones/batchrecordchanges?ignoreAccess=true") ~> batchChangeRoute(superUserAuth) ~> check {
status shouldBe OK
@@ -529,6 +596,23 @@ class BatchChangeRoutingSpec
resp.nextId shouldBe None
}
}
"return all Pending batch changes if ignoreAccess is true, approval status is `PendingApproval`," +
" and requester is a super user" in {
Get("/zones/batchrecordchanges?ignoreAccess=true&approvalStatus=PendingApproval") ~>
batchChangeRoute(superUserAuth) ~> check {
status shouldBe OK
val resp = responseAs[BatchChangeSummaryList]
resp.batchChanges.length shouldBe 2
resp.maxItems shouldBe 100
resp.startFrom shouldBe None
resp.nextId shouldBe None
resp.ignoreAccess shouldBe true
resp.approvalStatus shouldBe Some(BatchChangeApprovalStatus.PendingApproval)
}
}
}
"POST reject batch change" should {

View File

@@ -75,6 +75,12 @@ object BatchChangeStatus extends Enumeration {
object BatchChangeApprovalStatus extends Enumeration {
type BatchChangeApprovalStatus = Value
val AutoApproved, PendingApproval, ManuallyApproved, ManuallyRejected = Value
private val valueMap =
BatchChangeApprovalStatus.values.map(v => v.toString.toLowerCase -> v).toMap
def find(status: String): Option[BatchChangeApprovalStatus] =
valueMap.get(status.toLowerCase)
}
case class BatchChangeInfo(

View File

@@ -17,6 +17,7 @@
package vinyldns.core.domain.batch
import cats.effect.IO
import vinyldns.core.domain.batch.BatchChangeApprovalStatus.BatchChangeApprovalStatus
import vinyldns.core.repository.Repository
// $COVERAGE-OFF$
@@ -29,7 +30,8 @@ trait BatchChangeRepository extends Repository {
def getBatchChangeSummaries(
userId: Option[String],
startFrom: Option[Int] = None,
maxItems: Int = 100): IO[BatchChangeSummaryList]
maxItems: Int = 100,
approvalStatus: Option[BatchChangeApprovalStatus] = None): IO[BatchChangeSummaryList]
// updateSingleChanges updates status, recordSetId, recordChangeId and systemMessage (in data).
def updateSingleChanges(singleChanges: List[SingleChange]): IO[Option[BatchChange]]

View File

@@ -19,6 +19,7 @@ package vinyldns.core.domain.batch
import java.util.UUID
import org.joda.time.DateTime
import vinyldns.core.domain.batch.BatchChangeApprovalStatus.BatchChangeApprovalStatus
import vinyldns.core.domain.batch.BatchChangeStatus.BatchChangeStatus
case class BatchChangeSummary(
@@ -66,4 +67,5 @@ case class BatchChangeSummaryList(
startFrom: Option[Int] = None,
nextId: Option[Int] = None,
maxItems: Int = 100,
ignoreAccess: Boolean = false)
ignoreAccess: Boolean = false,
approvalStatus: Option[BatchChangeApprovalStatus] = None)

View File

@@ -112,7 +112,8 @@ class MySqlBatchChangeRepositoryIntegrationSpec
// listing/ordering changes
val timeBase: DateTime = DateTime.now
val change_one: BatchChange = pendingBatchChange.copy(createdTimestamp = timeBase)
val change_one: BatchChange = pendingBatchChange.copy(createdTimestamp = timeBase,
approvalStatus = BatchChangeApprovalStatus.PendingApproval)
val change_two: BatchChange =
completeBatchChange.copy(createdTimestamp = timeBase.plus(1000), ownerGroupId = None)
val otherUserBatchChange: BatchChange = randomBatchChange().copy(userId = "Other",
@@ -355,6 +356,28 @@ class MySqlBatchChangeRepositoryIntegrationSpec
areSame(f.unsafeRunSync(), expectedChanges)
}
"get batch change summaries by approval 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,
approvalStatus = Some(BatchChangeApprovalStatus.PendingApproval))
} yield retrieved
// from most recent descending
val expectedChanges = BatchChangeSummaryList(
List(BatchChangeSummary(change_one))
)
areSame(f.unsafeRunSync(), expectedChanges)
}
"get batch change summaries by user ID" in {
val f =
for {
@@ -493,6 +516,30 @@ class MySqlBatchChangeRepositoryIntegrationSpec
areSame(retrieved._2, secondPageExpectedChanges)
}
"get batch change summaries by user ID and approval 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))
} yield retrieved
// from most recent descending
val expectedChanges = BatchChangeSummaryList(
List(
BatchChangeSummary(change_four),
BatchChangeSummary(change_three),
BatchChangeSummary(change_two))
)
areSame(f.unsafeRunSync(), expectedChanges)
}
"return empty list if a batch change summary is not found by user ID" in {
val batchChangeSummaries = repo.getBatchChangeSummaries(Some("doesnotexist")).unsafeRunSync()
batchChangeSummaries.batchChanges shouldBe empty

View File

@@ -221,14 +221,21 @@ class MySqlBatchChangeRepository
def getBatchChangeSummaries(
userId: Option[String],
startFrom: Option[Int] = None,
maxItems: Int = 100): IO[BatchChangeSummaryList] =
maxItems: Int = 100,
approvalStatus: Option[BatchChangeApprovalStatus]): IO[BatchChangeSummaryList] =
monitor("repo.BatchChangeJDBC.getBatchChangeSummaries") {
IO {
DB.readOnly { implicit s =>
val startValue = startFrom.getOrElse(0)
val sb = new StringBuilder
sb.append(GET_BATCH_CHANGE_SUMMARY_BASE)
userId.foreach(uid => sb.append(s"WHERE bc.user_id = '$uid' "))
val uid = userId.map(u => s"bc.user_id = '$u'")
val as = approvalStatus.map(a => s"bc.approval_status = '${fromApprovalStatus(a)}'")
val opts = uid ++ as
if (opts.nonEmpty) sb.append("WHERE ").append(opts.mkString(" AND "))
sb.append(GET_BATCH_CHANGE_SUMMARY_END)
val query = sb.toString()
@@ -256,7 +263,14 @@ class MySqlBatchChangeRepository
.apply()
val maxQueries = queryResult.take(maxItems)
val nextId = if (queryResult.size <= maxItems) None else Some(startValue + maxItems)
BatchChangeSummaryList(maxQueries, startFrom, nextId, maxItems)
val ignoreAccess = userId.isEmpty
BatchChangeSummaryList(
maxQueries,
startFrom,
nextId,
maxItems,
ignoreAccess,
approvalStatus)
}
}
}