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 vinyldns_python import VinylDNSClient
from dns.resolver import * from dns.resolver import *
from vinyldns_context import VinylDNSTestContext from vinyldns_context import VinylDNSTestContext
from requests.compat import urljoin
def test_request_fails_when_user_account_is_locked(): 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 = VinylDNSClient(VinylDNSTestContext.vinyldns_url, 'okAccessKey', 'okSecretKey')
client.list_batch_change_summaries(status=200) 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: finally:
if result: if result:
client.delete_group(result['id'], status=(200,404)) 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: finally:
if saved_group: if saved_group:
ok_client.delete_group(saved_group['id'], status=(200,404)) 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,102 +17,109 @@
package vinyldns.api.route package vinyldns.api.route
import akka.http.scaladsl.model.StatusCodes import akka.http.scaladsl.model.StatusCodes
import akka.http.scaladsl.server import akka.http.scaladsl.server.{RejectionHandler, Route, ValidationRejection}
import akka.http.scaladsl.server.{Directives, RejectionHandler, Route, ValidationRejection}
import cats.data.EitherT
import cats.effect._
import vinyldns.api.VinylDNSConfig import vinyldns.api.VinylDNSConfig
import vinyldns.core.domain.auth.AuthPrincipal
import vinyldns.core.domain.batch._ import vinyldns.core.domain.batch._
import vinyldns.api.domain.batch._ import vinyldns.api.domain.batch._
trait BatchChangeRoute extends Directives { class BatchChangeRoute(
this: VinylDNSJsonProtocol with VinylDNSDirectives with JsonValidationRejection => 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 final private val MAX_ITEMS_LIMIT: Int = 100
val batchChangeRoute: AuthPrincipal => server.Route = { authPrincipal: AuthPrincipal => val batchChangeRoute: Route = {
val standardBatchChangeRoutes = (post & path("zones" / "batchrecordchanges")) { val standardBatchChangeRoutes = path("zones" / "batchrecordchanges") {
parameters("allowManualReview".as[Boolean].?(true)) { allowManualReview: Boolean => (post & monitor("Endpoint.postBatchChange")) {
{ parameters("allowManualReview".as[Boolean].?(true)) { allowManualReview: Boolean =>
monitor("Endpoint.postBatchChange") { authenticateAndExecuteWithEntity[BatchChange, BatchChangeInput](
entity(as[BatchChangeInput]) { batchChangeInput => (authPrincipal, batchChangeInput) =>
execute(batchChangeService batchChangeService
.applyBatchChange(batchChangeInput, authPrincipal, allowManualReview)) { chg => .applyBatchChange(batchChangeInput, authPrincipal, allowManualReview)) { chg =>
complete(StatusCodes.Accepted, 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( parameters(
"startFrom".as[Int].?, "startFrom".as[Int].?,
"maxItems".as[Int].?(MAX_ITEMS_LIMIT), "maxItems".as[Int].?(MAX_ITEMS_LIMIT),
"ignoreAccess".as[Boolean].?(false), "ignoreAccess".as[Boolean].?(false),
"approvalStatus".as[String].?) { "approvalStatus".as[String].?) {
( (
startFrom: Option[Int], startFrom: Option[Int],
maxItems: Int, maxItems: Int,
ignoreAccess: Boolean, ignoreAccess: Boolean,
approvalStatus: Option[String]) => approvalStatus: Option[String]) =>
{ {
val convertApprovalStatus = approvalStatus.flatMap(BatchChangeApprovalStatus.find) val convertApprovalStatus = approvalStatus.flatMap(BatchChangeApprovalStatus.find)
handleRejections(invalidQueryHandler) { handleRejections(invalidQueryHandler) {
validate( validate(
0 < maxItems && maxItems <= MAX_ITEMS_LIMIT, 0 < maxItems && maxItems <= MAX_ITEMS_LIMIT,
s"maxItems was $maxItems, maxItems must be between 1 and $MAX_ITEMS_LIMIT, inclusive.") { s"maxItems was $maxItems, maxItems must be between 1 and $MAX_ITEMS_LIMIT, inclusive.") {
execute( authenticateAndExecute(
batchChangeService.listBatchChangeSummaries( batchChangeService.listBatchChangeSummaries(
authPrincipal, _,
startFrom, startFrom,
maxItems, maxItems,
ignoreAccess, ignoreAccess,
convertApprovalStatus)) { summaries => convertApprovalStatus)) { summaries =>
complete(StatusCodes.OK, summaries) complete(StatusCodes.OK, summaries)
}
} }
} }
} }
} }
}
} ~
path("zones" / "batchrecordchanges" / Segment) { id =>
(get & monitor("Endpoint.getBatchChange")) {
authenticateAndExecute(batchChangeService.getBatchChange(id, _)) { chg =>
complete(StatusCodes.OK, chg)
}
} }
} }
val manualBatchReviewRoutes = val manualBatchReviewRoutes =
(post & path("zones" / "batchrecordchanges" / Segment / "reject")) { id => path("zones" / "batchrecordchanges" / Segment / "reject") { id =>
monitor("Endpoint.rejectBatchChange") { (post & monitor("Endpoint.rejectBatchChange")) {
entity(as[Option[RejectBatchChangeInput]]) { input => authenticateAndExecuteWithEntity[BatchChange, Option[RejectBatchChangeInput]](
execute( (authPrincipal, input) =>
batchChangeService batchChangeService
.rejectBatchChange(id, authPrincipal, input.getOrElse(RejectBatchChangeInput()))) { .rejectBatchChange(id, authPrincipal, input.getOrElse(RejectBatchChangeInput()))) {
chg => chg =>
complete(StatusCodes.OK, chg) complete(StatusCodes.OK, chg)
}
// TODO: Update response entity to return modified batch change
} }
// TODO: Update response entity to return modified batch change
} }
} ~ } ~
(post & path("zones" / "batchrecordchanges" / Segment / "approve")) { id => path("zones" / "batchrecordchanges" / Segment / "approve") { id =>
monitor("Endpoint.approveBatchChange") { (post & monitor("Endpoint.approveBatchChange")) {
entity(as[Option[ApproveBatchChangeInput]]) { input => authenticateAndExecuteWithEntity[BatchChange, Option[ApproveBatchChangeInput]](
execute( (authPrincipal, input) =>
batchChangeService batchChangeService
.approveBatchChange( .approveBatchChange(
id, id,
authPrincipal, authPrincipal,
input.getOrElse(ApproveBatchChangeInput()))) { chg => input.getOrElse(ApproveBatchChangeInput()))) { chg =>
complete(StatusCodes.OK, chg) complete(StatusCodes.OK, chg)
}
// TODO: Update response entity to return modified batch change // TODO: Update response entity to return modified batch change
} }
} }
@@ -130,18 +137,4 @@ trait BatchChangeRoute extends Directives {
complete(StatusCodes.BadRequest, msg) complete(StatusCodes.BadRequest, msg)
} }
.result() .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 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.Validated.{Invalid, Valid}
import cats.data._
import cats.implicits._
import com.fasterxml.jackson.core.JsonParseException import com.fasterxml.jackson.core.JsonParseException
import de.heikoseeberger.akkahttpjson4s.Json4sSupport import de.heikoseeberger.akkahttpjson4s.Json4sSupport
import org.joda.time.DateTime import org.joda.time.DateTime
@@ -30,26 +30,9 @@ import org.json4s.ext._
import org.json4s.jackson.JsonMethods._ import org.json4s.jackson.JsonMethods._
import scala.reflect.ClassTag import scala.reflect.ClassTag
import cats.data._
import cats.implicits._
case class JsonErrors(errors: List[String]) 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 // 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), // revert the date time formatting here. When changing to circe (updating to java8 instant),
// be sure to check the format of date time // be sure to check the format of date time

View File

@@ -18,170 +18,151 @@ package vinyldns.api.route
import akka.http.scaladsl.model.StatusCodes import akka.http.scaladsl.model.StatusCodes
import akka.http.scaladsl.server._ import akka.http.scaladsl.server._
import vinyldns.api.Interfaces.Result
import vinyldns.api.domain.membership._ import vinyldns.api.domain.membership._
import vinyldns.api.domain.zone.NotAuthorizedError import vinyldns.api.domain.zone.NotAuthorizedError
import vinyldns.api.route.MembershipJsonProtocol.{CreateGroupInput, UpdateGroupInput} import vinyldns.api.route.MembershipJsonProtocol.{CreateGroupInput, UpdateGroupInput}
import vinyldns.core.domain.auth.AuthPrincipal
import vinyldns.core.domain.membership.{Group, LockStatus} import vinyldns.core.domain.membership.{Group, LockStatus}
trait MembershipRoute extends Directives { class MembershipRoute(
this: VinylDNSJsonProtocol with VinylDNSDirectives with JsonValidationRejection => membershipService: MembershipServiceAlgebra,
val vinylDNSAuthenticator: VinylDNSAuthenticator)
extends VinylDNSJsonProtocol
with VinylDNSDirectives[Throwable] {
final private val DEFAULT_MAX_ITEMS: Int = 100 final private val DEFAULT_MAX_ITEMS: Int = 100
final private val MAX_ITEMS_LIMIT: Int = 1000 final private val MAX_ITEMS_LIMIT: Int = 1000
val membershipService: MembershipServiceAlgebra def getRoutes: Route = membershipRoute
val membershipRoute = { authPrincipal: AuthPrincipal => def handleErrors(e: Throwable): PartialFunction[Throwable, Route] = {
path("groups" / Segment) { groupId => case GroupNotFoundError(msg) => complete(StatusCodes.NotFound, msg)
get { case NotAuthorizedError(msg) => complete(StatusCodes.Forbidden, msg)
monitor("Endpoint.getGroup") { case GroupAlreadyExistsError(msg) => complete(StatusCodes.Conflict, msg)
execute(membershipService.getGroup(groupId, authPrincipal)) { group => case InvalidGroupError(msg) => complete(StatusCodes.BadRequest, msg)
complete(StatusCodes.OK, GroupInfo(group)) 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")) {
authenticateAndExecute(membershipService.deleteGroup(groupId, _)) { group =>
complete(StatusCodes.OK, GroupInfo(group))
}
}
} ~
path("groups") {
(post & monitor("Endpoint.createGroup")) {
authenticateAndExecuteWithEntity[Group, CreateGroupInput] { (authPrincipal, input) =>
val group = Group(
input.name,
input.email,
input.description,
memberIds = (input.members ++ input.admins).map(_.id),
adminUserIds = input.admins.map(_.id))
membershipService.createGroup(group, authPrincipal)
} { group =>
complete(StatusCodes.OK, GroupInfo(group))
} }
} ~ } ~
delete { (get & monitor("Endpoint.listMyGroups")) {
monitor("Endpoint.deleteGroup") { parameters("startFrom".?, "maxItems".as[Int].?(DEFAULT_MAX_ITEMS), "groupNameFilter".?) {
execute(membershipService.deleteGroup(groupId, authPrincipal)) { group => (startFrom: Option[String], maxItems: Int, groupNameFilter: Option[String]) =>
complete(StatusCodes.OK, GroupInfo(group)) {
} handleRejections(invalidQueryHandler) {
validate(
check = 0 < maxItems && maxItems <= MAX_ITEMS_LIMIT,
errorMsg = s"""
| maxItems was $maxItems, maxItems must be between 0 exclusive
| and $MAX_ITEMS_LIMIT inclusive"
""".stripMargin
) {
authenticateAndExecute(membershipService
.listMyGroups(groupNameFilter, startFrom, maxItems, _)) { groups =>
complete(StatusCodes.OK, groups)
}
}
}
}
} }
} }
} ~ } ~
path("groups") { path("groups" / Segment) { _ =>
post { (put & monitor("Endpoint.updateGroup")) {
monitor("Endpoint.createGroup") { authenticateAndExecuteWithEntity[Group, UpdateGroupInput](
entity(as[CreateGroupInput]) { input => (authPrincipal, input) =>
ifValid( membershipService.updateGroup(
Group input.id,
.build( input.name,
input.name, input.email,
input.email, input.description,
input.description, (input.members ++ input.admins).map(_.id),
input.members.map(_.id), input.admins.map(_.id),
input.admins.map(_.id))) { inputGroup: Group => authPrincipal)) { group =>
execute(membershipService.createGroup(inputGroup, authPrincipal)) { group => complete(StatusCodes.OK, GroupInfo(group))
complete(StatusCodes.OK, GroupInfo(group)) }
}
} ~
path("groups" / Segment / "members") { groupId =>
(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") {
authenticateAndExecute(membershipService
.listMembers(groupId, startFrom, maxItems, _)) { members =>
complete(StatusCodes.OK, members)
} }
} }
} }
} }
} ~ }
get { } ~
parameters("startFrom".?, "maxItems".as[Int].?(DEFAULT_MAX_ITEMS), "groupNameFilter".?) { path("groups" / Segment / "admins") { groupId =>
(startFrom: Option[String], maxItems: Int, groupNameFilter: Option[String]) => (get & monitor("Endpoint.listGroupAdmins")) {
{ authenticateAndExecute(membershipService.listAdmins(groupId, _)) { admins =>
monitor("Endpoint.listMyGroups") { complete(StatusCodes.OK, admins)
handleRejections(invalidQueryHandler) { }
validate( }
check = 0 < maxItems && maxItems <= MAX_ITEMS_LIMIT, } ~
errorMsg = s""" path("groups" / Segment / "activity") { groupId =>
| maxItems was $maxItems, maxItems must be between 0 exclusive (get & monitor("Endpoint.groupActivity")) {
| and $MAX_ITEMS_LIMIT inclusive" parameters("startFrom".?, "maxItems".as[Int].?(DEFAULT_MAX_ITEMS)) {
""".stripMargin (startFrom: Option[String], maxItems: Int) =>
) { handleRejections(invalidQueryHandler) {
execute(membershipService validate(
.listMyGroups(groupNameFilter, startFrom, maxItems, authPrincipal)) { 0 < maxItems && maxItems <= MAX_ITEMS_LIMIT,
groups => s"maxItems was $maxItems, maxItems must be between 0 and $MAX_ITEMS_LIMIT") {
complete(StatusCodes.OK, groups) authenticateAndExecute(membershipService
} .getGroupActivity(groupId, startFrom, maxItems, _)) { activity =>
} complete(StatusCodes.OK, activity)
}
}
}
}
}
} ~
path("groups" / Segment) { _ =>
put {
monitor("Endpoint.updateGroup") {
entity(as[UpdateGroupInput]) { input =>
ifValid(
Group.build(
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,
authPrincipal)) { group =>
complete(StatusCodes.OK, GroupInfo(group))
} }
} }
} }
}
} }
} ~ }
path("groups" / Segment / "members") { groupId => } ~
get { path("users" / Segment / "lock") { id =>
monitor("Endpoint.listGroupMembers") { (put & monitor("Endpoint.lockUser")) {
parameters("startFrom".?, "maxItems".as[Int].?(DEFAULT_MAX_ITEMS)) { authenticateAndExecute(membershipService.updateUserLockStatus(id, LockStatus.Locked, _)) {
(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 =>
complete(StatusCodes.OK, members)
}
}
}
}
}
}
} ~
path("groups" / Segment / "admins") { groupId =>
get {
monitor("Endpoint.listGroupAdmins") {
execute(membershipService.listAdmins(groupId, authPrincipal)) { admins =>
complete(StatusCodes.OK, admins)
}
}
}
} ~
path("groups" / Segment / "activity") { groupId =>
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 =>
complete(StatusCodes.OK, activity)
}
}
}
}
}
}
} ~
(put & path("users" / Segment / "lock") & monitor("Endpoint.lockUser")) { id =>
execute(membershipService.updateUserLockStatus(id, LockStatus.Locked, authPrincipal)) {
user =>
complete(StatusCodes.OK, UserInfo(user))
}
} ~
(put & path("users" / Segment / "unlock") & monitor("Endpoint.unlockUser")) { id =>
execute(membershipService.updateUserLockStatus(id, LockStatus.Unlocked, authPrincipal)) {
user => user =>
complete(StatusCodes.OK, UserInfo(user)) complete(StatusCodes.OK, UserInfo(user))
} }
} }
} } ~
path("users" / Segment / "unlock") { id =>
(put & monitor("Endpoint.unlockUser")) {
authenticateAndExecute(membershipService.updateUserLockStatus(id, LockStatus.Unlocked, _)) {
user =>
complete(StatusCodes.OK, UserInfo(user))
}
}
}
private val invalidQueryHandler = RejectionHandler private val invalidQueryHandler = RejectionHandler
.newBuilder() .newBuilder()
@@ -190,16 +171,4 @@ trait MembershipRoute extends Directives {
complete(StatusCodes.BadRequest, msg) complete(StatusCodes.BadRequest, msg)
} }
.result() .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 package vinyldns.api.route
import akka.http.scaladsl.model.StatusCodes 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 akka.util.Timeout
import vinyldns.api.Interfaces._ import vinyldns.api.Interfaces._
import vinyldns.api.domain.record.RecordSetServiceAlgebra import vinyldns.api.domain.record.RecordSetServiceAlgebra
import vinyldns.api.domain.zone._ import vinyldns.api.domain.zone._
import vinyldns.core.domain.auth.AuthPrincipal
import vinyldns.core.domain.record.RecordSet import vinyldns.core.domain.record.RecordSet
import vinyldns.core.domain.zone.ZoneCommandResult
import scala.concurrent.duration._ import scala.concurrent.duration._
@@ -35,115 +35,109 @@ case class ListRecordSetsResponse(
maxItems: Option[Int] = None, maxItems: Option[Int] = None,
recordNameFilter: Option[String] = None) recordNameFilter: Option[String] = None)
trait RecordSetRoute extends Directives { class RecordSetRoute(
this: VinylDNSJsonProtocol with VinylDNSDirectives with JsonValidationRejection => recordSetService: RecordSetServiceAlgebra,
final private val DEFAULT_MAX_ITEMS: Int = 100 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 // Timeout must be long enough to allow the cluster to form
implicit val rsCmdTimeout: Timeout = Timeout(10.seconds) implicit val rsCmdTimeout: Timeout = Timeout(10.seconds)
val recordSetRoute = { authPrincipal: AuthPrincipal => def handleErrors(e: Throwable): PartialFunction[Throwable, Route] = {
path("zones" / Segment / "recordsets") { zoneId => case ZoneNotFoundError(msg) => complete(StatusCodes.NotFound, msg)
post { case RecordSetAlreadyExists(msg) => complete(StatusCodes.Conflict, msg)
monitor("Endpoint.addRecordSet") { case ZoneInactiveError(msg) => complete(StatusCodes.BadRequest, msg)
entity(as[RecordSet]) { rs => case NotAuthorizedError(msg) => complete(StatusCodes.Forbidden, msg)
execute(recordSetService.addRecordSet(rs, authPrincipal)) { rc => case ZoneUnavailableError(msg) => complete(StatusCodes.Conflict, msg)
complete(StatusCodes.Accepted, rc) 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")) {
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") {
authenticateAndExecute(recordSetService
.listRecordSets(zoneId, startFrom, Some(maxItems), recordNameFilter, _)) {
rsResponse =>
complete(StatusCodes.OK, rsResponse)
}
}
} }
} }
}
} ~
path("zones" / Segment / "recordsets" / Segment) { (zoneId, rsId) =>
(get & monitor("Endpoint.getRecordSet")) {
authenticateAndExecute(recordSetService.getRecordSet(rsId, zoneId, _)) { rs =>
complete(StatusCodes.OK, GetRecordSetResponse(rs))
} }
} ~ } ~
get { (delete & monitor("Endpoint.deleteRecordSet")) {
monitor("Endpoint.getRecordSets") { authenticateAndExecute(recordSetService.deleteRecordSet(rsId, zoneId, _)) { rc =>
parameters("startFrom".?, "maxItems".as[Int].?(DEFAULT_MAX_ITEMS), "recordNameFilter".?) { complete(StatusCodes.Accepted, rc)
(startFrom: Option[String], maxItems: Int, recordNameFilter: Option[String]) => }
handleRejections(invalidQueryHandler) { } ~
validate( (put & monitor("Endpoint.updateRecordSet")) {
0 < maxItems && maxItems <= DEFAULT_MAX_ITEMS, authenticateAndExecuteWithEntity[ZoneCommandResult, RecordSet] {
s"maxItems was ${maxItems}, maxItems must be between 0 and $DEFAULT_MAX_ITEMS") { (authPrincipal, recordSet) =>
execute( recordSet match {
recordSetService.listRecordSets( case badRs if badRs.zoneId != zoneId =>
zoneId, Left(InvalidRequest("Cannot update RecordSet's zoneId attribute")).toResult
startFrom, case goodRs =>
Some(maxItems), recordSetService.updateRecordSet(goodRs, authPrincipal)
recordNameFilter, }
authPrincipal)) { rsResponse => } { rc =>
complete(StatusCodes.OK, rsResponse) complete(StatusCodes.Accepted, rc)
}
}
}
}
} }
} }
} ~ } ~
path("zones" / Segment / "recordsets" / Segment) { (zoneId, rsId) => path("zones" / Segment / "recordsets" / Segment / "changes" / Segment) {
get { (zoneId, _, changeId) =>
monitor("Endpoint.getRecordSet") { (get & monitor("Endpoint.getRecordSetChange")) {
execute(recordSetService.getRecordSet(rsId, zoneId, authPrincipal)) { rs => authenticateAndExecute(recordSetService.getRecordSetChange(zoneId, changeId, _)) {
complete(StatusCodes.OK, GetRecordSetResponse(rs)) change =>
} complete(StatusCodes.OK, change)
}
} ~
delete {
monitor("Endpoint.deleteRecordSet") {
execute(recordSetService.deleteRecordSet(rsId, zoneId, authPrincipal)) { 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 =>
complete(StatusCodes.Accepted, rc)
}
}
}
}
}
}
} ~
path("zones" / Segment / "recordsets" / Segment / "changes" / Segment) {
(zoneId, _, changeId) =>
get {
monitor("Endpoint.getRecordSetChange") {
execute(recordSetService.getRecordSetChange(zoneId, changeId, authPrincipal)) {
change =>
complete(StatusCodes.OK, change)
}
}
}
} ~
path("zones" / Segment / "recordsetchanges") { zoneId =>
get {
monitor("Endpoint.listRecordSetChanges") {
parameters("startFrom".?, "maxItems".as[Int].?(DEFAULT_MAX_ITEMS)) {
(startFrom: Option[String], maxItems: Int) =>
handleRejections(invalidQueryHandler) {
validate(
check = 0 < maxItems && maxItems <= DEFAULT_MAX_ITEMS,
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 =>
complete(StatusCodes.OK, changes)
}
}
}
}
} }
} }
} ~
path("zones" / Segment / "recordsetchanges") { zoneId =>
(get & monitor("Endpoint.listRecordSetChanges")) {
parameters("startFrom".?, "maxItems".as[Int].?(DEFAULT_MAX_ITEMS)) {
(startFrom: Option[String], maxItems: Int) =>
handleRejections(invalidQueryHandler) {
validate(
check = 0 < maxItems && maxItems <= DEFAULT_MAX_ITEMS,
errorMsg = s"maxItems was $maxItems, maxItems must be between 0 exclusive " +
s"and $DEFAULT_MAX_ITEMS inclusive"
) {
authenticateAndExecute(recordSetService
.listRecordSetChanges(zoneId, startFrom, maxItems, _)) { changes =>
complete(StatusCodes.OK, changes)
}
}
}
}
} }
} }
private val invalidQueryHandler = RejectionHandler private val invalidQueryHandler = RejectionHandler
.newBuilder() .newBuilder()
@@ -152,28 +146,4 @@ trait RecordSetRoute extends Directives {
complete(StatusCodes.BadRequest, msg) complete(StatusCodes.BadRequest, msg)
} }
.result() .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.RouteResult.{Complete, Rejected}
import akka.http.scaladsl.server._ import akka.http.scaladsl.server._
import akka.http.scaladsl.server.directives.BasicDirectives import akka.http.scaladsl.server.directives.BasicDirectives
import akka.http.scaladsl.unmarshalling.FromRequestUnmarshaller
import cats.data.Validated.{Invalid, Valid} 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.JsonDSL._
import org.json4s.jackson.JsonMethods._ import org.json4s.jackson.JsonMethods._
import vinyldns.core.domain.auth.AuthPrincipal import vinyldns.core.domain.auth.AuthPrincipal
@@ -31,10 +33,12 @@ import scala.concurrent.duration._
import scala.util.Failure import scala.util.Failure
import scala.util.control.NonFatal import scala.util.control.NonFatal
trait VinylDNSDirectives extends Directives { trait VinylDNSDirectives[E] extends Directives {
val vinylDNSAuthenticator: VinylDNSAuthenticator val vinylDNSAuthenticator: VinylDNSAuthenticator
def getRoutes: Route
def authenticate: Directive1[AuthPrincipal] = extractRequestContext.flatMap { ctx => def authenticate: Directive1[AuthPrincipal] = extractRequestContext.flatMap { ctx =>
extractStrictEntity(10.seconds).flatMap { strictEntity => extractStrictEntity(10.seconds).flatMap { strictEntity =>
onSuccess( onSuccess(
@@ -124,4 +128,49 @@ trait VinylDNSDirectives extends Directives {
case Invalid(errors) => case Invalid(errors) =>
reject(ValidationRejection(compact(render("errors" -> errors.toList.toSet)))) 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.event.Logging._
import akka.http.scaladsl.model._ import akka.http.scaladsl.model._
import akka.http.scaladsl.server import akka.http.scaladsl.server.{MalformedRequestContentRejection, RejectionHandler, Route}
import akka.http.scaladsl.server.Route
import akka.http.scaladsl.server.RouteResult.{Complete, Rejected} import akka.http.scaladsl.server.RouteResult.{Complete, Rejected}
import akka.http.scaladsl.server.directives.LogEntry import akka.http.scaladsl.server.directives.LogEntry
import cats.effect.IO import cats.effect.IO
import fs2.concurrent.SignallingRef import fs2.concurrent.SignallingRef
import io.prometheus.client.CollectorRegistry import io.prometheus.client.CollectorRegistry
import vinyldns.api.domain.auth.AuthPrincipalProvider import vinyldns.api.domain.auth.AuthPrincipalProvider
import org.json4s.MappingException
import vinyldns.api.domain.batch.BatchChangeServiceAlgebra import vinyldns.api.domain.batch.BatchChangeServiceAlgebra
import vinyldns.api.domain.membership.MembershipServiceAlgebra import vinyldns.api.domain.membership.MembershipServiceAlgebra
import vinyldns.api.domain.record.RecordSetServiceAlgebra import vinyldns.api.domain.record.RecordSetServiceAlgebra
@@ -35,6 +36,9 @@ import vinyldns.core.health.HealthService
import scala.util.matching.Regex import scala.util.matching.Regex
object VinylDNSService { object VinylDNSService {
import akka.http.scaladsl.server.Directives._
val ZoneIdRegex: Regex = "(?i)(/?zones/)(?:[0-9a-f]-?)+(.*)".r val ZoneIdRegex: Regex = "(?i)(/?zones/)(?:[0-9a-f]-?)+(.*)".r
val ZoneAndRecordIdRegex: Regex = val ZoneAndRecordIdRegex: Regex =
"(?i)(/?zones/)(?:[0-9a-f]-?)+(/recordsets/)(?:[0-9a-f]-?)+(.*)".r "(?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$ // $COVERAGE-OFF$
@@ -120,31 +142,25 @@ class VinylDNSService(
val batchChangeService: BatchChangeServiceAlgebra, val batchChangeService: BatchChangeServiceAlgebra,
val collectorRegistry: CollectorRegistry, val collectorRegistry: CollectorRegistry,
authPrincipalProvider: AuthPrincipalProvider) authPrincipalProvider: AuthPrincipalProvider)
extends VinylDNSDirectives extends PingRoute
with PingRoute
with ZoneRoute
with RecordSetRoute
with HealthCheckRoute with HealthCheckRoute
with BlueGreenRoute with BlueGreenRoute
with MembershipRoute
with StatusRoute with StatusRoute
with PrometheusRoute with PrometheusRoute
with BatchChangeRoute with VinylDNSJsonProtocol {
with VinylDNSJsonProtocol
with JsonValidationRejection { import VinylDNSService.validationRejectionHandler
val aws4Authenticator = new Aws4Authenticator val aws4Authenticator = new Aws4Authenticator
val vinylDNSAuthenticator: VinylDNSAuthenticator = val vinylDNSAuthenticator: VinylDNSAuthenticator =
new ProductionVinylDNSAuthenticator(aws4Authenticator, authPrincipalProvider) new ProductionVinylDNSAuthenticator(aws4Authenticator, authPrincipalProvider)
// Authenticated routes must go first val zoneRoute: Route = new ZoneRoute(zoneService, vinylDNSAuthenticator).getRoutes
def authenticatedRoutes: server.Route = val recordSetRoute: Route = new RecordSetRoute(recordSetService, vinylDNSAuthenticator).getRoutes
handleRejections(validationRejectionHandler)(authenticate { authPrincipal => val membershipRoute: Route =
batchChangeRoute(authPrincipal) ~ new MembershipRoute(membershipService, vinylDNSAuthenticator).getRoutes
zoneRoute(authPrincipal) ~ val batchChangeRoute: Route =
recordSetRoute(authPrincipal) ~ new BatchChangeRoute(batchChangeService, vinylDNSAuthenticator).getRoutes
membershipRoute(authPrincipal)
})
val unloggedUris = Seq( val unloggedUris = Seq(
Uri.Path("/health"), Uri.Path("/health"),
@@ -154,9 +170,16 @@ class VinylDNSService(
Uri.Path("/metrics/prometheus")) Uri.Path("/metrics/prometheus"))
val unloggedRoutes val unloggedRoutes
: Route = healthCheckRoute ~ pingRoute ~ colorRoute ~ statusRoute ~ prometheusRoute : Route = healthCheckRoute ~ pingRoute ~ colorRoute ~ statusRoute ~ prometheusRoute
val allRoutes: Route = unloggedRoutes ~
batchChangeRoute ~
zoneRoute ~
recordSetRoute ~
membershipRoute
val vinyldnsRoutes: Route = val vinyldnsRoutes: Route =
logRequestResult(VinylDNSService.buildLogEntry(unloggedUris))( logRequestResult(VinylDNSService.buildLogEntry(unloggedUris))(allRoutes)
unloggedRoutes ~ authenticatedRoutes) val routes: Route =
val routes = vinyldnsRoutes handleRejections(validationRejectionHandler)(allRoutes)
} }
// $COVERAGE-ON$ // $COVERAGE-ON$

View File

@@ -17,12 +17,10 @@
package vinyldns.api.route package vinyldns.api.route
import akka.http.scaladsl.model.StatusCodes 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 akka.util.Timeout
import vinyldns.api.Interfaces._
import vinyldns.api.crypto.Crypto import vinyldns.api.crypto.Crypto
import vinyldns.api.domain.zone._ import vinyldns.api.domain.zone._
import vinyldns.core.domain.auth.AuthPrincipal
import vinyldns.core.domain.zone._ import vinyldns.core.domain.zone._
import scala.concurrent.duration._ import scala.concurrent.duration._
@@ -30,10 +28,11 @@ import scala.concurrent.duration._
case class GetZoneResponse(zone: ZoneInfo) case class GetZoneResponse(zone: ZoneInfo)
case class ZoneRejected(zone: Zone, errors: List[String]) case class ZoneRejected(zone: Zone, errors: List[String])
trait ZoneRoute extends Directives { class ZoneRoute(zoneService: ZoneServiceAlgebra, val vinylDNSAuthenticator: VinylDNSAuthenticator)
this: VinylDNSJsonProtocol with VinylDNSDirectives with JsonValidationRejection => extends VinylDNSJsonProtocol
with VinylDNSDirectives[Throwable] {
val zoneService: ZoneServiceAlgebra def getRoutes: Route = zoneRoute
final private val DEFAULT_MAX_ITEMS: Int = 100 final private val DEFAULT_MAX_ITEMS: Int = 100
final private val MAX_ITEMS_LIMIT: 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 // Timeout must be long enough to allow the cluster to form
implicit val zoneCmdTimeout: Timeout = Timeout(10.seconds) implicit val zoneCmdTimeout: Timeout = Timeout(10.seconds)
val zoneRoute = { authPrincipal: AuthPrincipal => def handleErrors(errors: Throwable): PartialFunction[Throwable, Route] = {
(post & path("zones") & monitor("Endpoint.createZone")) { case ZoneAlreadyExistsError(msg) => complete(StatusCodes.Conflict, msg)
entity(as[CreateZoneInput]) { createZoneInput => case ConnectionFailed(_, msg) => complete(StatusCodes.BadRequest, msg)
execute(zoneService.connectToZone(encrypt(createZoneInput), authPrincipal)) { chg => case ZoneValidationFailed(zone, errorList, _) =>
complete(StatusCodes.Accepted, chg) 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( parameters(
"nameFilter".?, "nameFilter".?,
"startFrom".as[String].?, "startFrom".as[String].?,
@@ -65,79 +80,85 @@ trait ZoneRoute extends Directives {
validate( validate(
0 < maxItems && maxItems <= MAX_ITEMS_LIMIT, 0 < maxItems && maxItems <= MAX_ITEMS_LIMIT,
s"maxItems was $maxItems, maxItems must be between 0 and $MAX_ITEMS_LIMIT") { s"maxItems was $maxItems, maxItems must be between 0 and $MAX_ITEMS_LIMIT") {
execute(zoneService authenticateAndExecute(zoneService
.listZones(authPrincipal, nameFilter, startFrom, maxItems, ignoreAccess)) { .listZones(_, nameFilter, startFrom, maxItems, ignoreAccess)) { result =>
result => complete(StatusCodes.OK, 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) 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))
}
} ~
path("zones" / Segment) { id =>
(get & monitor("Endpoint.getZone")) {
authenticateAndExecute(zoneService.getZone(id, _)) { zone =>
complete(StatusCodes.OK, GetZoneResponse(zone)) complete(StatusCodes.OK, GetZoneResponse(zone))
} }
} ~ } ~
(get & path("zones" / "name" / Segment) & monitor("Endpoint.getZoneByName")) { zoneName => (put & monitor("Endpoint.updateZone")) {
execute(zoneService.getZoneByName(zoneName, authPrincipal)) { zone => authenticateAndExecuteWithEntity[ZoneCommandResult, UpdateZoneInput](
complete(StatusCodes.OK, GetZoneResponse(zone)) (authPrincipal, updateZoneInput) =>
} zoneService.updateZone(encrypt(updateZoneInput), authPrincipal)) { chg =>
} ~ complete(StatusCodes.Accepted, chg)
(delete & path("zones" / Segment) & monitor("Endpoint.deleteZone")) { id => }
execute(zoneService.deleteZone(id, authPrincipal)) { chg => } ~
complete(StatusCodes.Accepted, chg) (delete & monitor("Endpoint.deleteZone")) {
} authenticateAndExecute(zoneService.deleteZone(id, _)) { chg =>
} ~
(put & path("zones" / Segment) & monitor("Endpoint.updateZone")) { _ =>
entity(as[UpdateZoneInput]) { updateZoneInput =>
execute(zoneService.updateZone(encrypt(updateZoneInput), authPrincipal)) { chg =>
complete(StatusCodes.Accepted, chg) complete(StatusCodes.Accepted, chg)
} }
} }
} ~ } ~
(post & path("zones" / Segment / "sync") & monitor("Endpoint.syncZone")) { id => path("zones" / Segment / "sync") { id =>
execute(zoneService.syncZone(id, authPrincipal)) { chg => (post & monitor("Endpoint.syncZone")) {
authenticateAndExecute(zoneService.syncZone(id, _)) { chg =>
complete(StatusCodes.Accepted, 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)) { parameters("startFrom".?, "maxItems".as[Int].?(DEFAULT_MAX_ITEMS)) {
(startFrom: Option[String], maxItems: Int) => (startFrom: Option[String], maxItems: Int) =>
handleRejections(invalidQueryHandler) { handleRejections(invalidQueryHandler) {
validate( validate(
0 < maxItems && maxItems <= DEFAULT_MAX_ITEMS, 0 < maxItems && maxItems <= DEFAULT_MAX_ITEMS,
s"maxItems was $maxItems, maxItems must be between 0 exclusive and $DEFAULT_MAX_ITEMS inclusive") { 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 => changes =>
complete(StatusCodes.OK, changes) complete(StatusCodes.OK, changes)
} }
} }
} }
} }
}
} ~
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)
}
} ~ } ~
(put & path("zones" / Segment / "acl" / "rules") & monitor("Endpoint.addZoneACLRule")) { id => (delete & monitor("Endpoint.deleteZoneACLRule")) {
entity(as[ACLRuleInfo]) { rule => authenticateAndExecuteWithEntity[ZoneCommandResult, ACLRuleInfo]((authPrincipal, rule) =>
execute(zoneService.addACLRule(id, rule, authPrincipal)) { chg => zoneService.deleteACLRule(id, rule, authPrincipal)) { chg =>
complete(StatusCodes.Accepted, 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 =>
complete(StatusCodes.Accepted, chg)
}
}
}
}
/** /**
* Important! Will encrypt the key on the zone if a connection is present * Important! Will encrypt the key on the zone if a connection is present
@@ -164,23 +185,4 @@ trait ZoneRoute extends Directives {
complete(StatusCodes.BadRequest, msg) complete(StatusCodes.BadRequest, msg)
} }
.result() .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.server.Route
import akka.http.scaladsl.testkit.ScalatestRouteTest import akka.http.scaladsl.testkit.ScalatestRouteTest
import cats.data.EitherT import cats.data.EitherT
import cats.effect._
import cats.implicits._ import cats.implicits._
import org.joda.time.DateTime import org.joda.time.DateTime
import org.json4s.JsonDSL._ import org.json4s.JsonDSL._
import org.json4s._ import org.json4s._
import org.json4s.jackson.JsonMethods._ 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.api.domain.batch._
import vinyldns.core.TestMembershipData._ 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.BatchChangeIsEmpty
import vinyldns.core.domain.auth.AuthPrincipal import vinyldns.core.domain.auth.AuthPrincipal
import vinyldns.core.domain.batch.BatchChangeApprovalStatus.BatchChangeApprovalStatus import vinyldns.core.domain.batch.BatchChangeApprovalStatus.BatchChangeApprovalStatus
import vinyldns.core.domain.batch._ import vinyldns.core.domain.batch._
import vinyldns.core.domain.record.RecordType._
import vinyldns.core.domain.record._
class BatchChangeRoutingSpec class BatchChangeRoutingSpec()
extends WordSpec extends WordSpec
with ScalatestRouteTest with ScalatestRouteTest
with BatchChangeRoute with MockitoSugar
with JsonValidationRejection
with VinylDNSDirectives
with VinylDNSJsonProtocol with VinylDNSJsonProtocol
with Matchers { with VinylDNSRouteTestHelper
with Matchers
with BeforeAndAfterEach {
val batchChangeService: BatchChangeServiceAlgebra = TestBatchChangeService 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._ 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) } else if (auth.userId == superUserAuth.userId)
(auth, startFrom, maxItems, ignoreAccess, approvalStatus) match { (auth, startFrom, maxItems, ignoreAccess, approvalStatus) match {
case (_, None, 100, true, None) => case (_, None, 100, true, None) =>
@@ -351,7 +363,7 @@ class BatchChangeRoutingSpec
) )
) )
case (_) => EitherT.rightT(BatchChangeSummaryList(List())) case _ => EitherT.rightT(BatchChangeSummaryList(List()))
} else } else
EitherT.rightT(BatchChangeSummaryList(List())) EitherT.rightT(BatchChangeSummaryList(List()))
@@ -364,7 +376,7 @@ class BatchChangeRoutingSpec
case ("pendingBatchId", true) => EitherT(IO.pure(genericValidResponse.asRight)) case ("pendingBatchId", true) => EitherT(IO.pure(genericValidResponse.asRight))
case ("pendingBatchId", false) => case ("pendingBatchId", false) =>
EitherT(IO.pure(UserNotAuthorizedError("notAuthedID").asLeft)) EitherT(IO.pure(UserNotAuthorizedError("notAuthedID").asLeft))
case (_) => EitherT(IO.pure(BatchChangeNotPendingApproval("batchId").asLeft)) case _ => EitherT(IO.pure(BatchChangeNotPendingApproval("batchId").asLeft))
} }
def approveBatchChange( def approveBatchChange(
@@ -389,7 +401,7 @@ class BatchChangeRoutingSpec
Post("/zones/batchrecordchanges").withEntity( Post("/zones/batchrecordchanges").withEntity(
HttpEntity(ContentTypes.`application/json`, validRequestWithComments)) ~> HttpEntity(ContentTypes.`application/json`, validRequestWithComments)) ~>
batchChangeRoute(okAuth) ~> check { batchChangeRoute ~> check {
status shouldBe Accepted status shouldBe Accepted
@@ -403,7 +415,7 @@ class BatchChangeRoutingSpec
Post("/zones/batchrecordchanges").withEntity( Post("/zones/batchrecordchanges").withEntity(
HttpEntity(ContentTypes.`application/json`, validRequestWithoutComments)) ~> HttpEntity(ContentTypes.`application/json`, validRequestWithoutComments)) ~>
batchChangeRoute(okAuth) ~> check { batchChangeRoute ~> check {
status shouldBe Accepted status shouldBe Accepted
@@ -420,7 +432,7 @@ class BatchChangeRoutingSpec
Post("/zones/batchrecordchanges").withEntity( Post("/zones/batchrecordchanges").withEntity(
HttpEntity(ContentTypes.`application/json`, validRequestWithOwnerGroupId)) ~> HttpEntity(ContentTypes.`application/json`, validRequestWithOwnerGroupId)) ~>
batchChangeRoute(sharedAuth) ~> check { batchChangeRoute ~> check {
status shouldBe Accepted status shouldBe Accepted
val change = responseAs[JValue] val change = responseAs[JValue]
@@ -430,10 +442,9 @@ class BatchChangeRoutingSpec
"return a 202 Accepted for valid add and delete request with allowManualReview parameter" in { "return a 202 Accepted for valid add and delete request with allowManualReview parameter" in {
val validRequestWithoutComments: String = compact(render(changeList)) val validRequestWithoutComments: String = compact(render(changeList))
Post("/zones/batchrecordchanges?allowManualReview=false").withEntity( Post("/zones/batchrecordchanges?allowManualReview=false").withEntity(
HttpEntity(ContentTypes.`application/json`, validRequestWithoutComments)) ~> HttpEntity(ContentTypes.`application/json`, validRequestWithoutComments)) ~>
batchChangeRoute(okAuth) ~> check { batchChangeRoute ~> check {
status shouldBe Accepted status shouldBe Accepted
@@ -453,7 +464,7 @@ class BatchChangeRoutingSpec
Post("/zones/batchrecordchanges").withEntity( Post("/zones/batchrecordchanges").withEntity(
HttpEntity(ContentTypes.`application/json`, validRequestWithScheduledTime)) ~> HttpEntity(ContentTypes.`application/json`, validRequestWithScheduledTime)) ~>
batchChangeRoute(okAuth) ~> check { batchChangeRoute ~> check {
status shouldBe Accepted status shouldBe Accepted
@@ -470,7 +481,7 @@ class BatchChangeRoutingSpec
Post("/zones/batchrecordchanges").withEntity( Post("/zones/batchrecordchanges").withEntity(
HttpEntity(ContentTypes.`application/json`, emptyBatchRequest)) ~> HttpEntity(ContentTypes.`application/json`, emptyBatchRequest)) ~>
Route.seal(batchChangeRoute(okAuth)) ~> check { Route.seal(batchChangeRoute) ~> check {
status shouldBe BadRequest status shouldBe BadRequest
} }
@@ -486,7 +497,7 @@ class BatchChangeRoutingSpec
Post("/zones/batchrecordchanges").withEntity( Post("/zones/batchrecordchanges").withEntity(
HttpEntity(ContentTypes.`application/json`, invalidRequestChangeType)) ~> HttpEntity(ContentTypes.`application/json`, invalidRequestChangeType)) ~>
Route.seal(batchChangeRoute(okAuth)) ~> check { Route.seal(batchChangeRoute) ~> check {
status shouldBe BadRequest status shouldBe BadRequest
} }
@@ -497,7 +508,7 @@ class BatchChangeRoutingSpec
Post("/zones/batchrecordchanges").withEntity( Post("/zones/batchrecordchanges").withEntity(
HttpEntity(ContentTypes.`application/json`, runtimeError)) ~> HttpEntity(ContentTypes.`application/json`, runtimeError)) ~>
Route.seal(batchChangeRoute(okAuth)) ~> check { Route.seal(batchChangeRoute) ~> check {
status shouldBe InternalServerError status shouldBe InternalServerError
} }
@@ -506,7 +517,7 @@ class BatchChangeRoutingSpec
"GET Batch Change Info" should { "GET Batch Change Info" should {
"return the batch change info given a valid batch change id" in { "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 status shouldBe OK
@@ -517,7 +528,7 @@ class BatchChangeRoutingSpec
} }
"maintain backwards compatability for zoneName/recordName/zoneId" in { "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 status shouldBe OK
@@ -531,14 +542,14 @@ class BatchChangeRoutingSpec
} }
"return a NotFound error given a nonexistent batch change id" in { "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 status shouldBe NotFound
} }
} }
"return a Forbidden error if user did not create the batch change" in { "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 status shouldBe Forbidden
} }
@@ -547,7 +558,7 @@ class BatchChangeRoutingSpec
"GET batchChangesSummaries" should { "GET batchChangesSummaries" should {
"return the list of batch change summaries for the user that called it" in { "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 status shouldBe OK
val resp = responseAs[BatchChangeSummaryList] val resp = responseAs[BatchChangeSummaryList]
@@ -560,7 +571,7 @@ class BatchChangeRoutingSpec
} }
"return the first batch change summary for the user that called it" in { "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 status shouldBe OK
val resp = responseAs[BatchChangeSummaryList] 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 { "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 status shouldBe OK
val resp = responseAs[BatchChangeSummaryList] val resp = responseAs[BatchChangeSummaryList]
@@ -586,7 +597,7 @@ class BatchChangeRoutingSpec
} }
"return only the second batch change summary for the user that called it" in { "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 status shouldBe OK
val resp = responseAs[BatchChangeSummaryList] val resp = responseAs[BatchChangeSummaryList]
@@ -600,7 +611,7 @@ class BatchChangeRoutingSpec
"return user's Pending batch changes if approval status is `PendingApproval`" in { "return user's Pending batch changes if approval status is `PendingApproval`" in {
Get("/zones/batchrecordchanges?approvalStatus=pendingapproval") ~> Get("/zones/batchrecordchanges?approvalStatus=pendingapproval") ~>
batchChangeRoute(okAuth) ~> check { batchChangeRoute ~> check {
status shouldBe OK status shouldBe OK
val resp = responseAs[BatchChangeSummaryList] val resp = responseAs[BatchChangeSummaryList]
@@ -614,7 +625,7 @@ class BatchChangeRoutingSpec
} }
"return an error if maxItems is out of range" in { "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 status shouldBe BadRequest
responseEntity.toString should include( responseEntity.toString should include(
@@ -623,7 +634,8 @@ class BatchChangeRoutingSpec
} }
"return empty list of batch change summaries for the user that called it" in { "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 status shouldBe OK
val resp = responseAs[JValue] 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 { "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 status shouldBe OK
val resp = responseAs[BatchChangeSummaryList] val resp = responseAs[BatchChangeSummaryList]
@@ -647,8 +660,9 @@ class BatchChangeRoutingSpec
"return all Pending batch changes if ignoreAccess is true, approval status is `PendingApproval`," + "return all Pending batch changes if ignoreAccess is true, approval status is `PendingApproval`," +
" and requester is a super user" in { " and requester is a super user" in {
batchChangeRoute = superUserRoute.getRoutes
Get("/zones/batchrecordchanges?ignoreAccess=true&approvalStatus=PendingApproval") ~> Get("/zones/batchrecordchanges?ignoreAccess=true&approvalStatus=PendingApproval") ~>
batchChangeRoute(superUserAuth) ~> check { batchChangeRoute ~> check {
status shouldBe OK status shouldBe OK
val resp = responseAs[BatchChangeSummaryList] val resp = responseAs[BatchChangeSummaryList]
@@ -665,18 +679,20 @@ class BatchChangeRoutingSpec
"POST reject batch change" should { "POST reject batch change" should {
"return OK if review comment is provided, batch change is PendingApproval, and reviewer is authorized" in { "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( Post("/zones/batchrecordchanges/pendingBatchId/reject").withEntity(HttpEntity(
ContentTypes.`application/json`, ContentTypes.`application/json`,
compact(render("comments" -> "some comments")))) ~> compact(render("comments" -> "some comments")))) ~>
batchChangeRoute(supportUserAuth) ~> check { batchChangeRoute ~> check {
status shouldBe OK status shouldBe OK
} }
} }
"return OK if comments are not provided, batch change is PendingApproval, and reviewer is authorized" in { "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( Post("/zones/batchrecordchanges/pendingBatchId/reject").withEntity(
HttpEntity(ContentTypes.`application/json`, compact(render("")))) ~> HttpEntity(ContentTypes.`application/json`, compact(render("")))) ~>
batchChangeRoute(supportUserAuth) ~> check { batchChangeRoute ~> check {
status shouldBe OK status shouldBe OK
} }
} }
@@ -684,7 +700,7 @@ class BatchChangeRoutingSpec
"return Forbidden if user is not a super or support admin" in { "return Forbidden if user is not a super or support admin" in {
Post("/zones/batchrecordchanges/pendingBatchId/reject").withEntity( Post("/zones/batchrecordchanges/pendingBatchId/reject").withEntity(
HttpEntity(ContentTypes.`application/json`, compact(render("")))) ~> HttpEntity(ContentTypes.`application/json`, compact(render("")))) ~>
batchChangeRoute(okAuth) ~> check { batchChangeRoute ~> check {
status shouldBe Forbidden status shouldBe Forbidden
} }
} }
@@ -692,7 +708,7 @@ class BatchChangeRoutingSpec
"return BadRequest if comments exceed 1024 characters" in { "return BadRequest if comments exceed 1024 characters" in {
Post("/zones/batchrecordchanges/pendingBatchId/reject").withEntity(HttpEntity( Post("/zones/batchrecordchanges/pendingBatchId/reject").withEntity(HttpEntity(
ContentTypes.`application/json`, 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 status shouldBe BadRequest
responseEntity.toString should include("Comment length must not exceed 1024 characters.") 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 { "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 status shouldBe OK
} }
} }
"return BadRequest if batch change is not pending approval" in { "return BadRequest if batch change is not pending approval" in {
batchChangeRoute = supportUserRoute.getRoutes
Post("/zones/batchrecordchanges/batchId/reject").withEntity( Post("/zones/batchrecordchanges/batchId/reject").withEntity(
HttpEntity(ContentTypes.`application/json`, compact(render("")))) ~> HttpEntity(ContentTypes.`application/json`, compact(render("")))) ~>
batchChangeRoute(supportUserAuth) ~> check { batchChangeRoute ~> check {
status shouldBe BadRequest status shouldBe BadRequest
} }
} }
@@ -716,18 +734,20 @@ class BatchChangeRoutingSpec
"POST approve batch change" should { "POST approve batch change" should {
"return OK if review comment is provided, batch change is PendingApproval, and reviewer is authorized" in { "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( Post("/zones/batchrecordchanges/pendingBatchId/approve").withEntity(HttpEntity(
ContentTypes.`application/json`, ContentTypes.`application/json`,
compact(render("comments" -> "some comments")))) ~> compact(render("comments" -> "some comments")))) ~>
batchChangeRoute(supportUserAuth) ~> check { batchChangeRoute ~> check {
status shouldBe OK status shouldBe OK
} }
} }
"return OK if comments are not provided, batch change is PendingApproval, and reviewer is authorized" in { "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( Post("/zones/batchrecordchanges/pendingBatchId/approve").withEntity(
HttpEntity(ContentTypes.`application/json`, compact(render("")))) ~> HttpEntity(ContentTypes.`application/json`, compact(render("")))) ~>
batchChangeRoute(supportUserAuth) ~> check { batchChangeRoute ~> check {
status shouldBe OK status shouldBe OK
} }
} }
@@ -735,40 +755,44 @@ class BatchChangeRoutingSpec
"return Forbidden if user is not a super or support admin" in { "return Forbidden if user is not a super or support admin" in {
Post("/zones/batchrecordchanges/pendingBatchId/approve").withEntity( Post("/zones/batchrecordchanges/pendingBatchId/approve").withEntity(
HttpEntity(ContentTypes.`application/json`, compact(render("")))) ~> HttpEntity(ContentTypes.`application/json`, compact(render("")))) ~>
batchChangeRoute(okAuth) ~> check { batchChangeRoute ~> check {
status shouldBe Forbidden status shouldBe Forbidden
} }
} }
"return BadRequest if comments exceed 1024 characters" in { "return BadRequest if comments exceed 1024 characters" in {
batchChangeRoute = supportUserRoute.getRoutes
Post("/zones/batchrecordchanges/pendingBatchId/approve").withEntity(HttpEntity( Post("/zones/batchrecordchanges/pendingBatchId/approve").withEntity(HttpEntity(
ContentTypes.`application/json`, 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 status shouldBe BadRequest
responseEntity.toString should include("Comment length must not exceed 1024 characters.") responseEntity.toString should include("Comment length must not exceed 1024 characters.")
} }
} }
"return OK if no request entity is provided" in { "return OK no request entity is provided" in {
Post("/zones/batchrecordchanges/pendingBatchId/approve") ~> batchChangeRoute(supportUserAuth) ~> check { batchChangeRoute = supportUserRoute.getRoutes
Post("/zones/batchrecordchanges/pendingBatchId/approve") ~> batchChangeRoute ~> check {
status shouldBe OK status shouldBe OK
} }
} }
"return BadRequest if batch change is not pending approval" in { "return BadRequest if batch change is not pending approval" in {
batchChangeRoute = supportUserRoute.getRoutes
Post("/zones/batchrecordchanges/batchId/approve").withEntity( Post("/zones/batchrecordchanges/batchId/approve").withEntity(
HttpEntity(ContentTypes.`application/json`, compact(render("")))) ~> HttpEntity(ContentTypes.`application/json`, compact(render("")))) ~>
batchChangeRoute(supportUserAuth) ~> check { batchChangeRoute ~> check {
status shouldBe BadRequest status shouldBe BadRequest
} }
} }
"return NotFound if the requesting user cant be found" in { "return NotFound if the requesting user cant be found" in {
batchChangeRoute = supportUserRoute.getRoutes
Post("/zones/batchrecordchanges/notFoundUser/approve").withEntity(HttpEntity( Post("/zones/batchrecordchanges/notFoundUser/approve").withEntity(HttpEntity(
ContentTypes.`application/json`, ContentTypes.`application/json`,
compact(render("comments" -> "some comments")))) ~> compact(render("comments" -> "some comments")))) ~>
batchChangeRoute(supportUserAuth) ~> check { batchChangeRoute ~> check {
status shouldBe NotFound status shouldBe NotFound
} }
} }

View File

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

View File

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

View File

@@ -26,7 +26,7 @@ import org.mockito.Matchers._
import org.mockito.Mockito._ import org.mockito.Mockito._
import org.scalatest.mockito.MockitoSugar import org.scalatest.mockito.MockitoSugar
import org.scalatest.{BeforeAndAfterEach, Matchers, OneInstancePerTest, WordSpec} 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 vinyldns.core.route.Monitor
import scala.util.Failure import scala.util.Failure
@@ -37,15 +37,26 @@ class VinylDNSDirectivesSpec
with Matchers with Matchers
with MockitoSugar with MockitoSugar
with OneInstancePerTest with OneInstancePerTest
with VinylDNSDirectives with VinylDNSDirectives[Throwable]
with Directives with Directives
with VinylDNSJsonProtocol
with BeforeAndAfterEach { with BeforeAndAfterEach {
private val mockLatency = mock[Histogram] private val mockLatency = mock[Histogram]
private val mockErrors = mock[Meter] private val mockErrors = mock[Meter]
val vinylDNSAuthenticator: VinylDNSAuthenticator = new TestVinylDNSAuthenticator( def getRoutes: Route = zoneRoute
mock[AuthPrincipal])
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") { class TestMonitor extends Monitor("test") {
override val latency: Histogram = mockLatency 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 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.mockito.MockitoSugar
import org.scalatest.{Matchers, OneInstancePerTest, WordSpec} 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 import vinyldns.core.domain.auth.AuthPrincipal
class VinylDNSServiceSpec class VinylDNSServiceSpec
@@ -29,11 +30,17 @@ class VinylDNSServiceSpec
with Matchers with Matchers
with MockitoSugar with MockitoSugar
with OneInstancePerTest with OneInstancePerTest
with VinylDNSDirectives { with VinylDNSDirectives[Throwable] {
val vinylDNSAuthenticator: VinylDNSAuthenticator = new TestVinylDNSAuthenticator( val vinylDNSAuthenticator: VinylDNSAuthenticator = new TestVinylDNSAuthenticator(
mock[AuthPrincipal]) mock[AuthPrincipal])
def getRoutes: Route = mock[Route]
def handleErrors(e: Throwable): PartialFunction[Throwable, Route] = {
case _ => complete(StatusCodes.InternalServerError)
}
private def buildMockRequest( private def buildMockRequest(
path: String = "/path/to/resource", path: String = "/path/to/resource",
body: String = "request body") = { body: String = "request body") = {

View File

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

View File

@@ -77,27 +77,3 @@ case class Group(
memberIds = memberIdsUpdate ++ adminUserIdsUpdate, memberIds = memberIdsUpdate ++ adminUserIdsUpdate,
adminUserIds = 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> <parameter name="illegalImports"><![CDATA[sun._,java.awt._]]></parameter>
</parameters> </parameters>
</check> </check>
<check level="error" class="org.scalastyle.scalariform.NoWhitespaceBeforeLeftBracketChecker" enabled="true"></check> <check level="warning" 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.NoWhitespaceAfterLeftBracketChecker" enabled="true"></check>
<check level="warning" class="org.scalastyle.scalariform.ReturnChecker" 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="warning" class="org.scalastyle.scalariform.NullChecker" enabled="true"></check>
<check level="error" class="org.scalastyle.scalariform.NoCloneChecker" enabled="true"></check> <check level="error" class="org.scalastyle.scalariform.NoCloneChecker" enabled="true"></check>