2
0
mirror of https://github.com/VinylDNS/vinyldns synced 2025-08-31 14:25:30 +00:00

Properly reject requests sent to non-existent routes (#739)

* Refactor routing
This commit is contained in:
Michael Ly
2019-07-23 12:19:39 -04:00
committed by GitHub
parent a520bcca4d
commit 566ae1f3da
19 changed files with 869 additions and 752 deletions

View File

@@ -3,6 +3,7 @@ from hamcrest import *
from vinyldns_python import VinylDNSClient
from dns.resolver import *
from vinyldns_context import VinylDNSTestContext
from requests.compat import urljoin
def test_request_fails_when_user_account_is_locked():
@@ -27,3 +28,23 @@ def test_request_succeeds_when_user_is_found_and_not_locked():
client = VinylDNSClient(VinylDNSTestContext.vinyldns_url, 'okAccessKey', 'okSecretKey')
client.list_batch_change_summaries(status=200)
def test_request_fails_when_accessing_non_existent_route():
"""
Test request fails with NotFound (404) when route cannot be resolved, regardless of authentication
"""
client = VinylDNSClient(VinylDNSTestContext.vinyldns_url, 'unknownAccessKey', 'anyAccessSecretKey')
url = urljoin(VinylDNSTestContext.vinyldns_url, u'/no-existo')
_, data = client.make_request(url, u'GET', client.headers, status=404)
assert_that(data, is_("The requested path [/no-existo] does not exist."))
def test_request_fails_with_unsupported_http_method_for_route():
"""
Test request fails with MethodNotAllowed (405) when HTTP Method is not supported for specified route
"""
client = VinylDNSClient(VinylDNSTestContext.vinyldns_url, 'unknownAccessKey', 'anyAccessSecretKey')
url = urljoin(VinylDNSTestContext.vinyldns_url, u'/zones')
_, data = client.make_request(url, u'PUT', client.headers, status=405)
assert_that(data, is_("HTTP method not allowed, supported methods: GET, POST"))

View File

@@ -219,3 +219,26 @@ def test_create_group_no_members(shared_zone_test_context):
finally:
if result:
client.delete_group(result['id'], status=(200,404))
def test_create_group_adds_admins_to_member_list(shared_zone_test_context):
"""
Tests that creating a group adds admins to member list
"""
client = shared_zone_test_context.ok_vinyldns_client
result = None
try:
new_group = {
'name': 'test-create-group-add-admins-to-members',
'email': 'test@test.com',
'description': 'this is a description',
'members': [ {'id': 'ok'} ],
'admins': [ {'id': 'dummy'} ]
}
result = client.create_group(new_group, status=200)
assert_that(map(lambda x: x['id'], result['members']), contains('ok', 'dummy'))
assert_that(result['admins'][0]['id'], is_('dummy'))
finally:
if result:
client.delete_group(result['id'], status=(200,404))

View File

@@ -614,3 +614,31 @@ def test_update_group_not_authorized(shared_zone_test_context):
finally:
if saved_group:
ok_client.delete_group(saved_group['id'], status=(200,404))
def test_update_group_adds_admins_to_member_list(shared_zone_test_context):
"""
Tests that updating a group adds admins to member list
"""
ok_client = shared_zone_test_context.ok_vinyldns_client
dummy_client = shared_zone_test_context.dummy_vinyldns_client
result = None
try:
new_group = {
'name': 'test-update-group-add-admins-to-members',
'email': 'test@test.com',
'description': 'this is a description',
'members': [ {'id': 'ok'} ],
'admins': [ {'id': 'ok'} ]
}
saved_group = ok_client.create_group(new_group, status=200)
saved_group['admins'] = [ { 'id': 'dummy' }]
result = ok_client.update_group(saved_group['id'], saved_group, status=200)
assert_that(map(lambda x: x['id'], result['members']), contains('ok', 'dummy'))
assert_that(result['admins'][0]['id'], is_('dummy'))
finally:
if result:
dummy_client.delete_group(result['id'], status=(200,404))

View File

@@ -17,45 +17,46 @@
package vinyldns.api.route
import akka.http.scaladsl.model.StatusCodes
import akka.http.scaladsl.server
import akka.http.scaladsl.server.{Directives, RejectionHandler, Route, ValidationRejection}
import cats.data.EitherT
import cats.effect._
import akka.http.scaladsl.server.{RejectionHandler, Route, ValidationRejection}
import vinyldns.api.VinylDNSConfig
import vinyldns.core.domain.auth.AuthPrincipal
import vinyldns.core.domain.batch._
import vinyldns.api.domain.batch._
trait BatchChangeRoute extends Directives {
this: VinylDNSJsonProtocol with VinylDNSDirectives with JsonValidationRejection =>
class BatchChangeRoute(
batchChangeService: BatchChangeServiceAlgebra,
val vinylDNSAuthenticator: VinylDNSAuthenticator)
extends VinylDNSJsonProtocol
with VinylDNSDirectives[BatchChangeErrorResponse] {
val batchChangeService: BatchChangeServiceAlgebra
def getRoutes: Route = batchChangeRoute
def handleErrors(e: BatchChangeErrorResponse): PartialFunction[BatchChangeErrorResponse, Route] = {
case ibci: InvalidBatchChangeInput => complete(StatusCodes.BadRequest, ibci)
case crl: InvalidBatchChangeResponses => complete(StatusCodes.BadRequest, crl)
case cnf: BatchChangeNotFound => complete(StatusCodes.NotFound, cnf.message)
case una: UserNotAuthorizedError => complete(StatusCodes.Forbidden, una.message)
case uct: BatchConversionError => complete(StatusCodes.BadRequest, uct)
case bcnpa: BatchChangeNotPendingApproval =>
complete(StatusCodes.BadRequest, bcnpa.message)
case uce: UnknownConversionError => complete(StatusCodes.InternalServerError, uce)
case brnf: BatchRequesterNotFound => complete(StatusCodes.NotFound, brnf.message)
}
final private val MAX_ITEMS_LIMIT: Int = 100
val batchChangeRoute: AuthPrincipal => server.Route = { authPrincipal: AuthPrincipal =>
val standardBatchChangeRoutes = (post & path("zones" / "batchrecordchanges")) {
val batchChangeRoute: Route = {
val standardBatchChangeRoutes = path("zones" / "batchrecordchanges") {
(post & monitor("Endpoint.postBatchChange")) {
parameters("allowManualReview".as[Boolean].?(true)) { allowManualReview: Boolean =>
{
monitor("Endpoint.postBatchChange") {
entity(as[BatchChangeInput]) { batchChangeInput =>
execute(batchChangeService
authenticateAndExecuteWithEntity[BatchChange, BatchChangeInput](
(authPrincipal, batchChangeInput) =>
batchChangeService
.applyBatchChange(batchChangeInput, authPrincipal, allowManualReview)) { chg =>
complete(StatusCodes.Accepted, chg)
}
}
}
}
}
} ~
(get & path("zones" / "batchrecordchanges" / Segment)) { id =>
monitor("Endpoint.getBatchChange") {
execute(batchChangeService.getBatchChange(id, authPrincipal)) { chg =>
complete(StatusCodes.OK, chg)
}
}
} ~
(get & path("zones" / "batchrecordchanges") & monitor("Endpoint.listBatchChangeSummaries")) {
(get & monitor("Endpoint.listBatchChangeSummaries")) {
parameters(
"startFrom".as[Int].?,
"maxItems".as[Int].?(MAX_ITEMS_LIMIT),
@@ -73,9 +74,9 @@ trait BatchChangeRoute extends Directives {
validate(
0 < maxItems && maxItems <= MAX_ITEMS_LIMIT,
s"maxItems was $maxItems, maxItems must be between 1 and $MAX_ITEMS_LIMIT, inclusive.") {
execute(
authenticateAndExecute(
batchChangeService.listBatchChangeSummaries(
authPrincipal,
_,
startFrom,
maxItems,
ignoreAccess,
@@ -87,12 +88,20 @@ trait BatchChangeRoute extends Directives {
}
}
}
} ~
path("zones" / "batchrecordchanges" / Segment) { id =>
(get & monitor("Endpoint.getBatchChange")) {
authenticateAndExecute(batchChangeService.getBatchChange(id, _)) { chg =>
complete(StatusCodes.OK, chg)
}
}
}
val manualBatchReviewRoutes =
(post & path("zones" / "batchrecordchanges" / Segment / "reject")) { id =>
monitor("Endpoint.rejectBatchChange") {
entity(as[Option[RejectBatchChangeInput]]) { input =>
execute(
path("zones" / "batchrecordchanges" / Segment / "reject") { id =>
(post & monitor("Endpoint.rejectBatchChange")) {
authenticateAndExecuteWithEntity[BatchChange, Option[RejectBatchChangeInput]](
(authPrincipal, input) =>
batchChangeService
.rejectBatchChange(id, authPrincipal, input.getOrElse(RejectBatchChangeInput()))) {
chg =>
@@ -100,19 +109,17 @@ trait BatchChangeRoute extends Directives {
}
// TODO: Update response entity to return modified batch change
}
}
} ~
(post & path("zones" / "batchrecordchanges" / Segment / "approve")) { id =>
monitor("Endpoint.approveBatchChange") {
entity(as[Option[ApproveBatchChangeInput]]) { input =>
execute(
path("zones" / "batchrecordchanges" / Segment / "approve") { id =>
(post & monitor("Endpoint.approveBatchChange")) {
authenticateAndExecuteWithEntity[BatchChange, Option[ApproveBatchChangeInput]](
(authPrincipal, input) =>
batchChangeService
.approveBatchChange(
id,
authPrincipal,
input.getOrElse(ApproveBatchChangeInput()))) { chg =>
complete(StatusCodes.OK, chg)
}
// TODO: Update response entity to return modified batch change
}
}
@@ -130,18 +137,4 @@ trait BatchChangeRoute extends Directives {
complete(StatusCodes.BadRequest, msg)
}
.result()
private def execute[A](f: => EitherT[IO, BatchChangeErrorResponse, A])(rt: A => Route): Route =
onSuccess(f.value.unsafeToFuture()) {
case Right(a) => rt(a)
case Left(ibci: InvalidBatchChangeInput) => complete(StatusCodes.BadRequest, ibci)
case Left(crl: InvalidBatchChangeResponses) => complete(StatusCodes.BadRequest, crl)
case Left(cnf: BatchChangeNotFound) => complete(StatusCodes.NotFound, cnf.message)
case Left(una: UserNotAuthorizedError) => complete(StatusCodes.Forbidden, una.message)
case Left(uct: BatchConversionError) => complete(StatusCodes.BadRequest, uct)
case Left(bcnpa: BatchChangeNotPendingApproval) =>
complete(StatusCodes.BadRequest, bcnpa.message)
case Left(uce: UnknownConversionError) => complete(StatusCodes.InternalServerError, uce)
case Left(brnf: BatchRequesterNotFound) => complete(StatusCodes.NotFound, brnf.message)
}
}

View File

@@ -18,9 +18,9 @@ package vinyldns.api.route
import java.util.NoSuchElementException
import akka.http.scaladsl.model.{ContentTypes, HttpEntity, HttpResponse, StatusCodes}
import akka.http.scaladsl.server.{Directives, MalformedRequestContentRejection, RejectionHandler}
import cats.data.Validated.{Invalid, Valid}
import cats.data._
import cats.implicits._
import com.fasterxml.jackson.core.JsonParseException
import de.heikoseeberger.akkahttpjson4s.Json4sSupport
import org.joda.time.DateTime
@@ -30,26 +30,9 @@ import org.json4s.ext._
import org.json4s.jackson.JsonMethods._
import scala.reflect.ClassTag
import cats.data._
import cats.implicits._
case class JsonErrors(errors: List[String])
trait JsonValidationRejection extends Directives {
implicit def validationRejectionHandler: RejectionHandler =
RejectionHandler
.newBuilder()
.handle {
case MalformedRequestContentRejection(msg, MappingException(_, _)) =>
complete(
HttpResponse(
status = StatusCodes.BadRequest,
entity = HttpEntity(ContentTypes.`application/json`, msg)
))
}
.result()
}
// TODO: An update to json4s changed the date time formatting. In order to stay compatible, had to
// revert the date time formatting here. When changing to circe (updating to java8 instant),
// be sure to check the format of date time

View File

@@ -18,61 +18,60 @@ package vinyldns.api.route
import akka.http.scaladsl.model.StatusCodes
import akka.http.scaladsl.server._
import vinyldns.api.Interfaces.Result
import vinyldns.api.domain.membership._
import vinyldns.api.domain.zone.NotAuthorizedError
import vinyldns.api.route.MembershipJsonProtocol.{CreateGroupInput, UpdateGroupInput}
import vinyldns.core.domain.auth.AuthPrincipal
import vinyldns.core.domain.membership.{Group, LockStatus}
trait MembershipRoute extends Directives {
this: VinylDNSJsonProtocol with VinylDNSDirectives with JsonValidationRejection =>
class MembershipRoute(
membershipService: MembershipServiceAlgebra,
val vinylDNSAuthenticator: VinylDNSAuthenticator)
extends VinylDNSJsonProtocol
with VinylDNSDirectives[Throwable] {
final private val DEFAULT_MAX_ITEMS: Int = 100
final private val MAX_ITEMS_LIMIT: Int = 1000
val membershipService: MembershipServiceAlgebra
def getRoutes: Route = membershipRoute
val membershipRoute = { authPrincipal: AuthPrincipal =>
path("groups" / Segment) { groupId =>
get {
monitor("Endpoint.getGroup") {
execute(membershipService.getGroup(groupId, authPrincipal)) { group =>
complete(StatusCodes.OK, GroupInfo(group))
def handleErrors(e: Throwable): PartialFunction[Throwable, Route] = {
case GroupNotFoundError(msg) => complete(StatusCodes.NotFound, msg)
case NotAuthorizedError(msg) => complete(StatusCodes.Forbidden, msg)
case GroupAlreadyExistsError(msg) => complete(StatusCodes.Conflict, msg)
case InvalidGroupError(msg) => complete(StatusCodes.BadRequest, msg)
case UserNotFoundError(msg) => complete(StatusCodes.NotFound, msg)
case InvalidGroupRequestError(msg) => complete(StatusCodes.BadRequest, msg)
}
val membershipRoute: Route = path("groups" / Segment) { groupId =>
(get & monitor("Endpoint.getGroup")) {
authenticateAndExecute(membershipService.getGroup(groupId, _)) { group =>
complete(StatusCodes.OK, GroupInfo(group))
}
} ~
delete {
monitor("Endpoint.deleteGroup") {
execute(membershipService.deleteGroup(groupId, authPrincipal)) { group =>
(delete & monitor("Endpoint.deleteGroup")) {
authenticateAndExecute(membershipService.deleteGroup(groupId, _)) { group =>
complete(StatusCodes.OK, GroupInfo(group))
}
}
}
} ~
path("groups") {
post {
monitor("Endpoint.createGroup") {
entity(as[CreateGroupInput]) { input =>
ifValid(
Group
.build(
(post & monitor("Endpoint.createGroup")) {
authenticateAndExecuteWithEntity[Group, CreateGroupInput] { (authPrincipal, input) =>
val group = Group(
input.name,
input.email,
input.description,
input.members.map(_.id),
input.admins.map(_.id))) { inputGroup: Group =>
execute(membershipService.createGroup(inputGroup, authPrincipal)) { group =>
memberIds = (input.members ++ input.admins).map(_.id),
adminUserIds = input.admins.map(_.id))
membershipService.createGroup(group, authPrincipal)
} { group =>
complete(StatusCodes.OK, GroupInfo(group))
}
}
}
}
} ~
get {
(get & monitor("Endpoint.listMyGroups")) {
parameters("startFrom".?, "maxItems".as[Int].?(DEFAULT_MAX_ITEMS), "groupNameFilter".?) {
(startFrom: Option[String], maxItems: Int, groupNameFilter: Option[String]) =>
{
monitor("Endpoint.listMyGroups") {
handleRejections(invalidQueryHandler) {
validate(
check = 0 < maxItems && maxItems <= MAX_ITEMS_LIMIT,
@@ -81,9 +80,8 @@ trait MembershipRoute extends Directives {
| and $MAX_ITEMS_LIMIT inclusive"
""".stripMargin
) {
execute(membershipService
.listMyGroups(groupNameFilter, startFrom, maxItems, authPrincipal)) {
groups =>
authenticateAndExecute(membershipService
.listMyGroups(groupNameFilter, startFrom, maxItems, _)) { groups =>
complete(StatusCodes.OK, groups)
}
}
@@ -91,92 +89,75 @@ trait MembershipRoute extends Directives {
}
}
}
}
} ~
path("groups" / Segment) { _ =>
put {
monitor("Endpoint.updateGroup") {
entity(as[UpdateGroupInput]) { input =>
ifValid(
Group.build(
(put & monitor("Endpoint.updateGroup")) {
authenticateAndExecuteWithEntity[Group, UpdateGroupInput](
(authPrincipal, input) =>
membershipService.updateGroup(
input.id,
input.name,
input.email,
input.description,
input.members.map(_.id),
input.admins.map(_.id))) { inputGroup: Group =>
execute(
membershipService.updateGroup(
inputGroup.id,
inputGroup.name,
inputGroup.email,
inputGroup.description,
inputGroup.memberIds,
inputGroup.adminUserIds,
(input.members ++ input.admins).map(_.id),
input.admins.map(_.id),
authPrincipal)) { group =>
complete(StatusCodes.OK, GroupInfo(group))
}
}
}
}
}
} ~
path("groups" / Segment / "members") { groupId =>
get {
monitor("Endpoint.listGroupMembers") {
(get & monitor("Endpoint.listGroupMembers")) {
parameters("startFrom".?, "maxItems".as[Int].?(DEFAULT_MAX_ITEMS)) {
(startFrom: Option[String], maxItems: Int) =>
handleRejections(invalidQueryHandler) {
validate(
0 < maxItems && maxItems <= MAX_ITEMS_LIMIT,
s"maxItems was $maxItems, maxItems must be between 0 exclusive and $MAX_ITEMS_LIMIT inclusive") {
execute(
membershipService.listMembers(groupId, startFrom, maxItems, authPrincipal)) {
members =>
authenticateAndExecute(membershipService
.listMembers(groupId, startFrom, maxItems, _)) { members =>
complete(StatusCodes.OK, members)
}
}
}
}
}
}
} ~
path("groups" / Segment / "admins") { groupId =>
get {
monitor("Endpoint.listGroupAdmins") {
execute(membershipService.listAdmins(groupId, authPrincipal)) { admins =>
(get & monitor("Endpoint.listGroupAdmins")) {
authenticateAndExecute(membershipService.listAdmins(groupId, _)) { admins =>
complete(StatusCodes.OK, admins)
}
}
}
} ~
path("groups" / Segment / "activity") { groupId =>
get {
monitor("Endpoint.groupActivity") {
(get & monitor("Endpoint.groupActivity")) {
parameters("startFrom".?, "maxItems".as[Int].?(DEFAULT_MAX_ITEMS)) {
(startFrom: Option[String], maxItems: Int) =>
handleRejections(invalidQueryHandler) {
validate(
0 < maxItems && maxItems <= MAX_ITEMS_LIMIT,
s"maxItems was $maxItems, maxItems must be between 0 and $MAX_ITEMS_LIMIT") {
execute(membershipService
.getGroupActivity(groupId, startFrom, maxItems, authPrincipal)) { activity =>
authenticateAndExecute(membershipService
.getGroupActivity(groupId, startFrom, maxItems, _)) { activity =>
complete(StatusCodes.OK, activity)
}
}
}
}
}
}
} ~
(put & path("users" / Segment / "lock") & monitor("Endpoint.lockUser")) { id =>
execute(membershipService.updateUserLockStatus(id, LockStatus.Locked, authPrincipal)) {
path("users" / Segment / "lock") { id =>
(put & monitor("Endpoint.lockUser")) {
authenticateAndExecute(membershipService.updateUserLockStatus(id, LockStatus.Locked, _)) {
user =>
complete(StatusCodes.OK, UserInfo(user))
}
}
} ~
(put & path("users" / Segment / "unlock") & monitor("Endpoint.unlockUser")) { id =>
execute(membershipService.updateUserLockStatus(id, LockStatus.Unlocked, authPrincipal)) {
path("users" / Segment / "unlock") { id =>
(put & monitor("Endpoint.unlockUser")) {
authenticateAndExecute(membershipService.updateUserLockStatus(id, LockStatus.Unlocked, _)) {
user =>
complete(StatusCodes.OK, UserInfo(user))
}
@@ -190,16 +171,4 @@ trait MembershipRoute extends Directives {
complete(StatusCodes.BadRequest, msg)
}
.result()
private def execute[A](f: => Result[A])(rt: A => Route): Route =
onSuccess(f.value.unsafeToFuture()) {
case Right(a) => rt(a)
case Left(GroupNotFoundError(msg)) => complete(StatusCodes.NotFound, msg)
case Left(NotAuthorizedError(msg)) => complete(StatusCodes.Forbidden, msg)
case Left(GroupAlreadyExistsError(msg)) => complete(StatusCodes.Conflict, msg)
case Left(InvalidGroupError(msg)) => complete(StatusCodes.BadRequest, msg)
case Left(UserNotFoundError(msg)) => complete(StatusCodes.NotFound, msg)
case Left(InvalidGroupRequestError(msg)) => complete(StatusCodes.BadRequest, msg)
case Left(e) => failWith(e)
}
}

View File

@@ -17,13 +17,13 @@
package vinyldns.api.route
import akka.http.scaladsl.model.StatusCodes
import akka.http.scaladsl.server.{Directives, RejectionHandler, Route, ValidationRejection}
import akka.http.scaladsl.server.{RejectionHandler, Route, ValidationRejection}
import akka.util.Timeout
import vinyldns.api.Interfaces._
import vinyldns.api.domain.record.RecordSetServiceAlgebra
import vinyldns.api.domain.zone._
import vinyldns.core.domain.auth.AuthPrincipal
import vinyldns.core.domain.record.RecordSet
import vinyldns.core.domain.zone.ZoneCommandResult
import scala.concurrent.duration._
@@ -35,95 +35,92 @@ case class ListRecordSetsResponse(
maxItems: Option[Int] = None,
recordNameFilter: Option[String] = None)
trait RecordSetRoute extends Directives {
this: VinylDNSJsonProtocol with VinylDNSDirectives with JsonValidationRejection =>
final private val DEFAULT_MAX_ITEMS: Int = 100
class RecordSetRoute(
recordSetService: RecordSetServiceAlgebra,
val vinylDNSAuthenticator: VinylDNSAuthenticator)
extends VinylDNSJsonProtocol
with VinylDNSDirectives[Throwable] {
val recordSetService: RecordSetServiceAlgebra
def getRoutes: Route = recordSetRoute
final private val DEFAULT_MAX_ITEMS: Int = 100
// Timeout must be long enough to allow the cluster to form
implicit val rsCmdTimeout: Timeout = Timeout(10.seconds)
val recordSetRoute = { authPrincipal: AuthPrincipal =>
path("zones" / Segment / "recordsets") { zoneId =>
post {
monitor("Endpoint.addRecordSet") {
entity(as[RecordSet]) { rs =>
execute(recordSetService.addRecordSet(rs, authPrincipal)) { rc =>
def handleErrors(e: Throwable): PartialFunction[Throwable, Route] = {
case ZoneNotFoundError(msg) => complete(StatusCodes.NotFound, msg)
case RecordSetAlreadyExists(msg) => complete(StatusCodes.Conflict, msg)
case ZoneInactiveError(msg) => complete(StatusCodes.BadRequest, msg)
case NotAuthorizedError(msg) => complete(StatusCodes.Forbidden, msg)
case ZoneUnavailableError(msg) => complete(StatusCodes.Conflict, msg)
case RecordSetNotFoundError(msg) => complete(StatusCodes.NotFound, msg)
case InvalidRequest(msg) => complete(StatusCodes.UnprocessableEntity, msg)
case PendingUpdateError(msg) => complete(StatusCodes.Conflict, msg)
case RecordSetChangeNotFoundError(msg) => complete(StatusCodes.NotFound, msg)
case InvalidGroupError(msg) => complete(StatusCodes.UnprocessableEntity, msg)
}
val recordSetRoute: Route = path("zones" / Segment / "recordsets") { zoneId =>
(post & monitor("Endpoint.addRecordSet")) {
authenticateAndExecuteWithEntity[ZoneCommandResult, RecordSet]((authPrincipal, recordSet) =>
recordSetService.addRecordSet(recordSet, authPrincipal)) { rc =>
complete(StatusCodes.Accepted, rc)
}
}
}
} ~
get {
monitor("Endpoint.getRecordSets") {
(get & monitor("Endpoint.getRecordSets")) {
parameters("startFrom".?, "maxItems".as[Int].?(DEFAULT_MAX_ITEMS), "recordNameFilter".?) {
(startFrom: Option[String], maxItems: Int, recordNameFilter: Option[String]) =>
handleRejections(invalidQueryHandler) {
validate(
0 < maxItems && maxItems <= DEFAULT_MAX_ITEMS,
s"maxItems was ${maxItems}, maxItems must be between 0 and $DEFAULT_MAX_ITEMS") {
execute(
recordSetService.listRecordSets(
zoneId,
startFrom,
Some(maxItems),
recordNameFilter,
authPrincipal)) { rsResponse =>
s"maxItems was $maxItems, maxItems must be between 0 and $DEFAULT_MAX_ITEMS") {
authenticateAndExecute(recordSetService
.listRecordSets(zoneId, startFrom, Some(maxItems), recordNameFilter, _)) {
rsResponse =>
complete(StatusCodes.OK, rsResponse)
}
}
}
}
}
}
} ~
path("zones" / Segment / "recordsets" / Segment) { (zoneId, rsId) =>
get {
monitor("Endpoint.getRecordSet") {
execute(recordSetService.getRecordSet(rsId, zoneId, authPrincipal)) { rs =>
(get & monitor("Endpoint.getRecordSet")) {
authenticateAndExecute(recordSetService.getRecordSet(rsId, zoneId, _)) { rs =>
complete(StatusCodes.OK, GetRecordSetResponse(rs))
}
}
} ~
delete {
monitor("Endpoint.deleteRecordSet") {
execute(recordSetService.deleteRecordSet(rsId, zoneId, authPrincipal)) { rc =>
(delete & monitor("Endpoint.deleteRecordSet")) {
authenticateAndExecute(recordSetService.deleteRecordSet(rsId, zoneId, _)) { rc =>
complete(StatusCodes.Accepted, rc)
}
}
} ~
put {
monitor("Endpoint.updateRecordSet") {
entity(as[RecordSet]) { rs =>
handleRejections(invalidContentHandler) {
validate(
check = rs.zoneId == zoneId,
errorMsg = "Cannot update RecordSet's zoneId attribute"
) {
execute(recordSetService.updateRecordSet(rs, authPrincipal)) { rc =>
(put & monitor("Endpoint.updateRecordSet")) {
authenticateAndExecuteWithEntity[ZoneCommandResult, RecordSet] {
(authPrincipal, recordSet) =>
recordSet match {
case badRs if badRs.zoneId != zoneId =>
Left(InvalidRequest("Cannot update RecordSet's zoneId attribute")).toResult
case goodRs =>
recordSetService.updateRecordSet(goodRs, authPrincipal)
}
} { rc =>
complete(StatusCodes.Accepted, rc)
}
}
}
}
}
}
} ~
path("zones" / Segment / "recordsets" / Segment / "changes" / Segment) {
(zoneId, _, changeId) =>
get {
monitor("Endpoint.getRecordSetChange") {
execute(recordSetService.getRecordSetChange(zoneId, changeId, authPrincipal)) {
(get & monitor("Endpoint.getRecordSetChange")) {
authenticateAndExecute(recordSetService.getRecordSetChange(zoneId, changeId, _)) {
change =>
complete(StatusCodes.OK, change)
}
}
}
} ~
path("zones" / Segment / "recordsetchanges") { zoneId =>
get {
monitor("Endpoint.listRecordSetChanges") {
(get & monitor("Endpoint.listRecordSetChanges")) {
parameters("startFrom".?, "maxItems".as[Int].?(DEFAULT_MAX_ITEMS)) {
(startFrom: Option[String], maxItems: Int) =>
handleRejections(invalidQueryHandler) {
@@ -132,9 +129,8 @@ trait RecordSetRoute extends Directives {
errorMsg = s"maxItems was $maxItems, maxItems must be between 0 exclusive " +
s"and $DEFAULT_MAX_ITEMS inclusive"
) {
execute(recordSetService
.listRecordSetChanges(zoneId, startFrom, maxItems, authPrincipal)) {
changes =>
authenticateAndExecute(recordSetService
.listRecordSetChanges(zoneId, startFrom, maxItems, _)) { changes =>
complete(StatusCodes.OK, changes)
}
}
@@ -142,8 +138,6 @@ trait RecordSetRoute extends Directives {
}
}
}
}
}
private val invalidQueryHandler = RejectionHandler
.newBuilder()
@@ -152,28 +146,4 @@ trait RecordSetRoute extends Directives {
complete(StatusCodes.BadRequest, msg)
}
.result()
private val invalidContentHandler = RejectionHandler
.newBuilder()
.handle {
case ValidationRejection(msg, _) =>
complete(StatusCodes.UnprocessableEntity, msg)
}
.result()
private def execute[A](f: => Result[A])(rt: A => Route): Route =
onSuccess(f.value.unsafeToFuture()) {
case Right(a) => rt(a)
case Left(ZoneNotFoundError(msg)) => complete(StatusCodes.NotFound, msg)
case Left(RecordSetAlreadyExists(msg)) => complete(StatusCodes.Conflict, msg)
case Left(ZoneInactiveError(msg)) => complete(StatusCodes.BadRequest, msg)
case Left(NotAuthorizedError(msg)) => complete(StatusCodes.Forbidden, msg)
case Left(ZoneUnavailableError(msg)) => complete(StatusCodes.Conflict, msg)
case Left(RecordSetNotFoundError(msg)) => complete(StatusCodes.NotFound, msg)
case Left(InvalidRequest(msg)) => complete(StatusCodes.UnprocessableEntity, msg)
case Left(PendingUpdateError(msg)) => complete(StatusCodes.Conflict, msg)
case Left(RecordSetChangeNotFoundError(msg)) => complete(StatusCodes.NotFound, msg)
case Left(InvalidGroupError(msg)) => complete(StatusCodes.UnprocessableEntity, msg)
case Left(e) => failWith(e)
}
}

View File

@@ -20,8 +20,10 @@ import akka.http.scaladsl.model.{HttpEntity, HttpResponse, StatusCodes}
import akka.http.scaladsl.server.RouteResult.{Complete, Rejected}
import akka.http.scaladsl.server._
import akka.http.scaladsl.server.directives.BasicDirectives
import akka.http.scaladsl.unmarshalling.FromRequestUnmarshaller
import cats.data.Validated.{Invalid, Valid}
import cats.data.ValidatedNel
import cats.data.{EitherT, ValidatedNel}
import cats.effect.IO
import org.json4s.JsonDSL._
import org.json4s.jackson.JsonMethods._
import vinyldns.core.domain.auth.AuthPrincipal
@@ -31,10 +33,12 @@ import scala.concurrent.duration._
import scala.util.Failure
import scala.util.control.NonFatal
trait VinylDNSDirectives extends Directives {
trait VinylDNSDirectives[E] extends Directives {
val vinylDNSAuthenticator: VinylDNSAuthenticator
def getRoutes: Route
def authenticate: Directive1[AuthPrincipal] = extractRequestContext.flatMap { ctx =>
extractStrictEntity(10.seconds).flatMap { strictEntity =>
onSuccess(
@@ -124,4 +128,49 @@ trait VinylDNSDirectives extends Directives {
case Invalid(errors) =>
reject(ValidationRejection(compact(render("errors" -> errors.toList.toSet))))
}
/**
* Helpers to handle route authentication flow for routing. Implementing classes/objects
* must provide handleErrors implementation.
*/
def handleErrors(e: E): PartialFunction[E, Route]
/**
* Authenticate user and execute service call without request entity
*
* Flow:
* - Authenticate user. Proceed if successful; otherwise return unauthorized error to user.
* - Invoke service call, f, and return the response to the user.
*/
def authenticateAndExecute[A](f: AuthPrincipal => EitherT[IO, E, A])(g: A => Route): Route =
authenticate { authPrincipal =>
onSuccess(f(authPrincipal).value.unsafeToFuture()) {
case Right(a) => g(a)
case Left(e) => handleErrors(e).applyOrElse(e, failWith)
}
}
/**
* Authenticate user and execute service call using request entity
*
* Flow:
* - Authenticate user. Proceed if successful; otherwise return unauthorized error to user.
* - Deserialize request entity into expected data structure. Proceed if successful; otherwise
* return error to user.
* - Invoke service call, f, and return the response to the user.
*/
def authenticateAndExecuteWithEntity[A, B](f: (AuthPrincipal, B) => EitherT[IO, E, A])(
g: A => Route)(implicit um: FromRequestUnmarshaller[B]): Route =
authenticate { authPrincipal =>
entity(as[B]) { deserializedEntity =>
onSuccess(f(authPrincipal, deserializedEntity).value.unsafeToFuture()) {
case Right(a) => g(a)
case Left(e) => handleErrors(e).applyOrElse(e, failWith)
}
}
}
private def failWith(error: E): StandardRoute = error match {
case error: Throwable => StandardRoute(_.fail(error))
}
}

View File

@@ -18,14 +18,15 @@ package vinyldns.api.route
import akka.event.Logging._
import akka.http.scaladsl.model._
import akka.http.scaladsl.server
import akka.http.scaladsl.server.Route
import akka.http.scaladsl.server.{MalformedRequestContentRejection, RejectionHandler, Route}
import akka.http.scaladsl.server.RouteResult.{Complete, Rejected}
import akka.http.scaladsl.server.directives.LogEntry
import cats.effect.IO
import fs2.concurrent.SignallingRef
import io.prometheus.client.CollectorRegistry
import vinyldns.api.domain.auth.AuthPrincipalProvider
import org.json4s.MappingException
import vinyldns.api.domain.batch.BatchChangeServiceAlgebra
import vinyldns.api.domain.membership.MembershipServiceAlgebra
import vinyldns.api.domain.record.RecordSetServiceAlgebra
@@ -35,6 +36,9 @@ import vinyldns.core.health.HealthService
import scala.util.matching.Regex
object VinylDNSService {
import akka.http.scaladsl.server.Directives._
val ZoneIdRegex: Regex = "(?i)(/?zones/)(?:[0-9a-f]-?)+(.*)".r
val ZoneAndRecordIdRegex: Regex =
"(?i)(/?zones/)(?:[0-9a-f]-?)+(/recordsets/)(?:[0-9a-f]-?)+(.*)".r
@@ -108,6 +112,24 @@ object VinylDNSService {
}
}
}
implicit def validationRejectionHandler: RejectionHandler =
RejectionHandler
.newBuilder()
.handle {
case MalformedRequestContentRejection(msg, MappingException(_, _)) =>
complete(
HttpResponse(
status = StatusCodes.BadRequest,
entity = HttpEntity(ContentTypes.`application/json`, msg)
))
}
.handleNotFound {
extractUnmatchedPath { p =>
complete((StatusCodes.NotFound, s"The requested path [$p] does not exist."))
}
}
.result()
}
// $COVERAGE-OFF$
@@ -120,31 +142,25 @@ class VinylDNSService(
val batchChangeService: BatchChangeServiceAlgebra,
val collectorRegistry: CollectorRegistry,
authPrincipalProvider: AuthPrincipalProvider)
extends VinylDNSDirectives
with PingRoute
with ZoneRoute
with RecordSetRoute
extends PingRoute
with HealthCheckRoute
with BlueGreenRoute
with MembershipRoute
with StatusRoute
with PrometheusRoute
with BatchChangeRoute
with VinylDNSJsonProtocol
with JsonValidationRejection {
with VinylDNSJsonProtocol {
import VinylDNSService.validationRejectionHandler
val aws4Authenticator = new Aws4Authenticator
val vinylDNSAuthenticator: VinylDNSAuthenticator =
new ProductionVinylDNSAuthenticator(aws4Authenticator, authPrincipalProvider)
// Authenticated routes must go first
def authenticatedRoutes: server.Route =
handleRejections(validationRejectionHandler)(authenticate { authPrincipal =>
batchChangeRoute(authPrincipal) ~
zoneRoute(authPrincipal) ~
recordSetRoute(authPrincipal) ~
membershipRoute(authPrincipal)
})
val zoneRoute: Route = new ZoneRoute(zoneService, vinylDNSAuthenticator).getRoutes
val recordSetRoute: Route = new RecordSetRoute(recordSetService, vinylDNSAuthenticator).getRoutes
val membershipRoute: Route =
new MembershipRoute(membershipService, vinylDNSAuthenticator).getRoutes
val batchChangeRoute: Route =
new BatchChangeRoute(batchChangeService, vinylDNSAuthenticator).getRoutes
val unloggedUris = Seq(
Uri.Path("/health"),
@@ -154,9 +170,16 @@ class VinylDNSService(
Uri.Path("/metrics/prometheus"))
val unloggedRoutes
: Route = healthCheckRoute ~ pingRoute ~ colorRoute ~ statusRoute ~ prometheusRoute
val allRoutes: Route = unloggedRoutes ~
batchChangeRoute ~
zoneRoute ~
recordSetRoute ~
membershipRoute
val vinyldnsRoutes: Route =
logRequestResult(VinylDNSService.buildLogEntry(unloggedUris))(
unloggedRoutes ~ authenticatedRoutes)
val routes = vinyldnsRoutes
logRequestResult(VinylDNSService.buildLogEntry(unloggedUris))(allRoutes)
val routes: Route =
handleRejections(validationRejectionHandler)(allRoutes)
}
// $COVERAGE-ON$

View File

@@ -17,12 +17,10 @@
package vinyldns.api.route
import akka.http.scaladsl.model.StatusCodes
import akka.http.scaladsl.server.{Directives, RejectionHandler, Route, ValidationRejection}
import akka.http.scaladsl.server._
import akka.util.Timeout
import vinyldns.api.Interfaces._
import vinyldns.api.crypto.Crypto
import vinyldns.api.domain.zone._
import vinyldns.core.domain.auth.AuthPrincipal
import vinyldns.core.domain.zone._
import scala.concurrent.duration._
@@ -30,10 +28,11 @@ import scala.concurrent.duration._
case class GetZoneResponse(zone: ZoneInfo)
case class ZoneRejected(zone: Zone, errors: List[String])
trait ZoneRoute extends Directives {
this: VinylDNSJsonProtocol with VinylDNSDirectives with JsonValidationRejection =>
class ZoneRoute(zoneService: ZoneServiceAlgebra, val vinylDNSAuthenticator: VinylDNSAuthenticator)
extends VinylDNSJsonProtocol
with VinylDNSDirectives[Throwable] {
val zoneService: ZoneServiceAlgebra
def getRoutes: Route = zoneRoute
final private val DEFAULT_MAX_ITEMS: Int = 100
final private val MAX_ITEMS_LIMIT: Int = 100
@@ -41,15 +40,31 @@ trait ZoneRoute extends Directives {
// Timeout must be long enough to allow the cluster to form
implicit val zoneCmdTimeout: Timeout = Timeout(10.seconds)
val zoneRoute = { authPrincipal: AuthPrincipal =>
(post & path("zones") & monitor("Endpoint.createZone")) {
entity(as[CreateZoneInput]) { createZoneInput =>
execute(zoneService.connectToZone(encrypt(createZoneInput), authPrincipal)) { chg =>
def handleErrors(errors: Throwable): PartialFunction[Throwable, Route] = {
case ZoneAlreadyExistsError(msg) => complete(StatusCodes.Conflict, msg)
case ConnectionFailed(_, msg) => complete(StatusCodes.BadRequest, msg)
case ZoneValidationFailed(zone, errorList, _) =>
complete(StatusCodes.BadRequest, ZoneRejected(zone, errorList))
case NotAuthorizedError(msg) => complete(StatusCodes.Forbidden, msg)
case InvalidGroupError(msg) => complete(StatusCodes.BadRequest, msg)
case ZoneNotFoundError(msg) => complete(StatusCodes.NotFound, msg)
case ZoneUnavailableError(msg) => complete(StatusCodes.Conflict, msg)
case InvalidSyncStateError(msg) => complete(StatusCodes.BadRequest, msg)
case PendingUpdateError(msg) => complete(StatusCodes.Conflict, msg)
case RecentSyncError(msg) => complete(StatusCodes.Forbidden, msg)
case ZoneInactiveError(msg) => complete(StatusCodes.BadRequest, msg)
case InvalidRequest(msg) => complete(StatusCodes.BadRequest, msg)
}
val zoneRoute: Route = path("zones") {
(post & monitor("Endpoint.createZone")) {
authenticateAndExecuteWithEntity[ZoneCommandResult, CreateZoneInput](
(authPrincipal, createZoneInput) =>
zoneService.connectToZone(encrypt(createZoneInput), authPrincipal)) { chg =>
complete(StatusCodes.Accepted, chg)
}
}
} ~
(get & path("zones") & monitor("Endpoint.listZones")) {
(get & monitor("Endpoint.listZones")) {
parameters(
"nameFilter".?,
"startFrom".as[String].?,
@@ -65,76 +80,82 @@ trait ZoneRoute extends Directives {
validate(
0 < maxItems && maxItems <= MAX_ITEMS_LIMIT,
s"maxItems was $maxItems, maxItems must be between 0 and $MAX_ITEMS_LIMIT") {
execute(zoneService
.listZones(authPrincipal, nameFilter, startFrom, maxItems, ignoreAccess)) {
result =>
authenticateAndExecute(zoneService
.listZones(_, nameFilter, startFrom, maxItems, ignoreAccess)) { result =>
complete(StatusCodes.OK, result)
}
}
}
}
}
}
} ~
(get & path("zones" / "backendids") & monitor("Endpoint.getBackendIds")) {
execute(zoneService.getBackendIds()) { ids =>
path("zones" / "backendids") {
(get & monitor("Endpoint.getBackendIds")) {
authenticateAndExecute(_ => zoneService.getBackendIds()) { ids =>
complete(StatusCodes.OK, ids)
}
}
} ~
(get & path("zones" / Segment) & monitor("Endpoint.getZone")) { id =>
execute(zoneService.getZone(id, authPrincipal)) { zone =>
path("zones" / "name" / Segment) { zoneName =>
authenticateAndExecute(zoneService.getZoneByName(zoneName, _)) { zone =>
complete(StatusCodes.OK, GetZoneResponse(zone))
}
} ~
(get & path("zones" / "name" / Segment) & monitor("Endpoint.getZoneByName")) { zoneName =>
execute(zoneService.getZoneByName(zoneName, authPrincipal)) { zone =>
path("zones" / Segment) { id =>
(get & monitor("Endpoint.getZone")) {
authenticateAndExecute(zoneService.getZone(id, _)) { zone =>
complete(StatusCodes.OK, GetZoneResponse(zone))
}
} ~
(delete & path("zones" / Segment) & monitor("Endpoint.deleteZone")) { id =>
execute(zoneService.deleteZone(id, authPrincipal)) { chg =>
(put & monitor("Endpoint.updateZone")) {
authenticateAndExecuteWithEntity[ZoneCommandResult, UpdateZoneInput](
(authPrincipal, updateZoneInput) =>
zoneService.updateZone(encrypt(updateZoneInput), authPrincipal)) { chg =>
complete(StatusCodes.Accepted, chg)
}
} ~
(put & path("zones" / Segment) & monitor("Endpoint.updateZone")) { _ =>
entity(as[UpdateZoneInput]) { updateZoneInput =>
execute(zoneService.updateZone(encrypt(updateZoneInput), authPrincipal)) { chg =>
(delete & monitor("Endpoint.deleteZone")) {
authenticateAndExecute(zoneService.deleteZone(id, _)) { chg =>
complete(StatusCodes.Accepted, chg)
}
}
} ~
(post & path("zones" / Segment / "sync") & monitor("Endpoint.syncZone")) { id =>
execute(zoneService.syncZone(id, authPrincipal)) { chg =>
path("zones" / Segment / "sync") { id =>
(post & monitor("Endpoint.syncZone")) {
authenticateAndExecute(zoneService.syncZone(id, _)) { chg =>
complete(StatusCodes.Accepted, chg)
}
}
} ~
(get & path("zones" / Segment / "changes") & monitor("Endpoint.listZoneChanges")) { id =>
path("zones" / Segment / "changes") { id =>
(get & monitor("Endpoint.listZoneChanges")) {
parameters("startFrom".?, "maxItems".as[Int].?(DEFAULT_MAX_ITEMS)) {
(startFrom: Option[String], maxItems: Int) =>
handleRejections(invalidQueryHandler) {
validate(
0 < maxItems && maxItems <= DEFAULT_MAX_ITEMS,
s"maxItems was $maxItems, maxItems must be between 0 exclusive and $DEFAULT_MAX_ITEMS inclusive") {
execute(zoneService.listZoneChanges(id, authPrincipal, startFrom, maxItems)) {
authenticateAndExecute(zoneService.listZoneChanges(id, _, startFrom, maxItems)) {
changes =>
complete(StatusCodes.OK, changes)
}
}
}
}
} ~
(put & path("zones" / Segment / "acl" / "rules") & monitor("Endpoint.addZoneACLRule")) { id =>
entity(as[ACLRuleInfo]) { rule =>
execute(zoneService.addACLRule(id, rule, authPrincipal)) { chg =>
complete(StatusCodes.Accepted, chg)
}
}
} ~
(delete & path("zones" / Segment / "acl" / "rules") & monitor("Endpoint.deleteZoneACLRule")) {
id =>
entity(as[ACLRuleInfo]) { rule =>
execute(zoneService.deleteACLRule(id, rule, authPrincipal)) { chg =>
path("zones" / Segment / "acl" / "rules") { id =>
(put & monitor("Endpoint.addZoneACLRule")) {
authenticateAndExecuteWithEntity[ZoneCommandResult, ACLRuleInfo]((authPrincipal, rule) =>
zoneService.addACLRule(id, rule, authPrincipal)) { chg =>
complete(StatusCodes.Accepted, chg)
}
} ~
(delete & monitor("Endpoint.deleteZoneACLRule")) {
authenticateAndExecuteWithEntity[ZoneCommandResult, ACLRuleInfo]((authPrincipal, rule) =>
zoneService.deleteACLRule(id, rule, authPrincipal)) { chg =>
complete(StatusCodes.Accepted, chg)
}
}
}
@@ -164,23 +185,4 @@ trait ZoneRoute extends Directives {
complete(StatusCodes.BadRequest, msg)
}
.result()
private def execute[A](f: => Result[A])(rt: A => Route): Route =
onSuccess(f.value.unsafeToFuture()) {
case Right(a) => rt(a)
case Left(ZoneAlreadyExistsError(msg)) => complete(StatusCodes.Conflict, msg)
case Left(ConnectionFailed(_, msg)) => complete(StatusCodes.BadRequest, msg)
case Left(ZoneValidationFailed(zone, errors, _)) =>
complete(StatusCodes.BadRequest, ZoneRejected(zone, errors))
case Left(NotAuthorizedError(msg)) => complete(StatusCodes.Forbidden, msg)
case Left(InvalidGroupError(msg)) => complete(StatusCodes.BadRequest, msg)
case Left(ZoneNotFoundError(msg)) => complete(StatusCodes.NotFound, msg)
case Left(ZoneUnavailableError(msg)) => complete(StatusCodes.Conflict, msg)
case Left(InvalidSyncStateError(msg)) => complete(StatusCodes.BadRequest, msg)
case Left(PendingUpdateError(msg)) => complete(StatusCodes.Conflict, msg)
case Left(RecentSyncError(msg)) => complete(StatusCodes.Forbidden, msg)
case Left(ZoneInactiveError(msg)) => complete(StatusCodes.BadRequest, msg)
case Left(InvalidRequest(msg)) => complete(StatusCodes.BadRequest, msg)
case Left(e) => failWith(e)
}
}

View File

@@ -21,33 +21,45 @@ import akka.http.scaladsl.model.{ContentTypes, HttpEntity}
import akka.http.scaladsl.server.Route
import akka.http.scaladsl.testkit.ScalatestRouteTest
import cats.data.EitherT
import cats.effect._
import cats.implicits._
import org.joda.time.DateTime
import org.json4s.JsonDSL._
import org.json4s._
import org.json4s.jackson.JsonMethods._
import org.scalatest.{Matchers, WordSpec}
import org.scalatest.mockito.MockitoSugar
import org.scalatest.{BeforeAndAfterEach, Matchers, WordSpec}
import vinyldns.api.domain.batch._
import vinyldns.core.TestMembershipData._
import vinyldns.core.domain.record.RecordType._
import vinyldns.core.domain.record._
import cats.effect._
import vinyldns.core.domain.BatchChangeIsEmpty
import vinyldns.core.domain.auth.AuthPrincipal
import vinyldns.core.domain.batch.BatchChangeApprovalStatus.BatchChangeApprovalStatus
import vinyldns.core.domain.batch._
import vinyldns.core.domain.record.RecordType._
import vinyldns.core.domain.record._
class BatchChangeRoutingSpec
class BatchChangeRoutingSpec()
extends WordSpec
with ScalatestRouteTest
with BatchChangeRoute
with JsonValidationRejection
with VinylDNSDirectives
with MockitoSugar
with VinylDNSJsonProtocol
with Matchers {
with VinylDNSRouteTestHelper
with Matchers
with BeforeAndAfterEach {
val batchChangeService: BatchChangeServiceAlgebra = TestBatchChangeService
val vinylDNSAuthenticator: VinylDNSAuthenticator = new TestVinylDNSAuthenticator(okAuth)
val okAuthRoute: BatchChangeRoute =
new BatchChangeRoute(TestBatchChangeService, new TestVinylDNSAuthenticator(okAuth))
val notAuthRoute: BatchChangeRoute =
new BatchChangeRoute(TestBatchChangeService, new TestVinylDNSAuthenticator(notAuth))
val supportUserRoute: BatchChangeRoute =
new BatchChangeRoute(TestBatchChangeService, new TestVinylDNSAuthenticator(supportUserAuth))
val superUserRoute: BatchChangeRoute =
new BatchChangeRoute(TestBatchChangeService, new TestVinylDNSAuthenticator(superUserAuth))
var batchChangeRoute: Route = _
override def beforeEach(): Unit = batchChangeRoute = okAuthRoute.getRoutes
import vinyldns.core.domain.batch.SingleChangeStatus._
@@ -303,7 +315,7 @@ class BatchChangeRoutingSpec
)
)
case (_) => EitherT.rightT(BatchChangeSummaryList(List()))
case _ => EitherT.rightT(BatchChangeSummaryList(List()))
} else if (auth.userId == superUserAuth.userId)
(auth, startFrom, maxItems, ignoreAccess, approvalStatus) match {
case (_, None, 100, true, None) =>
@@ -351,7 +363,7 @@ class BatchChangeRoutingSpec
)
)
case (_) => EitherT.rightT(BatchChangeSummaryList(List()))
case _ => EitherT.rightT(BatchChangeSummaryList(List()))
} else
EitherT.rightT(BatchChangeSummaryList(List()))
@@ -364,7 +376,7 @@ class BatchChangeRoutingSpec
case ("pendingBatchId", true) => EitherT(IO.pure(genericValidResponse.asRight))
case ("pendingBatchId", false) =>
EitherT(IO.pure(UserNotAuthorizedError("notAuthedID").asLeft))
case (_) => EitherT(IO.pure(BatchChangeNotPendingApproval("batchId").asLeft))
case _ => EitherT(IO.pure(BatchChangeNotPendingApproval("batchId").asLeft))
}
def approveBatchChange(
@@ -389,7 +401,7 @@ class BatchChangeRoutingSpec
Post("/zones/batchrecordchanges").withEntity(
HttpEntity(ContentTypes.`application/json`, validRequestWithComments)) ~>
batchChangeRoute(okAuth) ~> check {
batchChangeRoute ~> check {
status shouldBe Accepted
@@ -403,7 +415,7 @@ class BatchChangeRoutingSpec
Post("/zones/batchrecordchanges").withEntity(
HttpEntity(ContentTypes.`application/json`, validRequestWithoutComments)) ~>
batchChangeRoute(okAuth) ~> check {
batchChangeRoute ~> check {
status shouldBe Accepted
@@ -420,7 +432,7 @@ class BatchChangeRoutingSpec
Post("/zones/batchrecordchanges").withEntity(
HttpEntity(ContentTypes.`application/json`, validRequestWithOwnerGroupId)) ~>
batchChangeRoute(sharedAuth) ~> check {
batchChangeRoute ~> check {
status shouldBe Accepted
val change = responseAs[JValue]
@@ -430,10 +442,9 @@ class BatchChangeRoutingSpec
"return a 202 Accepted for valid add and delete request with allowManualReview parameter" in {
val validRequestWithoutComments: String = compact(render(changeList))
Post("/zones/batchrecordchanges?allowManualReview=false").withEntity(
HttpEntity(ContentTypes.`application/json`, validRequestWithoutComments)) ~>
batchChangeRoute(okAuth) ~> check {
batchChangeRoute ~> check {
status shouldBe Accepted
@@ -453,7 +464,7 @@ class BatchChangeRoutingSpec
Post("/zones/batchrecordchanges").withEntity(
HttpEntity(ContentTypes.`application/json`, validRequestWithScheduledTime)) ~>
batchChangeRoute(okAuth) ~> check {
batchChangeRoute ~> check {
status shouldBe Accepted
@@ -470,7 +481,7 @@ class BatchChangeRoutingSpec
Post("/zones/batchrecordchanges").withEntity(
HttpEntity(ContentTypes.`application/json`, emptyBatchRequest)) ~>
Route.seal(batchChangeRoute(okAuth)) ~> check {
Route.seal(batchChangeRoute) ~> check {
status shouldBe BadRequest
}
@@ -486,7 +497,7 @@ class BatchChangeRoutingSpec
Post("/zones/batchrecordchanges").withEntity(
HttpEntity(ContentTypes.`application/json`, invalidRequestChangeType)) ~>
Route.seal(batchChangeRoute(okAuth)) ~> check {
Route.seal(batchChangeRoute) ~> check {
status shouldBe BadRequest
}
@@ -497,7 +508,7 @@ class BatchChangeRoutingSpec
Post("/zones/batchrecordchanges").withEntity(
HttpEntity(ContentTypes.`application/json`, runtimeError)) ~>
Route.seal(batchChangeRoute(okAuth)) ~> check {
Route.seal(batchChangeRoute) ~> check {
status shouldBe InternalServerError
}
@@ -506,7 +517,7 @@ class BatchChangeRoutingSpec
"GET Batch Change Info" should {
"return the batch change info given a valid batch change id" in {
Get(s"/zones/batchrecordchanges/${genericValidResponse.id}") ~> batchChangeRoute(okAuth) ~> check {
Get(s"/zones/batchrecordchanges/${genericValidResponse.id}") ~> batchChangeRoute ~> check {
status shouldBe OK
@@ -517,7 +528,7 @@ class BatchChangeRoutingSpec
}
"maintain backwards compatability for zoneName/recordName/zoneId" in {
Get(s"/zones/batchrecordchanges/${backwardsCompatable.id}") ~> batchChangeRoute(okAuth) ~> check {
Get(s"/zones/batchrecordchanges/${backwardsCompatable.id}") ~> batchChangeRoute ~> check {
status shouldBe OK
@@ -531,14 +542,14 @@ class BatchChangeRoutingSpec
}
"return a NotFound error given a nonexistent batch change id" in {
Get("/zones/batchrecordchanges/nonexistentID") ~> batchChangeRoute(okAuth) ~> check {
Get("/zones/batchrecordchanges/nonexistentID") ~> batchChangeRoute ~> check {
status shouldBe NotFound
}
}
"return a Forbidden error if user did not create the batch change" in {
Get("/zones/batchrecordchanges/notAuthedID") ~> batchChangeRoute(notAuth) ~> check {
Get("/zones/batchrecordchanges/notAuthedID") ~> batchChangeRoute ~> check {
status shouldBe Forbidden
}
@@ -547,7 +558,7 @@ class BatchChangeRoutingSpec
"GET batchChangesSummaries" should {
"return the list of batch change summaries for the user that called it" in {
Get("/zones/batchrecordchanges") ~> batchChangeRoute(okAuth) ~> check {
Get("/zones/batchrecordchanges") ~> batchChangeRoute ~> check {
status shouldBe OK
val resp = responseAs[BatchChangeSummaryList]
@@ -560,7 +571,7 @@ class BatchChangeRoutingSpec
}
"return the first batch change summary for the user that called it" in {
Get("/zones/batchrecordchanges?maxItems=1") ~> batchChangeRoute(okAuth) ~> check {
Get("/zones/batchrecordchanges?maxItems=1") ~> batchChangeRoute ~> check {
status shouldBe OK
val resp = responseAs[BatchChangeSummaryList]
@@ -573,7 +584,7 @@ class BatchChangeRoutingSpec
}
"return an offset of the batch change summaries for the user that called it" in {
Get("/zones/batchrecordchanges?startFrom=1") ~> batchChangeRoute(okAuth) ~> check {
Get("/zones/batchrecordchanges?startFrom=1") ~> batchChangeRoute ~> check {
status shouldBe OK
val resp = responseAs[BatchChangeSummaryList]
@@ -586,7 +597,7 @@ class BatchChangeRoutingSpec
}
"return only the second batch change summary for the user that called it" in {
Get("/zones/batchrecordchanges?startFrom=1&maxItems=1") ~> batchChangeRoute(okAuth) ~> check {
Get("/zones/batchrecordchanges?startFrom=1&maxItems=1") ~> batchChangeRoute ~> check {
status shouldBe OK
val resp = responseAs[BatchChangeSummaryList]
@@ -600,7 +611,7 @@ class BatchChangeRoutingSpec
"return user's Pending batch changes if approval status is `PendingApproval`" in {
Get("/zones/batchrecordchanges?approvalStatus=pendingapproval") ~>
batchChangeRoute(okAuth) ~> check {
batchChangeRoute ~> check {
status shouldBe OK
val resp = responseAs[BatchChangeSummaryList]
@@ -614,7 +625,7 @@ class BatchChangeRoutingSpec
}
"return an error if maxItems is out of range" in {
Get("/zones/batchrecordchanges?startFrom=1&maxItems=101") ~> batchChangeRoute(okAuth) ~> check {
Get("/zones/batchrecordchanges?startFrom=1&maxItems=101") ~> batchChangeRoute ~> check {
status shouldBe BadRequest
responseEntity.toString should include(
@@ -623,7 +634,8 @@ class BatchChangeRoutingSpec
}
"return empty list of batch change summaries for the user that called it" in {
Get("/zones/batchrecordchanges") ~> batchChangeRoute(notAuth) ~> check {
batchChangeRoute = notAuthRoute.getRoutes
Get("/zones/batchrecordchanges") ~> batchChangeRoute ~> check {
status shouldBe OK
val resp = responseAs[JValue]
@@ -633,7 +645,8 @@ class BatchChangeRoutingSpec
}
"return all batch changes if ignoreAccess is true and requester is a super user" in {
Get("/zones/batchrecordchanges?ignoreAccess=true") ~> batchChangeRoute(superUserAuth) ~> check {
batchChangeRoute = superUserRoute.getRoutes
Get("/zones/batchrecordchanges?ignoreAccess=true") ~> batchChangeRoute ~> check {
status shouldBe OK
val resp = responseAs[BatchChangeSummaryList]
@@ -647,8 +660,9 @@ class BatchChangeRoutingSpec
"return all Pending batch changes if ignoreAccess is true, approval status is `PendingApproval`," +
" and requester is a super user" in {
batchChangeRoute = superUserRoute.getRoutes
Get("/zones/batchrecordchanges?ignoreAccess=true&approvalStatus=PendingApproval") ~>
batchChangeRoute(superUserAuth) ~> check {
batchChangeRoute ~> check {
status shouldBe OK
val resp = responseAs[BatchChangeSummaryList]
@@ -665,18 +679,20 @@ class BatchChangeRoutingSpec
"POST reject batch change" should {
"return OK if review comment is provided, batch change is PendingApproval, and reviewer is authorized" in {
batchChangeRoute = supportUserRoute.getRoutes
Post("/zones/batchrecordchanges/pendingBatchId/reject").withEntity(HttpEntity(
ContentTypes.`application/json`,
compact(render("comments" -> "some comments")))) ~>
batchChangeRoute(supportUserAuth) ~> check {
batchChangeRoute ~> check {
status shouldBe OK
}
}
"return OK if comments are not provided, batch change is PendingApproval, and reviewer is authorized" in {
batchChangeRoute = supportUserRoute.getRoutes
Post("/zones/batchrecordchanges/pendingBatchId/reject").withEntity(
HttpEntity(ContentTypes.`application/json`, compact(render("")))) ~>
batchChangeRoute(supportUserAuth) ~> check {
batchChangeRoute ~> check {
status shouldBe OK
}
}
@@ -684,7 +700,7 @@ class BatchChangeRoutingSpec
"return Forbidden if user is not a super or support admin" in {
Post("/zones/batchrecordchanges/pendingBatchId/reject").withEntity(
HttpEntity(ContentTypes.`application/json`, compact(render("")))) ~>
batchChangeRoute(okAuth) ~> check {
batchChangeRoute ~> check {
status shouldBe Forbidden
}
}
@@ -692,7 +708,7 @@ class BatchChangeRoutingSpec
"return BadRequest if comments exceed 1024 characters" in {
Post("/zones/batchrecordchanges/pendingBatchId/reject").withEntity(HttpEntity(
ContentTypes.`application/json`,
compact(render("reviewComment" -> "a" * 1025)))) ~> Route.seal(batchChangeRoute(okAuth)) ~> check {
compact(render("reviewComment" -> "a" * 1025)))) ~> Route.seal(batchChangeRoute) ~> check {
status shouldBe BadRequest
responseEntity.toString should include("Comment length must not exceed 1024 characters.")
@@ -700,15 +716,17 @@ class BatchChangeRoutingSpec
}
"return OK if no request entity is provided" in {
Post("/zones/batchrecordchanges/pendingBatchId/reject") ~> batchChangeRoute(supportUserAuth) ~> check {
batchChangeRoute = supportUserRoute.getRoutes
Post("/zones/batchrecordchanges/pendingBatchId/reject") ~> batchChangeRoute ~> check {
status shouldBe OK
}
}
"return BadRequest if batch change is not pending approval" in {
batchChangeRoute = supportUserRoute.getRoutes
Post("/zones/batchrecordchanges/batchId/reject").withEntity(
HttpEntity(ContentTypes.`application/json`, compact(render("")))) ~>
batchChangeRoute(supportUserAuth) ~> check {
batchChangeRoute ~> check {
status shouldBe BadRequest
}
}
@@ -716,18 +734,20 @@ class BatchChangeRoutingSpec
"POST approve batch change" should {
"return OK if review comment is provided, batch change is PendingApproval, and reviewer is authorized" in {
batchChangeRoute = supportUserRoute.getRoutes
Post("/zones/batchrecordchanges/pendingBatchId/approve").withEntity(HttpEntity(
ContentTypes.`application/json`,
compact(render("comments" -> "some comments")))) ~>
batchChangeRoute(supportUserAuth) ~> check {
batchChangeRoute ~> check {
status shouldBe OK
}
}
"return OK if comments are not provided, batch change is PendingApproval, and reviewer is authorized" in {
batchChangeRoute = supportUserRoute.getRoutes
Post("/zones/batchrecordchanges/pendingBatchId/approve").withEntity(
HttpEntity(ContentTypes.`application/json`, compact(render("")))) ~>
batchChangeRoute(supportUserAuth) ~> check {
batchChangeRoute ~> check {
status shouldBe OK
}
}
@@ -735,40 +755,44 @@ class BatchChangeRoutingSpec
"return Forbidden if user is not a super or support admin" in {
Post("/zones/batchrecordchanges/pendingBatchId/approve").withEntity(
HttpEntity(ContentTypes.`application/json`, compact(render("")))) ~>
batchChangeRoute(okAuth) ~> check {
batchChangeRoute ~> check {
status shouldBe Forbidden
}
}
"return BadRequest if comments exceed 1024 characters" in {
batchChangeRoute = supportUserRoute.getRoutes
Post("/zones/batchrecordchanges/pendingBatchId/approve").withEntity(HttpEntity(
ContentTypes.`application/json`,
compact(render("reviewComment" -> "a" * 1025)))) ~> Route.seal(batchChangeRoute(okAuth)) ~> check {
compact(render("reviewComment" -> "a" * 1025)))) ~> Route.seal(batchChangeRoute) ~> check {
status shouldBe BadRequest
responseEntity.toString should include("Comment length must not exceed 1024 characters.")
}
}
"return OK if no request entity is provided" in {
Post("/zones/batchrecordchanges/pendingBatchId/approve") ~> batchChangeRoute(supportUserAuth) ~> check {
"return OK no request entity is provided" in {
batchChangeRoute = supportUserRoute.getRoutes
Post("/zones/batchrecordchanges/pendingBatchId/approve") ~> batchChangeRoute ~> check {
status shouldBe OK
}
}
"return BadRequest if batch change is not pending approval" in {
batchChangeRoute = supportUserRoute.getRoutes
Post("/zones/batchrecordchanges/batchId/approve").withEntity(
HttpEntity(ContentTypes.`application/json`, compact(render("")))) ~>
batchChangeRoute(supportUserAuth) ~> check {
batchChangeRoute ~> check {
status shouldBe BadRequest
}
}
"return NotFound if the requesting user cant be found" in {
batchChangeRoute = supportUserRoute.getRoutes
Post("/zones/batchrecordchanges/notFoundUser/approve").withEntity(HttpEntity(
ContentTypes.`application/json`,
compact(render("comments" -> "some comments")))) ~>
batchChangeRoute(supportUserAuth) ~> check {
batchChangeRoute ~> check {
status shouldBe NotFound
}
}

View File

@@ -30,28 +30,33 @@ import org.scalatest.mockito.MockitoSugar
import org.scalatest.{BeforeAndAfterEach, Matchers, WordSpec}
import vinyldns.api.Interfaces._
import vinyldns.api.domain.membership._
import vinyldns.core.domain.auth.AuthPrincipal
import vinyldns.core.domain.membership.{Group, LockStatus}
import vinyldns.api.domain.zone.NotAuthorizedError
import vinyldns.api.route.MembershipJsonProtocol.{CreateGroupInput, UpdateGroupInput}
import vinyldns.core.TestMembershipData._
import vinyldns.core.domain.auth.AuthPrincipal
import vinyldns.core.domain.membership.LockStatus.LockStatus
import vinyldns.core.domain.membership.{Group, LockStatus}
class MembershipRoutingSpec
extends WordSpec
with ScalatestRouteTest
with MembershipRoute
with VinylDNSJsonProtocol
with JsonValidationRejection
with VinylDNSDirectives
with VinylDNSRouteTestHelper
with Matchers
with MockitoSugar
with BeforeAndAfterEach {
val membershipService: MembershipService = mock[MembershipService]
val vinylDNSAuthenticator: VinylDNSAuthenticator = new TestVinylDNSAuthenticator(okAuth)
val okAuthRoute: Route =
new MembershipRoute(membershipService, new TestVinylDNSAuthenticator(okAuth)).getRoutes
val superUserRoute: Route =
new MembershipRoute(membershipService, new TestVinylDNSAuthenticator(superUserAuth)).getRoutes
var membershipRoute: Route = _
override protected def beforeEach(): Unit = reset(membershipService)
override protected def beforeEach(): Unit = {
reset(membershipService)
membershipRoute = okAuthRoute
}
private def js[A](info: A): String = compact(render(Extraction.decompose(info)))
@@ -81,7 +86,7 @@ class MembershipRoutingSpec
doReturn(result(okGroup)).when(membershipService).createGroup(any[Group], any[AuthPrincipal])
Post("/groups").withEntity(HttpEntity(ContentTypes.`application/json`, js(goodRequest))) ~> Route
.seal(membershipRoute(okAuth)) ~> check {
.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.OK
val result = responseAs[GroupInfo]
@@ -106,7 +111,7 @@ class MembershipRoutingSpec
.createGroup(any[Group], any[AuthPrincipal])
Post("/groups").withEntity(HttpEntity(ContentTypes.`application/json`, js(duplicateRequest))) ~> Route
.seal(membershipRoute(okAuth)) ~> check {
.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.Conflict
}
}
@@ -118,7 +123,7 @@ class MembershipRoutingSpec
.createGroup(any[Group], any[AuthPrincipal])
Post("/groups").withEntity(HttpEntity(ContentTypes.`application/json`, js(badRequest))) ~> Route
.seal(membershipRoute(okAuth)) ~> check {
.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.BadRequest
}
}
@@ -130,7 +135,7 @@ class MembershipRoutingSpec
val malformed = compact(render(missingFields))
Post("/groups").withEntity(HttpEntity(ContentTypes.`application/json`, malformed)) ~> Route
.seal(membershipRoute(okAuth)) ~> check {
.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.BadRequest
val result = responseAs[JValue]
val errs = (result \ "errors").extractOpt[List[String]]
@@ -156,7 +161,7 @@ class MembershipRoutingSpec
.createGroup(any[Group], any[AuthPrincipal])
Post("/groups").withEntity(HttpEntity(ContentTypes.`application/json`, js(notFoundRequest))) ~> Route
.seal(membershipRoute(okAuth)) ~> check {
.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.NotFound
}
}
@@ -173,7 +178,7 @@ class MembershipRoutingSpec
.createGroup(any[Group], any[AuthPrincipal])
Post("/groups").withEntity(HttpEntity(ContentTypes.`application/json`, js(badRequest))) ~> Route
.seal(membershipRoute(okAuth)) ~> check {
.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.InternalServerError
}
}
@@ -186,7 +191,7 @@ class MembershipRoutingSpec
result(ListMyGroupsResponse(Seq(okGroupInfo, twoUserGroupInfo), None, None, None, 100)))
.when(membershipService)
.listMyGroups(None, None, 100, okAuth)
Get("/groups") ~> Route.seal(membershipRoute(okAuth)) ~> check {
Get("/groups") ~> Route.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.OK
val result = responseAs[ListMyGroupsResponse]
@@ -212,7 +217,7 @@ class MembershipRoutingSpec
maxItems = 100,
okAuth)
Get("/groups?startFrom=anyString&maxItems=100&groupNameFilter=ok") ~> Route.seal(
membershipRoute(okAuth)) ~> check {
membershipRoute) ~> check {
status shouldBe StatusCodes.OK
val result = responseAs[ListMyGroupsResponse]
@@ -227,12 +232,12 @@ class MembershipRoutingSpec
}
}
"return with a 400 response when the page size is 0" in {
Get("/groups?maxItems=0") ~> Route.seal(membershipRoute(okAuth)) ~> check {
Get("/groups?maxItems=0") ~> Route.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.BadRequest
}
}
"return a 400 response when maxItems is more than 1000" in {
Get("/groups?maxItems=1001") ~> Route.seal(membershipRoute(okAuth)) ~> check {
Get("/groups?maxItems=1001") ~> Route.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.BadRequest
}
}
@@ -241,7 +246,7 @@ class MembershipRoutingSpec
.when(membershipService)
.listMyGroups(None, None, 100, okAuth)
Get("/groups") ~> Route.seal(membershipRoute(okAuth)) ~> check {
Get("/groups") ~> Route.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.InternalServerError
}
}
@@ -251,7 +256,7 @@ class MembershipRoutingSpec
"return a 200 response with the group when found" in {
val grpBaseTime = okGroup.copy(created = baseTime)
doReturn(result(grpBaseTime)).when(membershipService).getGroup("ok", okAuth)
Get("/groups/ok") ~> Route.seal(membershipRoute(okAuth)) ~> check {
Get("/groups/ok") ~> Route.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.OK
val result = responseAs[GroupInfo]
@@ -263,7 +268,7 @@ class MembershipRoutingSpec
doReturn(result(GroupNotFoundError("fail")))
.when(membershipService)
.getGroup("notFound", okAuth)
Get("/groups/notFound") ~> Route.seal(membershipRoute(okAuth)) ~> check {
Get("/groups/notFound") ~> Route.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.NotFound
}
}
@@ -273,7 +278,7 @@ class MembershipRoutingSpec
"return a 200 response with the deleted group when it exists" in {
val grpBaseTime = deletedGroup.copy(created = baseTime)
doReturn(result(grpBaseTime)).when(membershipService).deleteGroup("ok", okAuth)
Delete("/groups/ok") ~> Route.seal(membershipRoute(okAuth)) ~> check {
Delete("/groups/ok") ~> Route.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.OK
val result = responseAs[GroupInfo]
@@ -285,7 +290,7 @@ class MembershipRoutingSpec
doReturn(result(GroupNotFoundError("fail")))
.when(membershipService)
.deleteGroup("notFound", okAuth)
Delete("/groups/notFound") ~> Route.seal(membershipRoute(okAuth)) ~> check {
Delete("/groups/notFound") ~> Route.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.NotFound
}
}
@@ -294,7 +299,7 @@ class MembershipRoutingSpec
doReturn(result(InvalidGroupRequestError("fail")))
.when(membershipService)
.deleteGroup("adminGroup", okAuth)
Delete("/groups/adminGroup") ~> Route.seal(membershipRoute(okAuth)) ~> check {
Delete("/groups/adminGroup") ~> Route.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.BadRequest
}
}
@@ -304,7 +309,7 @@ class MembershipRoutingSpec
.when(membershipService)
.deleteGroup("bad", okAuth)
Delete("/groups/bad") ~> Route.seal(membershipRoute(okAuth)) ~> check {
Delete("/groups/bad") ~> Route.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.InternalServerError
}
}
@@ -313,7 +318,7 @@ class MembershipRoutingSpec
doReturn(result(NotAuthorizedError("forbidden")))
.when(membershipService)
.deleteGroup("forbidden", okAuth)
Delete("/groups/forbidden") ~> Route.seal(membershipRoute(okAuth)) ~> check {
Delete("/groups/forbidden") ~> Route.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.Forbidden
}
}
@@ -341,7 +346,7 @@ class MembershipRoutingSpec
any[AuthPrincipal])
Put("/groups/good").withEntity(HttpEntity(ContentTypes.`application/json`, js(goodRequest))) ~> Route
.seal(membershipRoute(okAuth)) ~> check {
.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.OK
val result = responseAs[GroupInfo]
@@ -375,7 +380,7 @@ class MembershipRoutingSpec
Put("/groups/duplicate").withEntity(
HttpEntity(ContentTypes.`application/json`, js(duplicateRequest))) ~> Route.seal(
membershipRoute(okAuth)) ~> check {
membershipRoute) ~> check {
status shouldBe StatusCodes.Conflict
}
}
@@ -400,7 +405,7 @@ class MembershipRoutingSpec
any[AuthPrincipal])
Put("/groups/notFound").withEntity(
HttpEntity(ContentTypes.`application/json`, js(notFoundRequest))) ~> Route.seal(
membershipRoute(okAuth)) ~> check {
membershipRoute) ~> check {
status shouldBe StatusCodes.NotFound
}
}
@@ -426,7 +431,7 @@ class MembershipRoutingSpec
any[AuthPrincipal])
Put("/groups/forbidden").withEntity(
HttpEntity(ContentTypes.`application/json`, js(forbiddenRequest))) ~> Route.seal(
membershipRoute(okAuth)) ~> check {
membershipRoute) ~> check {
status shouldBe StatusCodes.Forbidden
}
}
@@ -446,7 +451,7 @@ class MembershipRoutingSpec
any[AuthPrincipal])
Put("/groups/bad").withEntity(HttpEntity(ContentTypes.`application/json`, js(badRequest))) ~> Route
.seal(Route.seal(membershipRoute(okAuth))) ~> check {
.seal(Route.seal(membershipRoute)) ~> check {
status shouldBe StatusCodes.InternalServerError
}
}
@@ -466,7 +471,7 @@ class MembershipRoutingSpec
any[AuthPrincipal])
Put("/groups/bad").withEntity(HttpEntity(ContentTypes.`application/json`, js(badRequest))) ~> Route
.seal(membershipRoute(okAuth)) ~> check {
.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.BadRequest
}
}
@@ -478,7 +483,7 @@ class MembershipRoutingSpec
val malformed = compact(render(missingFields))
Put("/groups/malformed").withEntity(HttpEntity(ContentTypes.`application/json`, malformed)) ~> Route
.seal(membershipRoute(okAuth)) ~> check {
.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.BadRequest
val result = responseAs[JValue]
val errs = (result \ "errors").extractOpt[List[String]]
@@ -500,7 +505,7 @@ class MembershipRoutingSpec
.when(membershipService)
.listMembers(anyString, any[Option[String]], anyInt, any[AuthPrincipal])
Get(s"/groups/good/members") ~> Route.seal(membershipRoute(okAuth)) ~> check {
Get(s"/groups/good/members") ~> Route.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.OK
val result = responseAs[ListMembersResponse]
@@ -514,7 +519,7 @@ class MembershipRoutingSpec
.when(membershipService)
.listMembers(anyString, any[Option[String]], anyInt, any[AuthPrincipal])
Get(s"/groups/notFound/members") ~> Route.seal(membershipRoute(okAuth)) ~> check {
Get(s"/groups/notFound/members") ~> Route.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.NotFound
}
}
@@ -523,7 +528,7 @@ class MembershipRoutingSpec
.when(membershipService)
.listMembers(anyString, any[Option[String]], anyInt, any[AuthPrincipal])
Get(s"/groups/bad/members") ~> Route.seal(membershipRoute(okAuth)) ~> check {
Get(s"/groups/bad/members") ~> Route.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.InternalServerError
}
}
@@ -538,8 +543,7 @@ class MembershipRoutingSpec
.when(membershipService)
.listMembers("goodQuery", Some("dummy"), 50, okAuth)
Get(s"/groups/goodQuery/members?startFrom=dummy&maxItems=50") ~> Route.seal(
membershipRoute(okAuth)) ~> check {
Get(s"/groups/goodQuery/members?startFrom=dummy&maxItems=50") ~> Route.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.OK
val result = responseAs[ListMembersResponse]
@@ -554,7 +558,7 @@ class MembershipRoutingSpec
.when(membershipService)
.listMembers(anyString, any[Option[String]], anyInt, any[AuthPrincipal])
Get(s"/groups/pageSize/members") ~> Route.seal(membershipRoute(okAuth)) ~> check {
Get(s"/groups/pageSize/members") ~> Route.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.OK
val result = responseAs[ListMembersResponse]
@@ -572,12 +576,12 @@ class MembershipRoutingSpec
}
}
"return with a 400 response when the page size is 0" in {
Get(s"/groups/badPageSize/members?maxItems=0") ~> Route.seal(membershipRoute(okAuth)) ~> check {
Get(s"/groups/badPageSize/members?maxItems=0") ~> Route.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.BadRequest
}
}
"return a 400 response when maxItems is more than 1000" in {
Get(s"/groups/pageSize/members?maxItems=1001") ~> Route.seal(membershipRoute(okAuth)) ~> check {
Get(s"/groups/pageSize/members?maxItems=1001") ~> Route.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.BadRequest
}
}
@@ -588,18 +592,18 @@ class MembershipRoutingSpec
doReturn(result(ListAdminsResponse(Seq(dummyUserInfo, okUserInfo))))
.when(membershipService)
.listAdmins("good", okAuth)
Get(s"/groups/good/admins") ~> Route.seal(membershipRoute(okAuth)) ~> check {
Get(s"/groups/good/admins") ~> Route.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.OK
val result = responseAs[ListAdminsResponse]
result.admins should contain theSameElementsAs (Set(dummyUserInfo, okUserInfo))
result.admins should contain theSameElementsAs Set(dummyUserInfo, okUserInfo)
}
}
"return a 404 response when the group is not found" in {
doReturn(result(GroupNotFoundError("fail")))
.when(membershipService)
.listAdmins("notFound", okAuth)
Get(s"/groups/notFound/admins") ~> Route.seal(membershipRoute(okAuth)) ~> check {
Get(s"/groups/notFound/admins") ~> Route.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.NotFound
}
}
@@ -607,7 +611,7 @@ class MembershipRoutingSpec
doReturn(result(new RuntimeException("fail")))
.when(membershipService)
.listAdmins("bad", okAuth)
Get(s"/groups/bad/admins") ~> Route.seal(membershipRoute(okAuth)) ~> check {
Get(s"/groups/bad/admins") ~> Route.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.InternalServerError
}
}
@@ -623,14 +627,14 @@ class MembershipRoutingSpec
.when(membershipService)
.getGroupActivity("ok", None, 100, okAuth)
Get(s"/groups/ok/activity") ~> Route.seal(membershipRoute(okAuth)) ~> check {
Get(s"/groups/ok/activity") ~> Route.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.OK
val result = responseAs[ListGroupChangesResponse]
result.changes should contain theSameElementsAs (Seq(
result.changes should contain theSameElementsAs Seq(
okGroupChangeInfo,
okGroupChangeUpdateInfo,
okGroupChangeDeleteInfo))
okGroupChangeDeleteInfo)
result.maxItems shouldBe 100
result.nextId shouldBe None
result.startFrom shouldBe None
@@ -640,17 +644,17 @@ class MembershipRoutingSpec
doReturn(result(GroupNotFoundError("fail")))
.when(membershipService)
.getGroupActivity("notFound", None, 100, okAuth)
Get(s"/groups/notFound/activity") ~> Route.seal(membershipRoute(okAuth)) ~> check {
Get(s"/groups/notFound/activity") ~> Route.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.NotFound
}
}
"return with a 400 response when the page size is 0" in {
Get(s"/groups/badPageSize/activity?maxItems=0") ~> Route.seal(membershipRoute(okAuth)) ~> check {
Get(s"/groups/badPageSize/activity?maxItems=0") ~> Route.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.BadRequest
}
}
"return a 400 response when maxItems is more than 1000" in {
Get(s"/groups/pageSize/activity?maxItems=1001") ~> Route.seal(membershipRoute(okAuth)) ~> check {
Get(s"/groups/pageSize/activity?maxItems=1001") ~> Route.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.BadRequest
}
}
@@ -664,7 +668,7 @@ class MembershipRoutingSpec
.when(membershipService)
.getGroupActivity(anyString, any[Option[String]], anyInt, any[AuthPrincipal])
Get(s"/groups/pageSize/activity") ~> Route.seal(membershipRoute(okAuth)) ~> check {
Get(s"/groups/pageSize/activity") ~> Route.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.OK
val maxItemsCaptor = ArgumentCaptor.forClass(classOf[Int])
verify(membershipService).getGroupActivity(
@@ -680,19 +684,20 @@ class MembershipRoutingSpec
doReturn(result(new RuntimeException("fail")))
.when(membershipService)
.getGroupActivity("bad", None, 100, okAuth)
Get(s"/groups/bad/activity") ~> Route.seal(membershipRoute(okAuth)) ~> check {
Get(s"/groups/bad/activity") ~> Route.seal(membershipRoute) ~> check {
status shouldBe StatusCodes.InternalServerError
}
}
}
"PUT update user lock status" should {
"return a 200 response with the user locked" in {
membershipRoute = superUserRoute
val updatedUser = okUser.copy(lockStatus = LockStatus.Locked)
doReturn(result(updatedUser))
.when(membershipService)
.updateUserLockStatus("ok", LockStatus.Locked, superUserAuth)
Put("/users/ok/lock") ~> membershipRoute(superUserAuth) ~> check {
Put("/users/ok/lock") ~> membershipRoute ~> check {
status shouldBe StatusCodes.OK
val result = responseAs[UserInfo]
@@ -703,12 +708,13 @@ class MembershipRoutingSpec
}
"return a 200 response with the user unlocked" in {
membershipRoute = superUserRoute
val updatedUser = lockedUser.copy(lockStatus = LockStatus.Unlocked)
doReturn(result(updatedUser))
.when(membershipService)
.updateUserLockStatus("locked", LockStatus.Unlocked, superUserAuth)
Put("/users/locked/unlock") ~> membershipRoute(superUserAuth) ~> check {
Put("/users/locked/unlock") ~> membershipRoute ~> check {
status shouldBe StatusCodes.OK
val result = responseAs[UserInfo]
@@ -722,7 +728,7 @@ class MembershipRoutingSpec
doReturn(result(UserNotFoundError("fail")))
.when(membershipService)
.updateUserLockStatus(anyString, any[LockStatus], any[AuthPrincipal])
Put("/users/notFound/lock") ~> membershipRoute(superUserAuth) ~> check {
Put("/users/notFound/lock") ~> membershipRoute ~> check {
status shouldBe StatusCodes.NotFound
}
}
@@ -731,7 +737,7 @@ class MembershipRoutingSpec
doReturn(result(NotAuthorizedError("fail")))
.when(membershipService)
.updateUserLockStatus(anyString, any[LockStatus], any[AuthPrincipal])
Put("/users/forbidden/lock") ~> membershipRoute(okAuth) ~> check {
Put("/users/forbidden/lock") ~> membershipRoute ~> check {
status shouldBe StatusCodes.Forbidden
}
}

View File

@@ -39,10 +39,8 @@ import scala.util.Random
class RecordSetRoutingSpec
extends WordSpec
with ScalatestRouteTest
with RecordSetRoute
with VinylDNSJsonProtocol
with JsonValidationRejection
with VinylDNSDirectives
with VinylDNSRouteTestHelper
with Matchers {
private val zoneNotFound = Zone("not.found", "test@test.com")
@@ -505,8 +503,8 @@ class RecordSetRoutingSpec
}
val recordSetService: RecordSetServiceAlgebra = new TestService
val vinylDNSAuthenticator = new TestVinylDNSAuthenticator(okAuth)
val recordSetRoute: Route =
new RecordSetRoute(recordSetService, new TestVinylDNSAuthenticator(okAuth)).getRoutes
private def rsJson(recordSet: RecordSet): String =
compact(render(Extraction.decompose(recordSet)))
@@ -516,7 +514,7 @@ class RecordSetRoutingSpec
.withEntity(HttpEntity(ContentTypes.`application/json`, rsJson(recordSet)))
private def validateCreateRecordType(rs: RecordSet) =
post(rs) ~> recordSetRoute(okAuth) ~> check {
post(rs) ~> recordSetRoute ~> check {
status shouldBe StatusCodes.Accepted
val change = responseAs[RecordSetChange]
@@ -544,7 +542,7 @@ class RecordSetRoutingSpec
val errSet = expectedErrs.toSet
Post(s"/zones/${okZone.id}/recordsets")
.withEntity(HttpEntity(ContentTypes.`application/json`, compact(render(js)))) ~>
Route.seal(recordSetRoute(okAuth)) ~> check {
Route.seal(recordSetRoute) ~> check {
status shouldBe StatusCodes.BadRequest
val result = responseAs[JValue]
val errs = (result \ "errors").extractOpt[List[String]]
@@ -569,7 +567,7 @@ class RecordSetRoutingSpec
"GET recordset change" should {
"return the recordset change" in {
Get(s"/zones/${okZone.id}/recordsets/test/changes/good") ~> recordSetRoute(okAuth) ~> check {
Get(s"/zones/${okZone.id}/recordsets/test/changes/good") ~> recordSetRoute ~> check {
status shouldBe StatusCodes.OK
status shouldBe StatusCodes.OK
@@ -582,20 +580,19 @@ class RecordSetRoutingSpec
}
"return a 404 Not Found when the zone doesn't exist" in {
Get(s"/zones/${zoneNotFound.id}/recordsets/test/changes/zoneNotFound") ~> recordSetRoute(
okAuth) ~> check {
Get(s"/zones/${zoneNotFound.id}/recordsets/test/changes/zoneNotFound") ~> recordSetRoute ~> check {
status shouldBe StatusCodes.NotFound
}
}
"return a 404 Not Found when the change doesn't exist" in {
Get(s"/zones/${okZone.id}/recordsets/test/changes/changeNotFound") ~> recordSetRoute(okAuth) ~> check {
Get(s"/zones/${okZone.id}/recordsets/test/changes/changeNotFound") ~> recordSetRoute ~> check {
status shouldBe StatusCodes.NotFound
}
}
"return a forbidden when the user cant see the zone" in {
Get(s"/zones/${okZone.id}/recordsets/test/changes/forbidden") ~> recordSetRoute(okAuth) ~> check {
Get(s"/zones/${okZone.id}/recordsets/test/changes/forbidden") ~> recordSetRoute ~> check {
status shouldBe StatusCodes.Forbidden
}
}
@@ -603,7 +600,7 @@ class RecordSetRoutingSpec
"GET recordset changes" should {
"return the recordset changes" in {
Get(s"/zones/${okZone.id}/recordsetchanges") ~> recordSetRoute(okAuth) ~> check {
Get(s"/zones/${okZone.id}/recordsetchanges") ~> recordSetRoute ~> check {
val response = responseAs[ListRecordSetChangesResponse]
response.zoneId shouldBe okZone.id
@@ -613,22 +610,22 @@ class RecordSetRoutingSpec
}
"return the ZoneNotFoundError when the zone does not exist" in {
Get(s"/zones/${zoneNotFound.id}/recordsetchanges") ~> recordSetRoute(okAuth) ~> check {
Get(s"/zones/${zoneNotFound.id}/recordsetchanges") ~> recordSetRoute ~> check {
status shouldBe StatusCodes.NotFound
}
}
"return a Forbidden when the user is not authorized" in {
Get(s"/zones/${notAuthorizedZone.id}/recordsetchanges") ~> recordSetRoute(okAuth) ~> check {
Get(s"/zones/${notAuthorizedZone.id}/recordsetchanges") ~> recordSetRoute ~> check {
status shouldBe StatusCodes.Forbidden
}
}
"return a Bad Request when maxItems is out of Bounds" in {
Get(s"/zones/${okZone.id}/recordsetchanges?maxItems=101") ~> recordSetRoute(okAuth) ~> check {
Get(s"/zones/${okZone.id}/recordsetchanges?maxItems=101") ~> recordSetRoute ~> check {
status shouldBe StatusCodes.BadRequest
}
Get(s"/zones/${okZone.id}/recordsetchanges?maxItems=0") ~> recordSetRoute(okAuth) ~> check {
Get(s"/zones/${okZone.id}/recordsetchanges?maxItems=0") ~> recordSetRoute ~> check {
status shouldBe StatusCodes.BadRequest
}
}
@@ -636,7 +633,7 @@ class RecordSetRoutingSpec
"GET recordset" should {
"return the recordset summary info" in {
Get(s"/zones/${okZone.id}/recordsets/${rsOk.id}") ~> recordSetRoute(okAuth) ~> check {
Get(s"/zones/${okZone.id}/recordsets/${rsOk.id}") ~> recordSetRoute ~> check {
status shouldBe StatusCodes.OK
val resultRs = responseAs[GetRecordSetResponse].recordSet
resultRs.id shouldBe rsOk.id
@@ -646,13 +643,13 @@ class RecordSetRoutingSpec
}
"return a 404 Not Found when the record set doesn't exist" in {
Get(s"/zones/${okZone.id}/recordsets/${rsNotFound.id}") ~> recordSetRoute(okAuth) ~> check {
Get(s"/zones/${okZone.id}/recordsets/${rsNotFound.id}") ~> recordSetRoute ~> check {
status shouldBe StatusCodes.NotFound
}
}
"return a 404 Not Found when the zone doesn't exist" in {
Get(s"/zones/${zoneNotFound.id}/recordsets/${rsZoneNotFound.id}") ~> recordSetRoute(okAuth) ~> check {
Get(s"/zones/${zoneNotFound.id}/recordsets/${rsZoneNotFound.id}") ~> recordSetRoute ~> check {
status shouldBe StatusCodes.NotFound
}
}
@@ -661,8 +658,7 @@ class RecordSetRoutingSpec
"PUT recordset" should {
"save the changes to the recordset" in {
Put(s"/zones/${okZone.id}/recordsets/${rsOk.id}")
.withEntity(HttpEntity(ContentTypes.`application/json`, rsJson(rsOk))) ~> recordSetRoute(
okAuth) ~> check {
.withEntity(HttpEntity(ContentTypes.`application/json`, rsJson(rsOk))) ~> recordSetRoute ~> check {
status shouldBe StatusCodes.Accepted
val change = responseAs[RecordSetChange]
@@ -678,64 +674,56 @@ class RecordSetRoutingSpec
"return a 404 Not Found when the record set doesn't exist" in {
Put(s"/zones/${okZone.id}/recordsets/${rsNotFound.id}")
.withEntity(HttpEntity(ContentTypes.`application/json`, rsJson(rsNotFound))) ~> recordSetRoute(
okAuth) ~> check {
.withEntity(HttpEntity(ContentTypes.`application/json`, rsJson(rsNotFound))) ~> recordSetRoute ~> check {
status shouldBe StatusCodes.NotFound
}
}
"return a 404 Not Found when the zone doesn't exist" in {
Put(s"/zones/${zoneNotFound.id}/recordsets/${rsZoneNotFound.id}")
.withEntity(HttpEntity(ContentTypes.`application/json`, rsJson(rsZoneNotFound))) ~> recordSetRoute(
okAuth) ~> check {
.withEntity(HttpEntity(ContentTypes.`application/json`, rsJson(rsZoneNotFound))) ~> recordSetRoute ~> check {
status shouldBe StatusCodes.NotFound
}
}
"return a 409 Conflict when the update conflicts with an existing recordset" in {
Put(s"/zones/${okZone.id}/recordsets/${rsAlreadyExists.id}")
.withEntity(HttpEntity(ContentTypes.`application/json`, rsJson(rsAlreadyExists))) ~> recordSetRoute(
okAuth) ~> check {
.withEntity(HttpEntity(ContentTypes.`application/json`, rsJson(rsAlreadyExists))) ~> recordSetRoute ~> check {
status shouldBe StatusCodes.Conflict
}
}
"return a 400 BadRequest when the update is for a deleted zone" in {
Put(s"/zones/${zoneDeleted.id}/recordsets/${rsZoneDeleted.id}")
.withEntity(HttpEntity(ContentTypes.`application/json`, rsJson(rsZoneDeleted))) ~> recordSetRoute(
okAuth) ~> check {
.withEntity(HttpEntity(ContentTypes.`application/json`, rsJson(rsZoneDeleted))) ~> recordSetRoute ~> check {
status shouldBe StatusCodes.BadRequest
}
}
"return a 409 Conflict when the update conflicts with a pending recordset" in {
Put(s"/zones/${okZone.id}/recordsets/${rsPendingUpdate.id}")
.withEntity(HttpEntity(ContentTypes.`application/json`, rsJson(rsPendingUpdate))) ~> recordSetRoute(
okAuth) ~> check {
.withEntity(HttpEntity(ContentTypes.`application/json`, rsJson(rsPendingUpdate))) ~> recordSetRoute ~> check {
status shouldBe StatusCodes.Conflict
}
}
"return a 403 Forbidden when the update is not authorized" in {
Put(s"/zones/${notAuthorizedZone.id}/recordsets/${rsNotAuthorized.id}")
.withEntity(HttpEntity(ContentTypes.`application/json`, rsJson(rsNotAuthorized))) ~> recordSetRoute(
okAuth) ~> check {
.withEntity(HttpEntity(ContentTypes.`application/json`, rsJson(rsNotAuthorized))) ~> recordSetRoute ~> check {
status shouldBe StatusCodes.Forbidden
}
}
"return a 409 Conflict when the zone is syncing" in {
Put(s"/zones/${syncingZone.id}/recordsets/${rsZoneSyncing.id}")
.withEntity(HttpEntity(ContentTypes.`application/json`, rsJson(rsZoneSyncing))) ~> recordSetRoute(
okAuth) ~> check {
.withEntity(HttpEntity(ContentTypes.`application/json`, rsJson(rsZoneSyncing))) ~> recordSetRoute ~> check {
status shouldBe StatusCodes.Conflict
}
}
"return a 422 Unprocessable Entity when the request is invalid" in {
Put(s"/zones/${okZone.id}/recordsets/${rsInvalidRequest.id}")
.withEntity(HttpEntity(ContentTypes.`application/json`, rsJson(rsInvalidRequest))) ~> recordSetRoute(
okAuth) ~> check {
.withEntity(HttpEntity(ContentTypes.`application/json`, rsJson(rsInvalidRequest))) ~> recordSetRoute ~> check {
status shouldBe StatusCodes.UnprocessableEntity
}
}
@@ -743,7 +731,7 @@ class RecordSetRoutingSpec
"return appropriate errors for missing information" in {
Put(s"/zones/${okZone.id}/recordsets/${rsOk.id}")
.withEntity(HttpEntity(ContentTypes.`application/json`, compact(render(rsMissingData)))) ~>
Route.seal(recordSetRoute(okAuth)) ~> check {
Route.seal(recordSetRoute) ~> check {
status shouldBe StatusCodes.BadRequest
val result = responseAs[JValue]
val errs = (result \ "errors").extractOpt[List[String]]
@@ -763,7 +751,7 @@ class RecordSetRoutingSpec
HttpEntity(
ContentTypes.`application/json`,
rsJson(rsOk.copy(zoneId = invalidChangeZone.id)))) ~>
recordSetRoute(okAuth) ~> check {
recordSetRoute ~> check {
status shouldBe StatusCodes.UnprocessableEntity
val error = responseAs[String]
error shouldBe "Cannot update RecordSet's zoneId attribute"
@@ -773,7 +761,7 @@ class RecordSetRoutingSpec
"return appropriate errors for invalid information" in {
Put(s"/zones/${okZone.id}/recordsets/${rsOk.id}")
.withEntity(HttpEntity(ContentTypes.`application/json`, compact(render(rsInvalidType)))) ~>
Route.seal(recordSetRoute(okAuth)) ~> check {
Route.seal(recordSetRoute) ~> check {
status shouldBe StatusCodes.BadRequest
val result = responseAs[JValue]
val errs = (result \ "errors").extractOpt[List[String]]
@@ -785,7 +773,7 @@ class RecordSetRoutingSpec
"return appropriate errors for correct metadata but invalid records" in {
Put(s"/zones/${okZone.id}/recordsets/${rsOk.id}")
.withEntity(HttpEntity(ContentTypes.`application/json`, compact(render(rsInvalidRecord)))) ~>
Route.seal(recordSetRoute(okAuth)) ~> check {
Route.seal(recordSetRoute) ~> check {
status shouldBe StatusCodes.BadRequest
val result = responseAs[JValue]
val errs = (result \ "errors").extractOpt[List[String]]
@@ -797,7 +785,7 @@ class RecordSetRoutingSpec
"return appropriate errors for CNAME record set with multiple records" in {
Put(s"/zones/${okZone.id}/recordsets/${invalidCname.id}")
.withEntity(HttpEntity(ContentTypes.`application/json`, rsJson(invalidCname))) ~>
Route.seal(recordSetRoute(okAuth)) ~> check {
Route.seal(recordSetRoute) ~> check {
status shouldBe StatusCodes.BadRequest
val result = responseAs[JValue]
val errs = (result \ "errors").extractOpt[List[String]]
@@ -809,7 +797,7 @@ class RecordSetRoutingSpec
"GET recordsets" should {
"return all recordsets" in {
Get(s"/zones/${okZone.id}/recordsets") ~> recordSetRoute(okAuth) ~> check {
Get(s"/zones/${okZone.id}/recordsets") ~> recordSetRoute ~> check {
status shouldBe StatusCodes.OK
val resultRs = responseAs[ListRecordSetsResponse]
(resultRs.recordSets.map(_.id) should contain)
@@ -818,7 +806,7 @@ class RecordSetRoutingSpec
}
"return a 404 Not Found when the zone doesn't exist" in {
Get(s"/zones/${zoneNotFound.id}/recordsets") ~> recordSetRoute(okAuth) ~> check {
Get(s"/zones/${zoneNotFound.id}/recordsets") ~> recordSetRoute ~> check {
status shouldBe StatusCodes.NotFound
}
}
@@ -826,7 +814,7 @@ class RecordSetRoutingSpec
"DELETE recordset" should {
"delete the recordset" in {
Delete(s"/zones/${okZone.id}/recordsets/${rsOk.id}") ~> recordSetRoute(okAuth) ~> check {
Delete(s"/zones/${okZone.id}/recordsets/${rsOk.id}") ~> recordSetRoute ~> check {
status shouldBe StatusCodes.Accepted
val change = responseAs[RecordSetChange]
change.changeType shouldBe RecordSetChangeType.Delete
@@ -840,37 +828,37 @@ class RecordSetRoutingSpec
}
"return a 404 Not Found when the record set doesn't exist" in {
Delete(s"/zones/${okZone.id}/recordsets/${rsNotFound.id}") ~> recordSetRoute(okAuth) ~> check {
Delete(s"/zones/${okZone.id}/recordsets/${rsNotFound.id}") ~> recordSetRoute ~> check {
status shouldBe StatusCodes.NotFound
}
}
"return a 404 Not Found when the zone doesn't exist" in {
Delete(s"/zones/${zoneNotFound.id}/recordsets/${rsOk.id}") ~> recordSetRoute(okAuth) ~> check {
Delete(s"/zones/${zoneNotFound.id}/recordsets/${rsOk.id}") ~> recordSetRoute ~> check {
status shouldBe StatusCodes.NotFound
}
}
"return a 400 BadRequest when the delete is for a deleted zone" in {
Delete(s"/zones/${zoneDeleted.id}/recordsets/${rsZoneDeleted.id}") ~> recordSetRoute(okAuth) ~> check {
Delete(s"/zones/${zoneDeleted.id}/recordsets/${rsZoneDeleted.id}") ~> recordSetRoute ~> check {
status shouldBe StatusCodes.BadRequest
}
}
"return a 403 Forbidden when the delete is not authorized" in {
Delete(s"/zones/${notAuthorizedZone.id}/recordsets/${rsOk.id}") ~> recordSetRoute(okAuth) ~> check {
Delete(s"/zones/${notAuthorizedZone.id}/recordsets/${rsOk.id}") ~> recordSetRoute ~> check {
status shouldBe StatusCodes.Forbidden
}
}
"return a 409 Conflict when the zone is syncing" in {
Delete(s"/zones/${syncingZone.id}/recordsets/${rsZoneSyncing.id}") ~> recordSetRoute(okAuth) ~> check {
Delete(s"/zones/${syncingZone.id}/recordsets/${rsZoneSyncing.id}") ~> recordSetRoute ~> check {
status shouldBe StatusCodes.Conflict
}
}
"return a 422 Unprocessable Entity when the request is invalid" in {
Delete(s"/zones/${okZone.id}/recordsets/${rsInvalidRequest.id}") ~> recordSetRoute(okAuth) ~> check {
Delete(s"/zones/${okZone.id}/recordsets/${rsInvalidRequest.id}") ~> recordSetRoute ~> check {
status shouldBe StatusCodes.UnprocessableEntity
}
}
@@ -878,37 +866,37 @@ class RecordSetRoutingSpec
"POST recordset" should {
"return 202 Accepted when the the recordset is created" in {
post(rsOk) ~> recordSetRoute(okAuth) ~> check {
post(rsOk) ~> recordSetRoute ~> check {
status shouldBe StatusCodes.Accepted
}
}
"return a 404 NOT FOUND if the zone does not exist" in {
post(rsZoneNotFound) ~> recordSetRoute(okAuth) ~> check {
post(rsZoneNotFound) ~> recordSetRoute ~> check {
status shouldBe StatusCodes.NotFound
}
}
"return 409 Conflict when adding a record set with an existing name and type" in {
post(rsAlreadyExists) ~> recordSetRoute(okAuth) ~> check {
post(rsAlreadyExists) ~> recordSetRoute ~> check {
status shouldBe StatusCodes.Conflict
}
}
"return a 400 BadRequest when the create is for a deleted zone" in {
post(rsZoneDeleted) ~> recordSetRoute(okAuth) ~> check {
post(rsZoneDeleted) ~> recordSetRoute ~> check {
status shouldBe StatusCodes.BadRequest
}
}
"return a 403 Forbidden when the create is not authorized" in {
post(rsNotAuthorized) ~> recordSetRoute(okAuth) ~> check {
post(rsNotAuthorized) ~> recordSetRoute ~> check {
status shouldBe StatusCodes.Forbidden
}
}
"return a 409 Conflict when the zone is syncing" in {
post(rsZoneSyncing) ~> recordSetRoute(okAuth) ~> check {
post(rsZoneSyncing) ~> recordSetRoute ~> check {
status shouldBe StatusCodes.Conflict
}
}

View File

@@ -26,7 +26,7 @@ import org.mockito.Matchers._
import org.mockito.Mockito._
import org.scalatest.mockito.MockitoSugar
import org.scalatest.{BeforeAndAfterEach, Matchers, OneInstancePerTest, WordSpec}
import vinyldns.core.domain.auth.AuthPrincipal
import vinyldns.api.domain.zone.ZoneServiceAlgebra
import vinyldns.core.route.Monitor
import scala.util.Failure
@@ -37,15 +37,26 @@ class VinylDNSDirectivesSpec
with Matchers
with MockitoSugar
with OneInstancePerTest
with VinylDNSDirectives
with VinylDNSDirectives[Throwable]
with Directives
with VinylDNSJsonProtocol
with BeforeAndAfterEach {
private val mockLatency = mock[Histogram]
private val mockErrors = mock[Meter]
val vinylDNSAuthenticator: VinylDNSAuthenticator = new TestVinylDNSAuthenticator(
mock[AuthPrincipal])
def getRoutes: Route = zoneRoute
val zoneRoute: Route =
new ZoneRoute(mock[ZoneServiceAlgebra], mock[VinylDNSAuthenticator]).getRoutes
val zoneService: ZoneServiceAlgebra = mock[ZoneServiceAlgebra]
val vinylDNSAuthenticator: VinylDNSAuthenticator = mock[VinylDNSAuthenticator]
def handleErrors(e: Throwable): PartialFunction[Throwable, Route] = {
case _ => complete(StatusCodes.InternalServerError)
}
class TestMonitor extends Monitor("test") {
override val latency: Histogram = mockLatency
@@ -149,4 +160,20 @@ class VinylDNSDirectivesSpec
}
}
}
"GET" should {
"return 404 NotFound if route doesn't exist" in {
Get("/no-existo") ~> Route.seal(zoneRoute) ~> check {
response.status shouldBe StatusCodes.NotFound
}
}
}
"PUT" should {
"return 405 MethodNotAllowed if HTTP method is not allowed for that route" in {
Put("/zones") ~> Route.seal(zoneRoute) ~> check {
response.status shouldBe StatusCodes.MethodNotAllowed
}
}
}
}

View File

@@ -0,0 +1,41 @@
/*
* Copyright 2018 Comcast Cable Communications Management, LLC
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package vinyldns.api.route
import akka.http.scaladsl.model.{ContentTypes, HttpEntity, HttpResponse, StatusCodes}
import akka.http.scaladsl.server.Directives.{complete, extractUnmatchedPath}
import akka.http.scaladsl.server.{MalformedRequestContentRejection, RejectionHandler}
import org.json4s.MappingException
trait VinylDNSRouteTestHelper {
implicit def validationRejectionHandler: RejectionHandler =
RejectionHandler
.newBuilder()
.handle {
case MalformedRequestContentRejection(msg, MappingException(_, _)) =>
complete(
HttpResponse(
status = StatusCodes.BadRequest,
entity = HttpEntity(ContentTypes.`application/json`, msg)
))
}
.handleNotFound {
extractUnmatchedPath { p =>
complete((StatusCodes.NotFound, s"The requested path [$p] does not exist."))
}
}
.result()
}

View File

@@ -16,12 +16,13 @@
package vinyldns.api.route
import akka.event.Logging._
import akka.http.scaladsl.model._
import akka.http.scaladsl.model.headers.RawHeader
import akka.http.scaladsl.server.Route
import akka.http.scaladsl.server.directives.LogEntry
import org.scalatest.mockito.MockitoSugar
import org.scalatest.{Matchers, OneInstancePerTest, WordSpec}
import akka.event.Logging._
import akka.http.scaladsl.model.headers.RawHeader
import akka.http.scaladsl.model._
import akka.http.scaladsl.server.directives.LogEntry
import vinyldns.core.domain.auth.AuthPrincipal
class VinylDNSServiceSpec
@@ -29,11 +30,17 @@ class VinylDNSServiceSpec
with Matchers
with MockitoSugar
with OneInstancePerTest
with VinylDNSDirectives {
with VinylDNSDirectives[Throwable] {
val vinylDNSAuthenticator: VinylDNSAuthenticator = new TestVinylDNSAuthenticator(
mock[AuthPrincipal])
def getRoutes: Route = mock[Route]
def handleErrors(e: Throwable): PartialFunction[Throwable, Route] = {
case _ => complete(StatusCodes.InternalServerError)
}
private def buildMockRequest(
path: String = "/path/to/resource",
body: String = "request body") = {

View File

@@ -36,10 +36,8 @@ import vinyldns.core.domain.zone._
class ZoneRoutingSpec
extends WordSpec
with ScalatestRouteTest
with ZoneRoute
with VinylDNSJsonProtocol
with JsonValidationRejection
with VinylDNSDirectives
with VinylDNSRouteTestHelper
with OneInstancePerTest
with Matchers {
@@ -125,6 +123,9 @@ class ZoneRoutingSpec
startFrom = None,
maxItems = 100)
val zoneRoute: Route =
new ZoneRoute(TestZoneService, new TestVinylDNSAuthenticator(okAuth)).getRoutes
object TestZoneService extends ZoneServiceAlgebra {
def connectToZone(
createZoneInput: CreateZoneInput,
@@ -372,8 +373,6 @@ class ZoneRoutingSpec
val zoneService: ZoneServiceAlgebra = TestZoneService
val vinylDNSAuthenticator = new TestVinylDNSAuthenticator(okAuth)
def zoneJson(name: String, email: String): String =
zoneJson(Zone(name, email, connection = null, created = null, status = null, id = null))
@@ -408,8 +407,7 @@ class ZoneRoutingSpec
Set(RecordType.AAAA, RecordType.CNAME))
"return a 202 Accepted when the acl rule is good" in {
Put(s"/zones/${ok.id}/acl/rules")
.withEntity(HttpEntity(ContentTypes.`application/json`, js(goodRequest))) ~> zoneRoute(
okAuth) ~> check {
.withEntity(HttpEntity(ContentTypes.`application/json`, js(goodRequest))) ~> zoneRoute ~> check {
status shouldBe Accepted
val result = responseAs[ZoneChange]
@@ -436,8 +434,7 @@ class ZoneRoutingSpec
recordTypes = Set())
Put(s"/zones/${ok.id}/acl/rules")
.withEntity(HttpEntity(ContentTypes.`application/json`, js(requestAllUsers))) ~> zoneRoute(
okAuth) ~> check {
.withEntity(HttpEntity(ContentTypes.`application/json`, js(requestAllUsers))) ~> zoneRoute ~> check {
status shouldBe Accepted
val result = responseAs[ZoneChange]
@@ -452,16 +449,14 @@ class ZoneRoutingSpec
"return a 404 Not Found when the zone is not found" in {
Put(s"/zones/${notFound.id}/acl/rules")
.withEntity(HttpEntity(ContentTypes.`application/json`, js(userAclRuleInfo))) ~> zoneRoute(
okAuth) ~> check {
.withEntity(HttpEntity(ContentTypes.`application/json`, js(userAclRuleInfo))) ~> zoneRoute ~> check {
status shouldBe NotFound
}
}
"return a 403 Forbidden if not authorized" in {
Put(s"/zones/${notAuthorized.id}/acl/rules")
.withEntity(HttpEntity(ContentTypes.`application/json`, js(userAclRuleInfo))) ~> zoneRoute(
okAuth) ~> check {
.withEntity(HttpEntity(ContentTypes.`application/json`, js(userAclRuleInfo))) ~> zoneRoute ~> check {
status shouldBe Forbidden
}
}
@@ -469,7 +464,7 @@ class ZoneRoutingSpec
"return a 500 if there is an unexpected failure" in {
Put(s"/zones/${error.id}/acl/rules")
.withEntity(HttpEntity(ContentTypes.`application/json`, js(userAclRuleInfo))) ~> Route
.seal(zoneRoute(okAuth)) ~> check {
.seal(zoneRoute) ~> check {
status shouldBe InternalServerError
}
}
@@ -482,7 +477,7 @@ class ZoneRoutingSpec
Put(s"/zones/${ok.id}/acl/rules")
.withEntity(HttpEntity(
ContentTypes.`application/json`,
compact(render(missingACLAccessLevel)))) ~> Route.seal(zoneRoute(okAuth)) ~> check {
compact(render(missingACLAccessLevel)))) ~> Route.seal(zoneRoute) ~> check {
status shouldBe BadRequest
val result = responseAs[JValue]
val errs = (result \ "errors").extractOpt[List[String]]
@@ -502,7 +497,7 @@ class ZoneRoutingSpec
Put(s"/zones/${ok.id}/acl/rules")
.withEntity(HttpEntity(ContentTypes.`application/json`, compact(render(withUserAndGroup)))) ~> Route
.seal(zoneRoute(okAuth)) ~> check {
.seal(zoneRoute) ~> check {
status shouldBe BadRequest
val result = responseAs[JValue]
@@ -523,8 +518,7 @@ class ZoneRoutingSpec
Some("x{5,-3}"),
Set(RecordType.AAAA, RecordType.CNAME))
Put(s"/zones/${badRegex.id}/acl/rules")
.withEntity(HttpEntity(ContentTypes.`application/json`, js(badRequest))) ~> zoneRoute(
okAuth) ~> check {
.withEntity(HttpEntity(ContentTypes.`application/json`, js(badRequest))) ~> zoneRoute ~> check {
status shouldBe BadRequest
val result = responseAs[JValue]
@@ -543,8 +537,7 @@ class ZoneRoutingSpec
Set(RecordType.AAAA, RecordType.CNAME))
"return a 202 Accepted when the acl rule is good" in {
Delete(s"/zones/${ok.id}/acl/rules")
.withEntity(HttpEntity(ContentTypes.`application/json`, js(goodRequest))) ~> zoneRoute(
okAuth) ~> check {
.withEntity(HttpEntity(ContentTypes.`application/json`, js(goodRequest))) ~> zoneRoute ~> check {
status shouldBe Accepted
// just make sure we have a zone change as a response
@@ -555,16 +548,14 @@ class ZoneRoutingSpec
"return a 404 Not Found when the zone is not found" in {
Delete(s"/zones/${notFound.id}/acl/rules")
.withEntity(HttpEntity(ContentTypes.`application/json`, js(userAclRuleInfo))) ~> zoneRoute(
okAuth) ~> check {
.withEntity(HttpEntity(ContentTypes.`application/json`, js(userAclRuleInfo))) ~> zoneRoute ~> check {
status shouldBe NotFound
}
}
"return a 403 Forbidden if not authorized" in {
Delete(s"/zones/${notAuthorized.id}/acl/rules")
.withEntity(HttpEntity(ContentTypes.`application/json`, js(userAclRuleInfo))) ~> zoneRoute(
okAuth) ~> check {
.withEntity(HttpEntity(ContentTypes.`application/json`, js(userAclRuleInfo))) ~> zoneRoute ~> check {
status shouldBe Forbidden
}
}
@@ -572,7 +563,7 @@ class ZoneRoutingSpec
"return a 500 if there is an unexpected failure" in {
Delete(s"/zones/${error.id}/acl/rules")
.withEntity(HttpEntity(ContentTypes.`application/json`, js(userAclRuleInfo))) ~> Route
.seal(zoneRoute(okAuth)) ~> check {
.seal(zoneRoute) ~> check {
status shouldBe InternalServerError
}
}
@@ -585,7 +576,7 @@ class ZoneRoutingSpec
Delete(s"/zones/${ok.id}/acl/rules")
.withEntity(HttpEntity(
ContentTypes.`application/json`,
compact(render(missingACLAccessLevel)))) ~> Route.seal(zoneRoute(okAuth)) ~> check {
compact(render(missingACLAccessLevel)))) ~> Route.seal(zoneRoute) ~> check {
status shouldBe BadRequest
val result = responseAs[JValue]
@@ -606,7 +597,7 @@ class ZoneRoutingSpec
Delete(s"/zones/${ok.id}/acl/rules")
.withEntity(HttpEntity(ContentTypes.`application/json`, compact(render(withUserAndGroup)))) ~> Route
.seal(zoneRoute(okAuth)) ~> check {
.seal(zoneRoute) ~> check {
status shouldBe BadRequest
val result = responseAs[JValue]
@@ -621,13 +612,13 @@ class ZoneRoutingSpec
"POST zone" should {
"return 202 Accepted when the zone is created" in {
post(ok) ~> zoneRoute(okAuth) ~> check {
post(ok) ~> zoneRoute ~> check {
status shouldBe Accepted
}
}
"encrypt the connection and transfer connection keys" in {
post(connectionOk) ~> zoneRoute(okAuth) ~> check {
post(connectionOk) ~> zoneRoute ~> check {
status shouldBe Accepted
val result = responseAs[ZoneChange]
val resultKey = result.zone.connection.get.key
@@ -641,7 +632,7 @@ class ZoneRoutingSpec
}
"return a fully populated zone in the response" in {
post(ok) ~> zoneRoute(okAuth) ~> check {
post(ok) ~> zoneRoute ~> check {
val result = responseAs[ZoneChange]
result.changeType shouldBe ZoneChangeType.Create
Option(result.status) shouldBe defined
@@ -663,7 +654,7 @@ class ZoneRoutingSpec
}
"change the zone name to a fully qualified domain name" in {
post(trailingDot) ~> zoneRoute(okAuth) ~> check {
post(trailingDot) ~> zoneRoute ~> check {
status shouldBe Accepted
val result = responseAs[ZoneChange]
result.changeType shouldBe ZoneChangeType.Create
@@ -684,43 +675,43 @@ class ZoneRoutingSpec
}
"return 409 Conflict if the zone already exists" in {
post(alreadyExists) ~> zoneRoute(okAuth) ~> check {
post(alreadyExists) ~> zoneRoute ~> check {
status shouldBe Conflict
}
}
"return 400 BadRequest if the zone adminGroupId is invalid" in {
post(badAdminId) ~> zoneRoute(okAuth) ~> check {
post(badAdminId) ~> zoneRoute ~> check {
status shouldBe BadRequest
}
}
"return 403 Forbidden if the zone is shared and user is not authorized" in {
post(nonSuperUserSharedZone) ~> zoneRoute(okAuth) ~> check {
post(nonSuperUserSharedZone) ~> zoneRoute ~> check {
status shouldBe Forbidden
}
}
"validate the connection when it is posted" in {
post(connectionOk) ~> zoneRoute(okAuth) ~> check {
post(connectionOk) ~> zoneRoute ~> check {
status shouldBe Accepted
}
}
"fail if the connection validation fails" in {
post(connectionFailed) ~> zoneRoute(okAuth) ~> check {
post(connectionFailed) ~> zoneRoute ~> check {
status shouldBe BadRequest
}
}
"fail if the zone validation fails" in {
post(zoneValidationFailed) ~> zoneRoute(okAuth) ~> check {
post(zoneValidationFailed) ~> zoneRoute ~> check {
status shouldBe BadRequest
}
}
"report missing data" in {
post(missingFields) ~> Route.seal(zoneRoute(okAuth)) ~> check {
post(missingFields) ~> Route.seal(zoneRoute) ~> check {
status shouldBe BadRequest
val result = responseAs[JValue]
val errs = (result \ "errors").extractOpt[List[String]]
@@ -738,7 +729,7 @@ class ZoneRoutingSpec
}
"ignore fields not defined in CreateZoneInput" in {
post(zoneWithInvalidId) ~> Route.seal(zoneRoute(okAuth)) ~> check {
post(zoneWithInvalidId) ~> Route.seal(zoneRoute) ~> check {
status shouldBe Accepted
}
}
@@ -746,7 +737,7 @@ class ZoneRoutingSpec
"DELETE zone" should {
"return 202 on successful delete of existing zone" in {
Delete(s"/zones/${ok.id}") ~> zoneRoute(okAuth) ~> check {
Delete(s"/zones/${ok.id}") ~> zoneRoute ~> check {
status shouldBe Accepted
val result = responseAs[ZoneChange]
@@ -758,19 +749,19 @@ class ZoneRoutingSpec
}
"return 404 if the zone does not exist" in {
Delete(s"/zones/${notFound.id}") ~> zoneRoute(okAuth) ~> check {
Delete(s"/zones/${notFound.id}") ~> zoneRoute ~> check {
status shouldBe NotFound
}
}
"return 403 if the user is not authorized" in {
Delete(s"/zones/${notAuthorized.id}") ~> zoneRoute(okAuth) ~> check {
Delete(s"/zones/${notAuthorized.id}") ~> zoneRoute ~> check {
status shouldBe Forbidden
}
}
"return a 409 Conflict if the zone is unavailable" in {
Delete(s"/zones/${zone1.id}") ~> zoneRoute(okAuth) ~> check {
Delete(s"/zones/${zone1.id}") ~> zoneRoute ~> check {
status shouldBe Conflict
}
}
@@ -778,7 +769,7 @@ class ZoneRoutingSpec
"GET zone" should {
"return the zone is retrieved" in {
Get(s"/zones/${ok.id}") ~> zoneRoute(okAuth) ~> check {
Get(s"/zones/${ok.id}") ~> zoneRoute ~> check {
status shouldBe OK
val resultZone = responseAs[GetZoneResponse].zone
@@ -795,7 +786,7 @@ class ZoneRoutingSpec
}
"return 404 if the zone does not exist" in {
Get(s"/zones/${notFound.id}") ~> zoneRoute(okAuth) ~> check {
Get(s"/zones/${notFound.id}") ~> zoneRoute ~> check {
status shouldBe NotFound
}
}
@@ -803,7 +794,7 @@ class ZoneRoutingSpec
"GET zone by name " should {
"return the zone is retrieved" in {
Get(s"/zones/name/${ok.name}") ~> zoneRoute(okAuth) ~> check {
Get(s"/zones/name/${ok.name}") ~> zoneRoute ~> check {
status shouldBe OK
val resultZone = responseAs[GetZoneResponse].zone
@@ -820,7 +811,7 @@ class ZoneRoutingSpec
}
"return 404 if the zone does not exist" in {
Get(s"/zones/name/${notFound.name}") ~> zoneRoute(okAuth) ~> check {
Get(s"/zones/name/${notFound.name}") ~> zoneRoute ~> check {
status shouldBe NotFound
}
}
@@ -828,7 +819,7 @@ class ZoneRoutingSpec
"GET all zones" should {
"return the zones" in {
Get("/zones") ~> zoneRoute(okAuth) ~> check {
Get("/zones") ~> zoneRoute ~> check {
val zones = responseAs[ListZonesResponse].zones
(zones.map(_.id) should contain)
.only(zone1.id, zone2.id, zone3.id)
@@ -838,7 +829,7 @@ class ZoneRoutingSpec
"GET zones" should {
"return the next id when more results exist" in {
Get(s"/zones?startFrom=zone3.&maxItems=3") ~> zoneRoute(okAuth) ~> check {
Get(s"/zones?startFrom=zone3.&maxItems=3") ~> zoneRoute ~> check {
val resp = responseAs[ListZonesResponse]
val zones = resp.zones
(zones.map(_.id) should contain)
@@ -850,7 +841,7 @@ class ZoneRoutingSpec
}
"not return the next id when there are no more results" in {
Get(s"/zones?startFrom=zone4.&maxItems=4") ~> zoneRoute(okAuth) ~> check {
Get(s"/zones?startFrom=zone4.&maxItems=4") ~> zoneRoute ~> check {
val resp = responseAs[ListZonesResponse]
val zones = resp.zones
(zones.map(_.id) should contain)
@@ -863,7 +854,7 @@ class ZoneRoutingSpec
}
"not return the start from when not provided" in {
Get(s"/zones?maxItems=3") ~> zoneRoute(okAuth) ~> check {
Get(s"/zones?maxItems=3") ~> zoneRoute ~> check {
val resp = responseAs[ListZonesResponse]
val zones = resp.zones
(zones.map(_.id) should contain)
@@ -876,7 +867,7 @@ class ZoneRoutingSpec
}
"return the name filter when provided" in {
Get(s"/zones?nameFilter=foo&startFrom=zone4.&maxItems=4") ~> zoneRoute(okAuth) ~> check {
Get(s"/zones?nameFilter=foo&startFrom=zone4.&maxItems=4") ~> zoneRoute ~> check {
val resp = responseAs[ListZonesResponse]
val zones = resp.zones
(zones.map(_.id) should contain)
@@ -890,7 +881,7 @@ class ZoneRoutingSpec
}
"return all zones when list all is true" in {
Get(s"/zones?maxItems=5&ignoreAccess=true") ~> zoneRoute(okAuth) ~> check {
Get(s"/zones?maxItems=5&ignoreAccess=true") ~> zoneRoute ~> check {
val resp = responseAs[ListZonesResponse]
val zones = resp.zones
(zones.map(_.id) should contain)
@@ -904,7 +895,7 @@ class ZoneRoutingSpec
}
"return an error if the max items is out of range" in {
Get(s"/zones?maxItems=700") ~> zoneRoute(okAuth) ~> check {
Get(s"/zones?maxItems=700") ~> zoneRoute ~> check {
status shouldBe BadRequest
responseEntity.toString should include(
"maxItems was 700, maxItems must be between 0 and 100")
@@ -914,7 +905,7 @@ class ZoneRoutingSpec
"GET zone changes" should {
"return the zone changes" in {
Get(s"/zones/${ok.id}/changes") ~> zoneRoute(okAuth) ~> check {
Get(s"/zones/${ok.id}/changes") ~> zoneRoute ~> check {
val changes = responseAs[ListZoneChangesResponse]
changes.zoneId shouldBe ok.id
@@ -924,22 +915,22 @@ class ZoneRoutingSpec
}
"return the ZoneNotFoundError when the zone does not exist" in {
Get(s"/zones/${notFound.id}/changes") ~> zoneRoute(okAuth) ~> check {
Get(s"/zones/${notFound.id}/changes") ~> zoneRoute ~> check {
status shouldBe NotFound
}
}
"return a Forbidden when the user is not authorized" in {
Get(s"/zones/${notAuthorized.id}/changes") ~> zoneRoute(okAuth) ~> check {
Get(s"/zones/${notAuthorized.id}/changes") ~> zoneRoute ~> check {
status shouldBe Forbidden
}
}
"return a Bad Request when maxItems is out of Bounds" in {
Get(s"/zones/${ok.id}/changes?maxItems=101") ~> zoneRoute(okAuth) ~> check {
Get(s"/zones/${ok.id}/changes?maxItems=101") ~> zoneRoute ~> check {
status shouldBe BadRequest
}
Get(s"/zones/${ok.id}/changes?maxItems=0") ~> zoneRoute(okAuth) ~> check {
Get(s"/zones/${ok.id}/changes?maxItems=0") ~> zoneRoute ~> check {
status shouldBe BadRequest
}
}
@@ -948,7 +939,7 @@ class ZoneRoutingSpec
"PUT zone" should {
"return 202 when the zone is updated" in {
Put(s"/zones/${ok.id}")
.withEntity(HttpEntity(ContentTypes.`application/json`, zoneJson(ok))) ~> zoneRoute(okAuth) ~> check {
.withEntity(HttpEntity(ContentTypes.`application/json`, zoneJson(ok))) ~> zoneRoute ~> check {
status shouldBe Accepted
val result = responseAs[ZoneChange]
result.changeType shouldBe ZoneChangeType.Update
@@ -971,8 +962,7 @@ class ZoneRoutingSpec
"return 404 NotFound if the zone is not found" in {
Put(s"/zones/${notFound.id}")
.withEntity(HttpEntity(ContentTypes.`application/json`, zoneJson(notFound))) ~> zoneRoute(
okAuth) ~> check {
.withEntity(HttpEntity(ContentTypes.`application/json`, zoneJson(notFound))) ~> zoneRoute ~> check {
status shouldBe NotFound
}
}
@@ -980,47 +970,44 @@ class ZoneRoutingSpec
"return 403 if the user is not authorized" in {
Put(s"/zones/${notAuthorized.id}").withEntity(HttpEntity(
ContentTypes.`application/json`,
zoneJson(notAuthorized))) ~> zoneRoute(okAuth) ~> check {
zoneJson(notAuthorized))) ~> zoneRoute ~> check {
status shouldBe Forbidden
}
}
"return 400 BadRequest if the zone adminGroupId is invalid" in {
Put(s"/zones/${badAdminId.id}")
.withEntity(HttpEntity(ContentTypes.`application/json`, zoneJson(badAdminId))) ~> zoneRoute(
okAuth) ~> check {
.withEntity(HttpEntity(ContentTypes.`application/json`, zoneJson(badAdminId))) ~> zoneRoute ~> check {
status shouldBe BadRequest
}
}
"return a 409 Conflict if the zone is unavailable" in {
Put(s"/zones/${zone1.id}")
.withEntity(HttpEntity(ContentTypes.`application/json`, zoneJson(zone1))) ~> zoneRoute(
okAuth) ~> check {
.withEntity(HttpEntity(ContentTypes.`application/json`, zoneJson(zone1))) ~> zoneRoute ~> check {
status shouldBe Conflict
}
}
"validate the connection when the update is posted" in {
Put(s"/zones/${connectionOk.id}").withEntity(HttpEntity(
ContentTypes.`application/json`,
zoneJson(connectionOk))) ~> zoneRoute(okAuth) ~> check {
Put(s"/zones/${connectionOk.id}").withEntity(
HttpEntity(ContentTypes.`application/json`, zoneJson(connectionOk))) ~> zoneRoute ~> check {
status shouldBe Accepted
}
}
"fail the update if the connection validation fails" in {
Put(s"/zones/${connectionFailed.id}").withEntity(
HttpEntity(ContentTypes.`application/json`, zoneJson(connectionFailed))) ~> zoneRoute(
okAuth) ~> check {
Put(s"/zones/${connectionFailed.id}").withEntity(HttpEntity(
ContentTypes.`application/json`,
zoneJson(connectionFailed))) ~> zoneRoute ~> check {
status shouldBe BadRequest
}
}
"fail the update if the zone validation fails" in {
Put(s"/zones/${zoneValidationFailed.id}").withEntity(
HttpEntity(ContentTypes.`application/json`, zoneJson(zoneValidationFailed))) ~> zoneRoute(
okAuth) ~> check {
Put(s"/zones/${zoneValidationFailed.id}").withEntity(HttpEntity(
ContentTypes.`application/json`,
zoneJson(zoneValidationFailed))) ~> zoneRoute ~> check {
status shouldBe BadRequest
}
}
@@ -1028,7 +1015,7 @@ class ZoneRoutingSpec
"report missing data" in {
Put(s"/zones/${ok.id}").withEntity(
HttpEntity(ContentTypes.`application/json`, compact(render(missingFields)))) ~> Route.seal(
zoneRoute(okAuth)) ~> check {
zoneRoute) ~> check {
status shouldBe BadRequest
val result = responseAs[JValue]
val errs = (result \ "errors").extractOpt[List[String]]
@@ -1049,7 +1036,7 @@ class ZoneRoutingSpec
"report type mismatch" in {
Put(s"/zones/${ok.id}").withEntity(
HttpEntity(ContentTypes.`application/json`, compact(render(zoneWithInvalidId)))) ~> Route
.seal(zoneRoute(okAuth)) ~> check {
.seal(zoneRoute) ~> check {
status shouldBe BadRequest
val result = responseAs[JValue]
val errs = (result \ "errors").extractOpt[List[String]]
@@ -1061,39 +1048,39 @@ class ZoneRoutingSpec
"POST zone sync" should {
"return 202 Accepted if the zone can be synced" in {
Post(s"/zones/${ok.id}/sync") ~> zoneRoute(okAuth) ~> check {
Post(s"/zones/${ok.id}/sync") ~> zoneRoute ~> check {
val result = responseAs[ZoneChange]
result.changeType shouldBe ZoneChangeType.Sync
status shouldBe Accepted
}
}
"return 404 NotFound if the zone is not found" in {
Post(s"/zones/${notFound.id}/sync") ~> zoneRoute(okAuth) ~> check {
Post(s"/zones/${notFound.id}/sync") ~> zoneRoute ~> check {
status shouldBe NotFound
}
}
"return a Forbidden if the user is not authorized" in {
Post(s"/zones/${notAuthorized.id}/sync") ~> zoneRoute(okAuth) ~> check {
Post(s"/zones/${notAuthorized.id}/sync") ~> zoneRoute ~> check {
status shouldBe Forbidden
}
}
"return a BadRequest if the zone is in an invalid state to be synced" in {
Post(s"/zones/${zone1.id}/sync") ~> zoneRoute(okAuth) ~> check {
Post(s"/zones/${zone1.id}/sync") ~> zoneRoute ~> check {
status shouldBe BadRequest
}
}
"return a Conflict if the zone has a pending update" in {
Post(s"/zones/${zone2.id}/sync") ~> zoneRoute(okAuth) ~> check {
Post(s"/zones/${zone2.id}/sync") ~> zoneRoute ~> check {
status shouldBe Conflict
}
}
"return Forbidden if the zone has recently been synced" in {
Post(s"/zones/${zone3.id}/sync") ~> zoneRoute(okAuth) ~> check {
Post(s"/zones/${zone3.id}/sync") ~> zoneRoute ~> check {
status shouldBe Forbidden
}
}
"return a Conflict if the zone is currently syncing" in {
Post(s"/zones/${zone5.id}/sync") ~> zoneRoute(okAuth) ~> check {
Post(s"/zones/${zone5.id}/sync") ~> zoneRoute ~> check {
status shouldBe Conflict
}
}
@@ -1101,7 +1088,7 @@ class ZoneRoutingSpec
"GET backendids" should {
"return a 200 OK with the backend ids" in {
Get("/zones/backendids") ~> zoneRoute(okAuth) ~> check {
Get("/zones/backendids") ~> zoneRoute ~> check {
status shouldBe OK
val result = responseAs[List[String]]
result shouldBe List("backend-1", "backend-2")

View File

@@ -77,27 +77,3 @@ case class Group(
memberIds = memberIdsUpdate ++ adminUserIdsUpdate,
adminUserIds = adminUserIdsUpdate)
}
object Group {
import cats.data.ValidatedNel
import cats.implicits._
def build(
name: String,
email: String,
description: Option[String],
members: Set[String],
admins: Set[String]): ValidatedNel[String, Group] =
Group(name, email, description, memberIds = members ++ admins, adminUserIds = admins)
.validNel[String]
def build(
id: String,
name: String,
email: String,
description: Option[String],
members: Set[String],
admins: Set[String]): ValidatedNel[String, Group] =
Group(name, email, description, id, memberIds = members ++ admins, adminUserIds = admins)
.validNel[String]
}

View File

@@ -39,8 +39,8 @@
<parameter name="illegalImports"><![CDATA[sun._,java.awt._]]></parameter>
</parameters>
</check>
<check level="error" class="org.scalastyle.scalariform.NoWhitespaceBeforeLeftBracketChecker" enabled="true"></check>
<check level="error" class="org.scalastyle.scalariform.NoWhitespaceAfterLeftBracketChecker" enabled="true"></check>
<check level="warning" class="org.scalastyle.scalariform.NoWhitespaceBeforeLeftBracketChecker" enabled="true"></check>
<check level="warning" class="org.scalastyle.scalariform.NoWhitespaceAfterLeftBracketChecker" enabled="true"></check>
<check level="warning" class="org.scalastyle.scalariform.ReturnChecker" enabled="true"></check>
<check level="warning" class="org.scalastyle.scalariform.NullChecker" enabled="true"></check>
<check level="error" class="org.scalastyle.scalariform.NoCloneChecker" enabled="true"></check>