2
0
mirror of https://github.com/VinylDNS/vinyldns synced 2025-08-30 22:05:21 +00:00

Approval up front checks (#755)

* up front approval conversions

* with tests

* fix approval func tests

* address comments
This commit is contained in:
Rebecca Star
2019-07-22 10:07:00 -04:00
committed by GitHub
parent c2c103add3
commit 000139ff8f
11 changed files with 156 additions and 23 deletions

View File

@@ -9,7 +9,7 @@ def test_approve_batch_change_with_invalid_batch_change_id_fails(shared_zone_tes
client = shared_zone_test_context.ok_vinyldns_client client = shared_zone_test_context.ok_vinyldns_client
error = client.reject_batch_change("some-id", status=404) error = client.approve_batch_change("some-id", status=404)
assert_that(error, is_("Batch change with id some-id cannot be found")) assert_that(error, is_("Batch change with id some-id cannot be found"))
@pytest.mark.manual_batch_review @pytest.mark.manual_batch_review
@@ -19,10 +19,10 @@ def test_approve_batch_change_with_comments_exceeding_max_length_fails(shared_zo
""" """
client = shared_zone_test_context.ok_vinyldns_client client = shared_zone_test_context.ok_vinyldns_client
reject_batch_change_input = { approve_batch_change_input = {
"reviewComment": "a"*1025 "reviewComment": "a"*1025
} }
errors = client.reject_batch_change("some-id", reject_batch_change_input, status=400)['errors'] errors = client.approve_batch_change("some-id", approve_batch_change_input, status=400)['errors']
assert_that(errors, contains_inanyorder("Comment length must not exceed 1024 characters.")) assert_that(errors, contains_inanyorder("Comment length must not exceed 1024 characters."))
@pytest.mark.manual_batch_review @pytest.mark.manual_batch_review
@@ -42,7 +42,7 @@ def test_approve_batch_change_fails_with_forbidden_error_for_non_system_admins(s
result = client.create_batch_change(batch_change_input, status=202) result = client.create_batch_change(batch_change_input, status=202)
completed_batch = client.wait_until_batch_change_completed(result) completed_batch = client.wait_until_batch_change_completed(result)
to_delete = [(change['zoneId'], change['recordSetId']) for change in completed_batch['changes']] to_delete = [(change['zoneId'], change['recordSetId']) for change in completed_batch['changes']]
error = client.reject_batch_change(completed_batch['id'], status=403) error = client.approve_batch_change(completed_batch['id'], status=403)
assert_that(error, is_("User does not have access to item " + completed_batch['id'])) assert_that(error, is_("User does not have access to item " + completed_batch['id']))
finally: finally:
clear_zoneid_rsid_tuple_list(to_delete, client) clear_zoneid_rsid_tuple_list(to_delete, client)

View File

@@ -28,6 +28,7 @@ import org.slf4j.LoggerFactory
import vinyldns.api.backend.CommandHandler import vinyldns.api.backend.CommandHandler
import vinyldns.api.crypto.Crypto import vinyldns.api.crypto.Crypto
import vinyldns.api.domain.AccessValidations import vinyldns.api.domain.AccessValidations
import vinyldns.api.domain.auth.MembershipAuthPrincipalProvider
import vinyldns.api.domain.batch.{BatchChangeConverter, BatchChangeService, BatchChangeValidations} import vinyldns.api.domain.batch.{BatchChangeConverter, BatchChangeService, BatchChangeValidations}
import vinyldns.api.domain.membership._ import vinyldns.api.domain.membership._
import vinyldns.api.domain.record.RecordSetService import vinyldns.api.domain.record.RecordSetService
@@ -128,11 +129,16 @@ object Boot extends App {
loaderResponse.healthChecks) loaderResponse.healthChecks)
val batchChangeConverter = val batchChangeConverter =
new BatchChangeConverter(repositories.batchChangeRepository, messageQueue) new BatchChangeConverter(repositories.batchChangeRepository, messageQueue)
val authPrincipalProvider =
new MembershipAuthPrincipalProvider(
repositories.userRepository,
repositories.membershipRepository)
val batchChangeService = BatchChangeService( val batchChangeService = BatchChangeService(
repositories, repositories,
batchChangeValidations, batchChangeValidations,
batchChangeConverter, batchChangeConverter,
VinylDNSConfig.manualBatchReviewEnabled) VinylDNSConfig.manualBatchReviewEnabled,
authPrincipalProvider)
val collectorRegistry = CollectorRegistry.defaultRegistry val collectorRegistry = CollectorRegistry.defaultRegistry
val vinyldnsService = new VinylDNSService( val vinyldnsService = new VinylDNSService(
membershipService, membershipService,
@@ -142,8 +148,7 @@ object Boot extends App {
recordSetService, recordSetService,
batchChangeService, batchChangeService,
collectorRegistry, collectorRegistry,
repositories.userRepository, authPrincipalProvider
repositories.membershipRepository
) )
DefaultExports.initialize() DefaultExports.initialize()

View File

@@ -49,3 +49,8 @@ final case class BatchChangeNotPendingApproval(id: String) extends BatchChangeEr
def message: String = def message: String =
s"""Batch change $id is not pending approval, so it cannot be rejected.""" s"""Batch change $id is not pending approval, so it cannot be rejected."""
} }
final case class BatchRequesterNotFound(userId: String, userName: String) extends BatchChangeErrorResponse {
def message: String =
s"The requesting user with id $userId and name $userName cannot be found in VinylDNS"
}

View File

@@ -29,6 +29,16 @@ final case class BatchChangeInput(
changes: List[ChangeInput], changes: List[ChangeInput],
ownerGroupId: Option[String] = None) ownerGroupId: Option[String] = None)
object BatchChangeInput {
def apply(batchChange: BatchChange): BatchChangeInput = {
val changes = batchChange.changes.map {
case add: SingleAddChange => AddChangeInput(add)
case del: SingleDeleteChange => DeleteChangeInput(del)
}
new BatchChangeInput(batchChange.comments, changes, batchChange.ownerGroupId)
}
}
sealed trait ChangeInput { sealed trait ChangeInput {
val inputName: String val inputName: String
val typ: RecordType val typ: RecordType
@@ -89,6 +99,9 @@ object AddChangeInput {
} }
new AddChangeInput(transformName, typ, ttl, record) new AddChangeInput(transformName, typ, ttl, record)
} }
def apply(sc: SingleAddChange): AddChangeInput =
AddChangeInput(sc.inputName, sc.typ, Some(sc.ttl), sc.recordData)
} }
object DeleteChangeInput { object DeleteChangeInput {
@@ -99,6 +112,9 @@ object DeleteChangeInput {
} }
new DeleteChangeInput(transformName, typ) new DeleteChangeInput(transformName, typ)
} }
def apply(sc: SingleDeleteChange): DeleteChangeInput =
DeleteChangeInput(sc.inputName, sc.typ)
} }
object ChangeInputType extends Enumeration { object ChangeInputType extends Enumeration {

View File

@@ -23,6 +23,7 @@ import cats.implicits._
import org.joda.time.DateTime import org.joda.time.DateTime
import org.slf4j.{Logger, LoggerFactory} import org.slf4j.{Logger, LoggerFactory}
import vinyldns.api.domain.DomainValidations._ import vinyldns.api.domain.DomainValidations._
import vinyldns.api.domain.auth.AuthPrincipalProvider
import vinyldns.api.domain.batch.BatchChangeInterfaces._ import vinyldns.api.domain.batch.BatchChangeInterfaces._
import vinyldns.api.domain.batch.BatchTransformations._ import vinyldns.api.domain.batch.BatchTransformations._
import vinyldns.api.domain.dns.DnsConversions._ import vinyldns.api.domain.dns.DnsConversions._
@@ -42,7 +43,8 @@ object BatchChangeService {
dataAccessor: ApiDataAccessor, dataAccessor: ApiDataAccessor,
batchChangeValidations: BatchChangeValidationsAlgebra, batchChangeValidations: BatchChangeValidationsAlgebra,
batchChangeConverter: BatchChangeConverterAlgebra, batchChangeConverter: BatchChangeConverterAlgebra,
manualReviewEnabled: Boolean): BatchChangeService = manualReviewEnabled: Boolean,
authProvider: AuthPrincipalProvider): BatchChangeService =
new BatchChangeService( new BatchChangeService(
dataAccessor.zoneRepository, dataAccessor.zoneRepository,
dataAccessor.recordSetRepository, dataAccessor.recordSetRepository,
@@ -50,7 +52,8 @@ object BatchChangeService {
batchChangeValidations, batchChangeValidations,
dataAccessor.batchChangeRepository, dataAccessor.batchChangeRepository,
batchChangeConverter, batchChangeConverter,
manualReviewEnabled manualReviewEnabled,
authProvider
) )
} }
@@ -61,7 +64,8 @@ class BatchChangeService(
batchChangeValidations: BatchChangeValidationsAlgebra, batchChangeValidations: BatchChangeValidationsAlgebra,
batchChangeRepo: BatchChangeRepository, batchChangeRepo: BatchChangeRepository,
batchChangeConverter: BatchChangeConverterAlgebra, batchChangeConverter: BatchChangeConverterAlgebra,
manualReviewEnabled: Boolean) manualReviewEnabled: Boolean,
authProvider: AuthPrincipalProvider)
extends BatchChangeServiceAlgebra { extends BatchChangeServiceAlgebra {
import batchChangeValidations._ import batchChangeValidations._
@@ -111,6 +115,11 @@ class BatchChangeService(
for { for {
batchChange <- getExistingBatchChange(batchChangeId) batchChange <- getExistingBatchChange(batchChangeId)
_ <- validateBatchChangeApproval(batchChange, authPrincipal).toBatchResult _ <- validateBatchChangeApproval(batchChange, authPrincipal).toBatchResult
_ = BatchChangeInput(batchChange)
_ <- EitherT.fromOptionF[IO, BatchChangeErrorResponse, AuthPrincipal](
authProvider.getAuthPrincipalByUserId(batchChange.userId),
BatchRequesterNotFound(batchChange.userId, batchChange.userName)
)
} yield batchChange } yield batchChange
def getBatchChange(id: String, auth: AuthPrincipal): BatchResult[BatchChangeInfo] = def getBatchChange(id: String, auth: AuthPrincipal): BatchResult[BatchChangeInfo] =

View File

@@ -58,7 +58,7 @@ trait BatchChangeJsonProtocol extends JsonValidation {
(js \ "comments").optional[String], (js \ "comments").optional[String],
changeList, changeList,
(js \ "ownerGroupId").optional[String] (js \ "ownerGroupId").optional[String]
).mapN(BatchChangeInput) ).mapN(BatchChangeInput(_, _, _))
} }
} }

View File

@@ -137,5 +137,6 @@ trait BatchChangeRoute extends Directives {
case Left(bcnpa: BatchChangeNotPendingApproval) => case Left(bcnpa: BatchChangeNotPendingApproval) =>
complete(StatusCodes.BadRequest, bcnpa.message) complete(StatusCodes.BadRequest, bcnpa.message)
case Left(uce: UnknownConversionError) => complete(StatusCodes.InternalServerError, uce) case Left(uce: UnknownConversionError) => complete(StatusCodes.InternalServerError, uce)
case Left(brnf: BatchRequesterNotFound) => complete(StatusCodes.NotFound, brnf.message)
} }
} }

View File

@@ -25,12 +25,11 @@ import akka.http.scaladsl.server.directives.LogEntry
import cats.effect.IO import cats.effect.IO
import fs2.concurrent.SignallingRef import fs2.concurrent.SignallingRef
import io.prometheus.client.CollectorRegistry import io.prometheus.client.CollectorRegistry
import vinyldns.api.domain.auth.MembershipAuthPrincipalProvider import vinyldns.api.domain.auth.AuthPrincipalProvider
import vinyldns.api.domain.batch.BatchChangeServiceAlgebra import vinyldns.api.domain.batch.BatchChangeServiceAlgebra
import vinyldns.api.domain.membership.MembershipServiceAlgebra import vinyldns.api.domain.membership.MembershipServiceAlgebra
import vinyldns.api.domain.record.RecordSetServiceAlgebra import vinyldns.api.domain.record.RecordSetServiceAlgebra
import vinyldns.api.domain.zone.ZoneServiceAlgebra import vinyldns.api.domain.zone.ZoneServiceAlgebra
import vinyldns.core.domain.membership.{MembershipRepository, UserRepository}
import vinyldns.core.health.HealthService import vinyldns.core.health.HealthService
import scala.util.matching.Regex import scala.util.matching.Regex
@@ -120,8 +119,7 @@ class VinylDNSService(
val recordSetService: RecordSetServiceAlgebra, val recordSetService: RecordSetServiceAlgebra,
val batchChangeService: BatchChangeServiceAlgebra, val batchChangeService: BatchChangeServiceAlgebra,
val collectorRegistry: CollectorRegistry, val collectorRegistry: CollectorRegistry,
userRepository: UserRepository, authPrincipalProvider: AuthPrincipalProvider)
membershipRepository: MembershipRepository)
extends VinylDNSDirectives extends VinylDNSDirectives
with PingRoute with PingRoute
with ZoneRoute with ZoneRoute
@@ -136,8 +134,6 @@ class VinylDNSService(
with JsonValidationRejection { with JsonValidationRejection {
val aws4Authenticator = new Aws4Authenticator val aws4Authenticator = new Aws4Authenticator
val authPrincipalProvider =
new MembershipAuthPrincipalProvider(userRepository, membershipRepository)
val vinylDNSAuthenticator: VinylDNSAuthenticator = val vinylDNSAuthenticator: VinylDNSAuthenticator =
new ProductionVinylDNSAuthenticator(aws4Authenticator, authPrincipalProvider) new ProductionVinylDNSAuthenticator(aws4Authenticator, authPrincipalProvider)

View File

@@ -17,10 +17,11 @@
package vinyldns.api.domain.batch package vinyldns.api.domain.batch
import cats.data.NonEmptyList import cats.data.NonEmptyList
import org.joda.time.DateTime
import org.scalatest.{Matchers, WordSpec} import org.scalatest.{Matchers, WordSpec}
import vinyldns.api.VinylDNSConfig import vinyldns.api.VinylDNSConfig
import vinyldns.core.domain.ZoneDiscoveryError import vinyldns.core.domain.{DomainValidationErrorType, SingleChangeError, ZoneDiscoveryError}
import vinyldns.core.domain.batch.{SingleAddChange, SingleChangeStatus, SingleDeleteChange} import vinyldns.core.domain.batch._
import vinyldns.core.domain.record.RecordType._ import vinyldns.core.domain.record.RecordType._
import vinyldns.core.domain.record.{AAAAData, AData, CNAMEData} import vinyldns.core.domain.record.{AAAAData, AData, CNAMEData}
@@ -88,4 +89,59 @@ class BatchChangeInputSpec extends WordSpec with Matchers {
asDelete.recordSetId shouldBe None asDelete.recordSetId shouldBe None
} }
} }
"apply from SingleChange" should {
"properly convert changes to adds and deletes" in {
val singleAddChange = SingleAddChange(
Some("testZoneId"),
Some("testZoneName"),
Some("testRname"),
"testRname.testZoneName.",
A,
1234,
AData("1.2.3.4"),
SingleChangeStatus.NeedsReview,
Some("msg"),
None,
None,
List(SingleChangeError(DomainValidationErrorType.ZoneDiscoveryError, "test err"))
)
val expectedAddChange =
AddChangeInput("testRname.testZoneName.", A, Some(1234), AData("1.2.3.4"))
val singleDelChange = SingleDeleteChange(
Some("testZoneId"),
Some("testZoneName"),
Some("testRname"),
"testRname.testZoneName.",
A,
SingleChangeStatus.NeedsReview,
Some("msg"),
None,
None,
List(SingleChangeError(DomainValidationErrorType.ZoneDiscoveryError, "test err"))
)
val expectedDelChange =
DeleteChangeInput("testRname.testZoneName.", A)
val change = BatchChange(
"userId",
"userName",
Some("comments"),
DateTime.now(),
List(singleAddChange, singleDelChange),
Some("owner"),
BatchChangeApprovalStatus.PendingApproval
)
val expectedInput =
BatchChangeInput(
Some("comments"),
List(expectedAddChange, expectedDelChange),
Some("owner"))
BatchChangeInput(change) shouldBe expectedInput
}
}
} }

View File

@@ -24,6 +24,7 @@ import org.joda.time.DateTime
import org.scalatest.{BeforeAndAfterEach, EitherValues, Matchers, WordSpec} import org.scalatest.{BeforeAndAfterEach, EitherValues, Matchers, WordSpec}
import vinyldns.api.ValidatedBatchMatcherImprovements.containChangeForValidation import vinyldns.api.ValidatedBatchMatcherImprovements.containChangeForValidation
import vinyldns.api._ import vinyldns.api._
import vinyldns.api.domain.auth.AuthPrincipalProvider
import vinyldns.api.domain.batch.BatchChangeInterfaces.{BatchResult, _} import vinyldns.api.domain.batch.BatchChangeInterfaces.{BatchResult, _}
import vinyldns.api.domain.batch.BatchTransformations._ import vinyldns.api.domain.batch.BatchTransformations._
import vinyldns.api.domain.{AccessValidations, _} import vinyldns.api.domain.{AccessValidations, _}
@@ -233,6 +234,16 @@ class BatchChangeServiceSpec
IO.pure(dbZones.filter(z => zoneNames.exists(z.name.endsWith))) IO.pure(dbZones.filter(z => zoneNames.exists(z.name.endsWith)))
} }
object TestAuth extends AuthPrincipalProvider {
def getAuthPrincipal(accessKey: String): IO[Option[AuthPrincipal]] = IO.pure(None)
def getAuthPrincipalByUserId(userId: String): IO[Option[AuthPrincipal]] =
userId match {
case okAuth.userId => IO.pure(Some(okAuth))
case _ => IO.pure(None)
}
}
private val underTest = new BatchChangeService( private val underTest = new BatchChangeService(
TestZoneRepo, TestZoneRepo,
TestRecordSetRepo, TestRecordSetRepo,
@@ -240,7 +251,8 @@ class BatchChangeServiceSpec
validations, validations,
batchChangeRepo, batchChangeRepo,
EmptyBatchConverter, EmptyBatchConverter,
false) false,
TestAuth)
private val underTestManualEnabled = new BatchChangeService( private val underTestManualEnabled = new BatchChangeService(
TestZoneRepo, TestZoneRepo,
@@ -249,7 +261,8 @@ class BatchChangeServiceSpec
validations, validations,
batchChangeRepo, batchChangeRepo,
EmptyBatchConverter, EmptyBatchConverter,
true) true,
TestAuth)
"applyBatchChange" should { "applyBatchChange" should {
"succeed if all inputs are good" in { "succeed if all inputs are good" in {
@@ -484,6 +497,26 @@ class BatchChangeServiceSpec
result shouldBe UserNotAuthorizedError(batchChange.id) result shouldBe UserNotAuthorizedError(batchChange.id)
} }
"fail if the requesting user cannot be found" in {
val batchChange =
BatchChange(
"someOtherUserId",
"someUn",
None,
DateTime.now,
List(),
approvalStatus = BatchChangeApprovalStatus.PendingApproval)
batchChangeRepo.save(batchChange)
val result =
leftResultOf(
underTest
.approveBatchChange(batchChange.id, superUserAuth, ApproveBatchChangeInput())
.value)
result shouldBe BatchRequesterNotFound("someOtherUserId", "someUn")
}
} }
"getBatchChange" should { "getBatchChange" should {
@@ -688,7 +721,8 @@ class BatchChangeServiceSpec
validations, validations,
batchChangeRepo, batchChangeRepo,
EmptyBatchConverter, EmptyBatchConverter,
false) false,
TestAuth)
val ip = "2001:0db8:0000:0000:0000:ff00:0042:8329" val ip = "2001:0db8:0000:0000:0000:ff00:0042:8329"
val possibleZones = List( val possibleZones = List(
@@ -722,7 +756,8 @@ class BatchChangeServiceSpec
validations, validations,
batchChangeRepo, batchChangeRepo,
EmptyBatchConverter, EmptyBatchConverter,
false) false,
TestAuth)
val ip1 = "::1" val ip1 = "::1"
val possibleZones1 = (5 to 16).map(num0s => ("0." * num0s) + "ip6.arpa.") val possibleZones1 = (5 to 16).map(num0s => ("0." * num0s) + "ip6.arpa.")

View File

@@ -365,6 +365,7 @@ class BatchChangeRoutingSpec
case ("pendingBatchId", true) => EitherT(IO.pure(genericValidResponse.asRight)) case ("pendingBatchId", true) => EitherT(IO.pure(genericValidResponse.asRight))
case ("pendingBatchId", false) => case ("pendingBatchId", false) =>
EitherT(IO.pure(UserNotAuthorizedError("notAuthedID").asLeft)) EitherT(IO.pure(UserNotAuthorizedError("notAuthedID").asLeft))
case ("notFoundUser", _) => EitherT(IO.pure(BatchRequesterNotFound("someid", "somename").asLeft))
case (_, _) => EitherT(IO.pure(BatchChangeNotPendingApproval("batchId").asLeft)) case (_, _) => EitherT(IO.pure(BatchChangeNotPendingApproval("batchId").asLeft))
} }
} }
@@ -715,5 +716,14 @@ class BatchChangeRoutingSpec
status shouldBe BadRequest status shouldBe BadRequest
} }
} }
"return NotFound if the requesting user cant be found" in {
Post("/zones/batchrecordchanges/notFoundUser/approve").withEntity(HttpEntity(
ContentTypes.`application/json`,
compact(render("comments" -> "some comments")))) ~>
batchChangeRoute(supportUserAuth) ~> check {
status shouldBe NotFound
}
}
} }
} }