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

Replace the repos in the portal with dynamodb and core (#206)

Replace the repos in the portal with dynamodb and core

* Remove all data stores from the portal
* Use the user and user change repository from core and dynamodb
* Remove the UserAccount type, use core User instead
* Remove the UserChangeLog types, use core UserChange instead
* Clean up duplication in VinylDNS
* Moved `Module` to `modules.VinylDNSModule`.  The reason is that
you cannot disable the "default" module for unit tests.
* Use mock configuration for VinylDNSSpec and FrontendControllerSpec.
The mock app configuration is what allows us to run without dynamodb
* Added a TestApplicationData trait to cut down on duplication
This commit is contained in:
Paul Cleary
2018-09-18 11:51:31 -04:00
committed by GitHub
parent b9bc20870e
commit 0f2fdc9c7b
33 changed files with 744 additions and 2012 deletions

View File

@@ -73,7 +73,7 @@ lazy val testSettings = Seq(
parallelExecution in Test := false, parallelExecution in Test := false,
parallelExecution in IntegrationTest := false, parallelExecution in IntegrationTest := false,
fork in IntegrationTest := false, fork in IntegrationTest := false,
testOptions in Test += Tests.Argument("-oD"), testOptions in Test += Tests.Argument("-oDNCXEHPQRMIK"),
logBuffered in Test := false logBuffered in Test := false
) )
@@ -315,7 +315,7 @@ lazy val portal = (project in file("modules/portal")).enablePlugins(PlayScala, A
// change the name of the output to portal.zip // change the name of the output to portal.zip
packageName in Universal := "portal" packageName in Universal := "portal"
) )
.dependsOn(core) .dependsOn(core, dynamodb)
lazy val docSettings = Seq( lazy val docSettings = Seq(
git.remoteRepo := "https://github.com/vinyldns/vinyldns", git.remoteRepo := "https://github.com/vinyldns/vinyldns",

View File

@@ -18,6 +18,7 @@ package vinyldns.core.domain.membership
import java.util.UUID import java.util.UUID
import org.apache.commons.lang3.RandomStringUtils
import org.joda.time.DateTime import org.joda.time.DateTime
import vinyldns.core.domain.membership.LockStatus.LockStatus import vinyldns.core.domain.membership.LockStatus.LockStatus
@@ -26,7 +27,7 @@ object LockStatus extends Enumeration {
val Locked, Unlocked = Value val Locked, Unlocked = Value
} }
case class User( final case class User(
userName: String, userName: String,
accessKey: String, accessKey: String,
secretKey: String, secretKey: String,
@@ -41,4 +42,11 @@ case class User(
def updateUserLockStatus(lockStatus: LockStatus): User = def updateUserLockStatus(lockStatus: LockStatus): User =
this.copy(lockStatus = lockStatus) this.copy(lockStatus = lockStatus)
def regenerateCredentials(): User =
copy(accessKey = User.generateKey, secretKey = User.generateKey)
}
object User {
def generateKey: String = RandomStringUtils.randomAlphanumeric(20)
} }

View File

@@ -15,6 +15,8 @@
*/ */
package vinyldns.core.domain.membership package vinyldns.core.domain.membership
import java.util.UUID
import org.joda.time.DateTime import org.joda.time.DateTime
sealed abstract class UserChangeType(val value: String) sealed abstract class UserChangeType(val value: String)
@@ -48,14 +50,18 @@ sealed trait UserChange {
def created: DateTime def created: DateTime
} }
object UserChange { object UserChange {
final case class CreateUser(id: String, newUser: User, madeByUserId: String, created: DateTime) final case class CreateUser(
extends UserChange
final case class UpdateUser(
id: String,
newUser: User, newUser: User,
madeByUserId: String, madeByUserId: String,
created: DateTime, created: DateTime,
oldUser: User) id: String = UUID.randomUUID().toString)
extends UserChange
final case class UpdateUser(
newUser: User,
madeByUserId: String,
created: DateTime,
oldUser: User,
id: String = UUID.randomUUID().toString)
extends UserChange extends UserChange
def apply( def apply(
@@ -67,10 +73,10 @@ object UserChange {
changeType: UserChangeType): Either[IllegalArgumentException, UserChange] = changeType: UserChangeType): Either[IllegalArgumentException, UserChange] =
changeType match { changeType match {
case UserChangeType.Create => case UserChangeType.Create =>
Right(CreateUser(id, newUser, madeByUserId, created)) Right(CreateUser(newUser, madeByUserId, created, id))
case UserChangeType.Update => case UserChangeType.Update =>
oldUser oldUser
.map(u => Right(UpdateUser(id, newUser, madeByUserId, created, u))) .map(u => Right(UpdateUser(newUser, madeByUserId, created, u, id)))
.getOrElse(Left(new IllegalArgumentException( .getOrElse(Left(new IllegalArgumentException(
s"Unable to create update user change, old user is not defined"))) s"Unable to create update user change, old user is not defined")))
} }

View File

@@ -31,5 +31,7 @@ trait UserRepository extends Repository {
def getUserByAccessKey(accessKey: String): IO[Option[User]] def getUserByAccessKey(accessKey: String): IO[Option[User]]
def getUserByName(userName: String): IO[Option[User]]
def save(user: User): IO[User] def save(user: User): IO[User]
} }

View File

@@ -27,12 +27,12 @@ class UserChangeSpec extends WordSpec with Matchers with EitherMatchers with Eit
"apply" should { "apply" should {
"succeed for CreateUser" in { "succeed for CreateUser" in {
val result = UserChange("foo", newUser, "bar", currentDate, None, UserChangeType.Create) val result = UserChange("foo", newUser, "bar", currentDate, None, UserChangeType.Create)
result shouldBe Right(UserChange.CreateUser("foo", newUser, "bar", currentDate)) result shouldBe Right(UserChange.CreateUser(newUser, "bar", currentDate, "foo"))
} }
"succeed for UpdateUser" in { "succeed for UpdateUser" in {
val result = val result =
UserChange("foo", newUser, "bar", currentDate, Some(newUser), UserChangeType.Update) UserChange("foo", newUser, "bar", currentDate, Some(newUser), UserChangeType.Update)
result shouldBe Right(UserChange.UpdateUser("foo", newUser, "bar", currentDate, newUser)) result shouldBe Right(UserChange.UpdateUser(newUser, "bar", currentDate, newUser, "foo"))
} }
"fail for invalid parameters" in { "fail for invalid parameters" in {
val result = UserChange("foo", newUser, "bar", currentDate, None, UserChangeType.Update) val result = UserChange("foo", newUser, "bar", currentDate, None, UserChangeType.Update)

View File

@@ -54,7 +54,7 @@ class DynamoDBUserChangeRepositoryIntegrationSpec extends DynamoDBIntegrationSpe
"DynamoDBUserChangeRepository" should { "DynamoDBUserChangeRepository" should {
"save a user change" in { "save a user change" in {
val auth = AuthPrincipal(testUser, Seq.empty) val auth = AuthPrincipal(testUser, Seq.empty)
val c = UserChange.CreateUser("foo", testUser, auth.userId, DateTime.now) val c = UserChange.CreateUser(testUser, auth.userId, DateTime.now, "foo")
val t = for { val t = for {
_ <- repo.save(c) _ <- repo.save(c)
@@ -67,7 +67,7 @@ class DynamoDBUserChangeRepositoryIntegrationSpec extends DynamoDBIntegrationSpe
"save a change for a modified user" in { "save a change for a modified user" in {
val auth = AuthPrincipal(testUser, Seq.empty) val auth = AuthPrincipal(testUser, Seq.empty)
val updated = testUser.copy(userName = testUser.userName + "-updated") val updated = testUser.copy(userName = testUser.userName + "-updated")
val c = UserChange.UpdateUser("foo", updated, auth.userId, DateTime.now, testUser) val c = UserChange.UpdateUser(updated, auth.userId, DateTime.now, testUser, "foo")
val t = for { val t = for {
_ <- repo.save(c) _ <- repo.save(c)

View File

@@ -72,7 +72,7 @@ object DynamoDBUserChangeRepository {
new AttributeValue().withM(DynamoDBUserRepository.toItem(crypto, change.newUser))) new AttributeValue().withM(DynamoDBUserRepository.toItem(crypto, change.newUser)))
change match { change match {
case UserChange.UpdateUser(_, _, _, _, oldUser) => case UserChange.UpdateUser(_, _, _, oldUser, _) =>
item.put( item.put(
OLD_USER, OLD_USER,
new AttributeValue().withM(DynamoDBUserRepository.toItem(crypto, oldUser))) new AttributeValue().withM(DynamoDBUserRepository.toItem(crypto, oldUser)))

View File

@@ -161,6 +161,30 @@ class DynamoDBUserRepository private[repository] (
.value .value
} }
def getUserByName(username: String): IO[Option[User]] = {
val attributeNames = new util.HashMap[String, String]()
attributeNames.put("#uname", USER_NAME)
val attributeValues = new util.HashMap[String, AttributeValue]()
attributeValues.put(":uname", new AttributeValue().withS(username))
val request = new QueryRequest()
.withTableName(userTableName)
.withKeyConditionExpression("#uname = :uname")
.withExpressionAttributeNames(attributeNames)
.withExpressionAttributeValues(attributeValues)
.withIndexName(USER_NAME_INDEX_NAME)
// the question is what to do with duplicate usernames, in the portal we just log loudly, staying the same here
dynamoDBHelper.query(request).flatMap { result =>
result.getItems.asScala.toList match {
case x :: Nil => fromItem(x).map(Some(_))
case Nil => IO.pure(None)
case x :: _ =>
log.error(s"Inconsistent data, multiple user records found for user name '$username'")
fromItem(x).map(Some(_))
}
}
}
def getUsers( def getUsers(
userIds: Set[String], userIds: Set[String],
exclusiveStartKey: Option[String], exclusiveStartKey: Option[String],

View File

@@ -1,92 +0,0 @@
/*
* 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.
*/
import com.amazonaws.auth.{AWSStaticCredentialsProvider, BasicAWSCredentials}
import com.amazonaws.client.builder.AwsClientBuilder.EndpointConfiguration
import com.amazonaws.services.dynamodbv2.{AmazonDynamoDBClient, AmazonDynamoDBClientBuilder}
import com.google.inject.AbstractModule
import controllers._
import controllers.datastores.{
DynamoDBChangeLogStore,
DynamoDBUserAccountStore,
InMemoryChangeLogStore,
InMemoryUserAccountStore
}
import play.api.{Configuration, Environment}
import vinyldns.core.crypto.CryptoAlgebra
class Module(environment: Environment, configuration: Configuration) extends AbstractModule {
val settings = new Settings(configuration)
def configure(): Unit = {
val crypto = CryptoAlgebra.load(configuration.underlying.getConfig("crypto")).unsafeRunSync()
bind(classOf[Authenticator]).toInstance(authenticator())
bind(classOf[UserAccountStore]).toInstance(userAccountStore(crypto))
bind(classOf[ChangeLogStore]).toInstance(changeLogStore(crypto))
}
private def authenticator(): Authenticator =
/**
* Why not load config here you ask? Well, there is some ugliness in the LdapAuthenticator
* that I am not looking to undo at this time. There are private classes
* that do some wrapping. It all seems to work, so I am leaving it alone
* to complete the Play framework upgrade
*/
LdapAuthenticator(settings)
private def userAccountStore(crypto: CryptoAlgebra) = {
val useDummy = configuration.get[Boolean]("users.dummy")
if (useDummy)
new InMemoryUserAccountStore
else {
// Important! For some reason the basic credentials get lost in Jenkins. Set the aws system properties
// just in case
val dynamoAKID = configuration.get[String]("dynamo.key")
val dynamoSecret = configuration.get[String]("dynamo.secret")
val dynamoEndpoint = configuration.get[String]("dynamo.endpoint")
val region = configuration.get[String]("dynamo.region")
val credentials = new BasicAWSCredentials(dynamoAKID, dynamoSecret)
val dynamoClient = AmazonDynamoDBClientBuilder
.standard()
.withCredentials(new AWSStaticCredentialsProvider(credentials))
.withEndpointConfiguration(new EndpointConfiguration(dynamoEndpoint, region))
.build()
.asInstanceOf[AmazonDynamoDBClient]
new DynamoDBUserAccountStore(dynamoClient, configuration, crypto)
}
}
private def changeLogStore(crypto: CryptoAlgebra) = {
val useDummy = configuration.get[Boolean]("changelog.dummy")
if (useDummy)
new InMemoryChangeLogStore
else {
val dynamoAKID = configuration.get[String]("dynamo.key")
val dynamoSecret = configuration.get[String]("dynamo.secret")
val dynamoEndpoint = configuration.get[String]("dynamo.endpoint")
val region = configuration.get[String]("dynamo.region")
val credentials = new BasicAWSCredentials(dynamoAKID, dynamoSecret)
val dynamoClient = AmazonDynamoDBClientBuilder
.standard()
.withCredentials(new AWSStaticCredentialsProvider(credentials))
.withEndpointConfiguration(new EndpointConfiguration(dynamoEndpoint, region))
.build()
.asInstanceOf[AmazonDynamoDBClient]
new DynamoDBChangeLogStore(dynamoClient, configuration, crypto)
}
}
}

View File

@@ -1,59 +0,0 @@
/*
* 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 controllers
import models.UserAccount
import org.joda.time.DateTime
import scala.util.Try
// $COVERAGE-OFF$
object ChangeType {
def apply(s: String): ChangeType =
s.toLowerCase match {
case "created" => Create
case "updated" => Update
case "deleted" => Delete
case _ => throw new IllegalArgumentException(s"$s is not a valid change type")
}
}
sealed trait ChangeType
case object Create extends ChangeType {
override val toString = "created"
}
case object Update extends ChangeType {
override val toString = "updated"
}
case object Delete extends ChangeType {
override val toString = "deleted"
}
sealed trait ChangeLogMessage
final case class UserChangeMessage(
userId: String,
username: String,
timeStamp: DateTime,
changeType: ChangeType,
updatedUser: UserAccount,
previousUser: Option[UserAccount])
extends ChangeLogMessage
trait ChangeLogStore {
def log(change: ChangeLogMessage): Try[ChangeLogMessage]
}
// $COVERAGE-ON$

View File

@@ -16,24 +16,13 @@
package controllers package controllers
import cats.effect.IO
import javax.inject.{Inject, Singleton} import javax.inject.{Inject, Singleton}
import models.UserAccount
import org.joda.time.DateTime import org.joda.time.DateTime
import vinyldns.core.domain.membership.{User, UserChange, UserChangeRepository, UserRepository}
import scala.util.{Failure, Success, Try}
case class UserChangeRecord(
changeId: Long,
userId: String,
user: String,
timeStamp: DateTime,
changeType: UserChangeType,
newUser: UserAccount,
oldUser: UserAccount)
@Singleton @Singleton
class UserAccountAccessor @Inject()(store: UserAccountStore) { class UserAccountAccessor @Inject()(users: UserRepository, changes: UserChangeRepository) {
/** /**
* Lookup a user in the store. Using identifier as the user id and/or name * Lookup a user in the store. Using identifier as the user id and/or name
@@ -42,16 +31,24 @@ class UserAccountAccessor @Inject()(store: UserAccountStore) {
* @return Success(Some(user account)) on success, Success(None) if the user does not exist and Failure when there * @return Success(Some(user account)) on success, Success(None) if the user does not exist and Failure when there
* was an error. * was an error.
*/ */
def get(identifier: String): Try[Option[UserAccount]] = def get(identifier: String): IO[Option[User]] =
store.getUserById(identifier) match { users.getUser(identifier).flatMap {
case Success(None) => store.getUserByName(identifier) case None => users.getUserByName(identifier)
case Success(Some(user)) => Success(Some(user)) case found => IO(found)
case Failure(ex) => Failure(ex)
} }
def put(user: UserAccount): Try[UserAccount] = def create(user: User): IO[User] =
store.storeUser(user) for {
_ <- users.save(user)
_ <- changes.save(UserChange.CreateUser(user, "system", DateTime.now))
} yield user
def getUserByKey(key: String): Try[Option[UserAccount]] = def update(user: User, oldUser: User): IO[User] =
store.getUserByKey(key) for {
_ <- users.save(user)
_ <- changes.save(UserChange.UpdateUser(user, "system", DateTime.now, oldUser))
} yield user
def getUserByKey(key: String): IO[Option[User]] =
users.getUserByAccessKey(key)
} }

View File

@@ -1,36 +0,0 @@
/*
* 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 controllers
import models.UserAccount
import scala.util.Try
// $COVERAGE-OFF$
trait UserAccountStore {
def getUserById(userId: String): Try[Option[UserAccount]]
def getUserByName(username: String): Try[Option[UserAccount]]
def getUserByKey(accessKey: String): Try[Option[UserAccount]]
def storeUser(user: UserAccount): Try[UserAccount]
}
sealed trait UserChangeType
final case object Created extends UserChangeType
final case object Updated extends UserChangeType
final case object Deleted extends UserChangeType
// $COVERAGE-ON$

View File

@@ -19,7 +19,7 @@ package controllers
import java.util import java.util
import com.amazonaws.auth.{AWSCredentials, BasicAWSCredentials, SignerFactory} import com.amazonaws.auth.{AWSCredentials, BasicAWSCredentials, SignerFactory}
import models.{SignableVinylDNSRequest, UserAccount, VinylDNSRequest} import models.{SignableVinylDNSRequest, VinylDNSRequest}
import org.joda.time.DateTime import org.joda.time.DateTime
import play.api.{Logger, _} import play.api.{Logger, _}
import play.api.data.Form import play.api.data.Form
@@ -28,7 +28,11 @@ import play.api.libs.json._
import play.api.libs.ws.WSClient import play.api.libs.ws.WSClient
import play.api.mvc._ import play.api.mvc._
import java.util.HashMap import java.util.HashMap
import cats.effect.IO
import javax.inject.{Inject, Singleton} import javax.inject.{Inject, Singleton}
import vinyldns.core.domain.membership.LockStatus.LockStatus
import vinyldns.core.domain.membership.{LockStatus, User}
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.ExecutionContext.Implicits.global
@@ -44,8 +48,6 @@ object VinylDNS {
private val MSG = "alertMessage" private val MSG = "alertMessage"
def error(msg: String): Flash = Flash(Map(TYPE -> "danger", MSG -> msg)) def error(msg: String): Flash = Flash(Map(TYPE -> "danger", MSG -> msg))
def warning(msg: String): Flash = Flash(Map(TYPE -> "warning", MSG -> msg)) def warning(msg: String): Flash = Flash(Map(TYPE -> "warning", MSG -> msg))
def info(msg: String): Flash = Flash(Map(TYPE -> "info", MSG -> msg))
def success(msg: String): Flash = Flash(Map(TYPE -> "success", MSG -> msg))
def fromFlash(flash: Flash): Option[Alert] = def fromFlash(flash: Flash): Option[Alert] =
(flash.get(TYPE), flash.get(MSG)) match { (flash.get(TYPE), flash.get(MSG)) match {
@@ -63,14 +65,14 @@ object VinylDNS {
isSuper: Boolean, isSuper: Boolean,
id: String) id: String)
object UserInfo { object UserInfo {
def fromAccount(account: UserAccount): UserInfo = def fromUser(user: User): UserInfo =
UserInfo( UserInfo(
userName = account.username, userName = user.userName,
firstName = account.firstName, firstName = user.firstName,
lastName = account.lastName, lastName = user.lastName,
email = account.email, email = user.email,
isSuper = account.isSuper, isSuper = user.isSuper,
id = account.userId id = user.id
) )
} }
@@ -93,7 +95,6 @@ class VinylDNS @Inject()(
configuration: Configuration, configuration: Configuration,
authenticator: Authenticator, authenticator: Authenticator,
userAccountAccessor: UserAccountAccessor, userAccountAccessor: UserAccountAccessor,
auditLogAccessor: ChangeLogStore,
wsClient: WSClient, wsClient: WSClient,
components: ControllerComponents) components: ControllerComponents)
extends AbstractController(components) { extends AbstractController(components) {
@@ -205,27 +206,22 @@ class VinylDNS @Inject()(
withAuthenticatedUser { user => withAuthenticatedUser { user =>
val response = userAccountAccessor.get(user).map { val response = userAccountAccessor.get(user).map {
case Some(userDetails) => case Some(userDetails) =>
Ok(Json.toJson(VinylDNS.UserInfo.fromAccount(userDetails))) Ok(Json.toJson(VinylDNS.UserInfo.fromUser(userDetails)))
.withHeaders(cacheHeaders: _*) .withHeaders(cacheHeaders: _*)
case _ => case _ =>
Status(404)(s"Did not find user data for '$user'") Status(404)(s"Did not find user data for '$user'")
} }
Future.fromTry(response) response.unsafeToFuture()
} }
} }
private def processCsv(username: String, account: UserAccount): Result = private def processCsv(username: String, user: User): Result =
account.username match { user.userName match {
case accountUsername: String if accountUsername == username => case accountUsername: String if accountUsername == username =>
Logger.info( Logger.info(s"Sending credentials for user=$username with key accessKey=${user.accessKey}")
s"Sending credentials for user=$username with key accessKey=${account.accessKey}")
Ok( Ok(
s"NT ID, access key, secret key,api url\n%s,%s,%s,%s" s"NT ID, access key, secret key,api url\n%s,%s,%s,%s"
.format( .format(user.userName, user.accessKey, user.secretKey, vinyldnsServiceBackend))
account.username,
account.accessKey,
account.accessSecret,
vinyldnsServiceBackend))
.as("text/csv") .as("text/csv")
case _ => case _ =>
@@ -236,12 +232,15 @@ class VinylDNS @Inject()(
def serveCredsFile(fileName: String): Action[AnyContent] = Action.async { implicit request => def serveCredsFile(fileName: String): Action[AnyContent] = Action.async { implicit request =>
Logger.info(s"Serving credentials for file $fileName") Logger.info(s"Serving credentials for file $fileName")
withAuthenticatedUser { username => withAuthenticatedUser { username =>
userAccountAccessor.get(username) match { userAccountAccessor
case Success(Some(account)) => Future(processCsv(username, account)) .get(username)
case Success(None) => .flatMap {
throw new UnsupportedOperationException(s"Error - User account for $username not found") case Some(account) => IO(processCsv(username, account))
case Failure(ex) => throw ex case None =>
} IO.raiseError(
new UnsupportedOperationException(s"Error - User account for $username not found"))
}
.unsafeToFuture()
} }
} }
@@ -252,7 +251,7 @@ class VinylDNS @Inject()(
.map(response => { .map(response => {
Status(200)("Successfully regenerated credentials") Status(200)("Successfully regenerated credentials")
.withHeaders(cacheHeaders: _*) .withHeaders(cacheHeaders: _*)
.withSession("username" -> response.username, "accessKey" -> response.accessKey) .withSession("username" -> response.userName, "accessKey" -> response.accessKey)
}) })
.recover { .recover {
case _: UserDoesNotExistException => case _: UserDoesNotExistException =>
@@ -261,62 +260,50 @@ class VinylDNS @Inject()(
} }
} }
private def processRegenerate(oldAccountName: String): Try[UserAccount] = private def processRegenerate(oldAccountName: String): Try[User] = {
for { val update = for {
oldAccount <- userAccountAccessor.get(oldAccountName).flatMap { oldUser <- userAccountAccessor.get(oldAccountName).flatMap {
case Some(account) => Success(account) case Some(u) => IO.pure(u)
case None => case None =>
Failure( IO.raiseError(
new UserDoesNotExistException(s"Error - User account for $oldAccountName not found")) new UserDoesNotExistException(s"Error - User account for $oldAccountName not found"))
} }
account = oldAccount.regenerateCredentials() newUser = oldUser.regenerateCredentials()
_ <- userAccountAccessor.put(account) _ <- userAccountAccessor.update(newUser, oldUser)
_ <- auditLogAccessor.log(
UserChangeMessage(
account.userId,
account.username,
DateTime.now(),
ChangeType("updated"),
account,
Some(oldAccount)))
} yield { } yield {
Logger.info(s"Credentials successfully regenerated for ${account.username}") Logger.info(s"Credentials successfully regenerated for ${newUser.userName}")
account
}
private def createNewUser(details: UserDetails): Try[UserAccount] = {
val newAccount =
UserAccount(details.username, details.firstName, details.lastName, details.email)
for {
newUser <- userAccountAccessor.put(newAccount)
} yield {
auditLogAccessor.log(
UserChangeMessage(
newUser.userId,
newUser.username,
DateTime.now(),
ChangeType("created"),
newUser,
None))
newUser newUser
} }
update.attempt.unsafeRunSync().toTry
}
private def createNewUser(details: UserDetails): IO[User] = {
val newUser =
User(
details.username,
User.generateKey,
User.generateKey,
details.firstName,
details.lastName,
details.email)
userAccountAccessor.create(newUser)
} }
def getUserDataByUsername(username: String): Action[AnyContent] = Action.async { def getUserDataByUsername(username: String): Action[AnyContent] = Action.async {
implicit request => implicit request =>
withAuthenticatedUser { _ => withAuthenticatedUser { _ =>
Future {
.fromTry { for {
for { userDetails <- IO.fromEither(authenticator.lookup(username).toEither)
userDetails <- authenticator.lookup(username) existingAccount <- userAccountAccessor.get(userDetails.username)
existingAccount <- userAccountAccessor.get(userDetails.username) userAccount <- existingAccount match {
userAccount <- existingAccount match { case Some(user) => IO(VinylDNS.UserInfo.fromUser(user))
case Some(user) => Try(VinylDNS.UserInfo.fromAccount(user)) case None =>
case None => createNewUser(userDetails).map(VinylDNS.UserInfo.fromUser)
createNewUser(userDetails).map(VinylDNS.UserInfo.fromAccount) }
} } yield userAccount
} yield userAccount }.unsafeToFuture()
}
.map(Json.toJson(_)) .map(Json.toJson(_))
.map(Ok(_).withHeaders(cacheHeaders: _*)) .map(Ok(_).withHeaders(cacheHeaders: _*))
.recover { .recover {
@@ -335,33 +322,24 @@ class VinylDNS @Inject()(
Logger.info( Logger.info(
s"user [${userDetails.username}] logged in with ldap path [${userDetails.nameInNamespace}]") s"user [${userDetails.username}] logged in with ldap path [${userDetails.nameInNamespace}]")
// get or create the new style user account val user = userAccountAccessor
val userAccount = userAccountAccessor
.get(userDetails.username) .get(userDetails.username)
.flatMap { .flatMap {
case None => case None =>
Logger.info(s"Creating user account for ${userDetails.username}") Logger.info(s"Creating user account for ${userDetails.username}")
createNewUser(userDetails).map { createNewUser(userDetails).map { u: User =>
case user: UserAccount => Logger.info(s"User account for ${u.userName} created with id ${u.id}")
Logger.info(s"User account for ${user.username} created with id ${user.userId}") u
user
} }
case Some(user) => case Some(u) =>
Logger.info(s"User account for ${user.username} exists with id ${user.userId}") Logger.info(s"User account for ${u.userName} exists with id ${u.id}")
Success(user) IO.pure(u)
} }
.recoverWith { .unsafeRunSync()
case ex =>
Logger.error(
s"User retrieval or creation failed for user ${userDetails.username} with message ${ex.getMessage}")
throw ex
}
.get
Logger.info( Logger.info(s"--NEW MEMBERSHIP-- user [${user.userName}] logged in with id [${user.id}]")
s"--NEW MEMBERSHIP-- user [${userAccount.username}] logged in with id [${userAccount.userId}]")
Redirect("/index") Redirect("/index")
.withSession("username" -> userAccount.username, "accessKey" -> userAccount.accessKey) .withSession("username" -> user.userName, "accessKey" -> user.accessKey)
} }
def getZones: Action[AnyContent] = Action.async { implicit request => def getZones: Action[AnyContent] = Action.async { implicit request =>
@@ -564,13 +542,13 @@ class VinylDNS @Inject()(
def getUserCreds(keyOption: Option[String]): BasicAWSCredentials = def getUserCreds(keyOption: Option[String]): BasicAWSCredentials =
keyOption match { keyOption match {
case Some(key) => case Some(key) =>
userAccountAccessor.getUserByKey(key) match { userAccountAccessor.getUserByKey(key).attempt.unsafeRunSync() match {
case Success(Some(account)) => case Right(Some(account)) =>
new BasicAWSCredentials(account.accessKey, account.accessSecret) new BasicAWSCredentials(account.accessKey, account.secretKey)
case Success(None) => case Right(None) =>
throw new IllegalArgumentException( throw new IllegalArgumentException(
s"Key [$key] Not Found!! Please logout then back in.") s"Key [$key] Not Found!! Please logout then back in.")
case Failure(ex) => throw ex case Left(ex) => throw ex
} }
case None => throw new IllegalArgumentException("No Key Found!!") case None => throw new IllegalArgumentException("No Key Found!!")
} }

View File

@@ -1,88 +0,0 @@
/*
* 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 controllers.datastores
import java.util
import com.amazonaws.AmazonClientException
import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClient
import com.amazonaws.services.dynamodbv2.model._
import controllers.{ChangeLogMessage, ChangeLogStore, UserChangeMessage}
import play.api.Configuration
import vinyldns.core.crypto.CryptoAlgebra
import scala.util.Try
class DynamoDBChangeLogStore(
dynamo: AmazonDynamoDBClient,
config: Configuration,
crypto: CryptoAlgebra)
extends ChangeLogStore {
private val tableName = config.get[String]("changelog.tablename")
private val readThroughput =
config.getOptional[Long]("changelog.provisionedReadThroughput").getOrElse(1L)
private val writeThroughput =
config.getOptional[Long]("changelog.provisionedWriteThroughput").getOrElse(1L)
private val TIME_STAMP = "timestamp"
private val tableAttributes = Seq(
new AttributeDefinition(TIME_STAMP, "S")
)
try {
dynamo.describeTable(new DescribeTableRequest(tableName))
} catch {
case _: AmazonClientException =>
dynamo.createTable(
new CreateTableRequest()
.withTableName(tableName)
.withAttributeDefinitions(tableAttributes: _*)
.withKeySchema(new KeySchemaElement(TIME_STAMP, KeyType.HASH))
.withProvisionedThroughput(new ProvisionedThroughput(readThroughput, writeThroughput)))
dynamo.describeTable(new DescribeTableRequest(tableName))
}
def log(change: ChangeLogMessage): Try[ChangeLogMessage] =
Try {
change match {
case ucm: UserChangeMessage =>
dynamo.putItem(tableName, toDynamoItem(ucm))
ucm
}
}
def toDynamoItem(message: UserChangeMessage): java.util.HashMap[String, AttributeValue] = {
val item = new util.HashMap[String, AttributeValue]()
item.put("timestamp", new AttributeValue(message.timeStamp.toString))
item.put("userId", new AttributeValue(message.userId))
item.put("username", new AttributeValue(message.username))
item.put("changeType", new AttributeValue(message.changeType.toString))
item.put(
"updatedUser",
new AttributeValue().withM(DynamoDBUserAccountStore.toItem(message.updatedUser, crypto)))
message.previousUser match {
case Some(user) =>
item.put(
"previousUser",
new AttributeValue().withM(DynamoDBUserAccountStore.toItem(user, crypto)))
case None => ()
}
item
}
}

View File

@@ -1,214 +0,0 @@
/*
* 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 controllers.datastores
import controllers.UserAccountStore
import java.util
import com.amazonaws.AmazonClientException
import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClient
import com.amazonaws.services.dynamodbv2.model._
import models.UserAccount
import org.joda.time.DateTime
import play.api.Logger
import play.api.Configuration
import vinyldns.core.crypto.CryptoAlgebra
import scala.util.Try
import scala.collection.JavaConverters._
object DynamoDBUserAccountStore {
def getAttributeOrNone(
items: java.util.Map[String, AttributeValue],
attribute: String): Option[String] =
Try(items.get(attribute).getS).toOption
def fromItem(items: java.util.Map[String, AttributeValue], crypto: CryptoAlgebra): UserAccount = {
val superUser: Try[Boolean] = Try(items.get("super").getBOOL)
new UserAccount(
items.get("userid").getS,
items.get("username").getS,
getAttributeOrNone(items, "firstname"),
getAttributeOrNone(items, "lastname"),
getAttributeOrNone(items, "email"),
new DateTime(items.get("created").getN.toLong),
items.get("accesskey").getS,
crypto.decrypt(items.get("secretkey").getS),
superUser.getOrElse(false)
)
}
def toItem(ua: UserAccount, crypto: CryptoAlgebra): java.util.Map[String, AttributeValue] = {
val item = new util.HashMap[String, AttributeValue]()
item.put("userid", new AttributeValue().withS(ua.userId))
item.put("username", new AttributeValue().withS(ua.username))
ua.firstName.foreach { firstname =>
item.put("firstname", new AttributeValue().withS(firstname))
}
ua.lastName.foreach { lastname =>
item.put("lastname", new AttributeValue().withS(lastname))
}
ua.email.foreach { email =>
item.put("email", new AttributeValue().withS(email))
}
item.put("created", new AttributeValue().withN(ua.created.getMillis.toString))
item.put("accesskey", new AttributeValue().withS(ua.accessKey))
item.put("secretkey", new AttributeValue().withS(crypto.encrypt(ua.accessSecret)))
item
}
}
class DynamoDBUserAccountStore(
dynamo: AmazonDynamoDBClient,
config: Configuration,
crypto: CryptoAlgebra)
extends UserAccountStore {
private val tableName = config.get[String]("users.tablename")
private val readThroughput =
config.getOptional[Long]("users.provisionedReadThroughput").getOrElse(1L)
private val writeThroughput =
config.getOptional[Long]("users.provisionedWriteThroughput").getOrElse(1L)
private val USER_ID = "userid"
private val USER_NAME = "username"
private val USER_INDEX_NAME = "username_index"
private val ACCESS_KEY = "accesskey"
private val ACCESS_KEY_INDEX_NAME = "access_key_index"
private val tableAttributes = Seq(
new AttributeDefinition(USER_ID, "S"),
new AttributeDefinition(USER_NAME, "S"),
new AttributeDefinition(ACCESS_KEY, "S")
)
private val gsis = Seq(
new GlobalSecondaryIndex()
.withIndexName(USER_INDEX_NAME)
.withProvisionedThroughput(new ProvisionedThroughput(readThroughput, writeThroughput))
.withKeySchema(new KeySchemaElement(USER_NAME, KeyType.HASH))
.withProjection(new Projection().withProjectionType("ALL")),
new GlobalSecondaryIndex()
.withIndexName(ACCESS_KEY_INDEX_NAME)
.withProvisionedThroughput(new ProvisionedThroughput(readThroughput, writeThroughput))
.withKeySchema(new KeySchemaElement(ACCESS_KEY, KeyType.HASH))
.withProjection(new Projection().withProjectionType("ALL"))
)
try {
dynamo.describeTable(new DescribeTableRequest(tableName))
} catch {
case _: AmazonClientException =>
dynamo.createTable(
new CreateTableRequest()
.withTableName(tableName)
.withAttributeDefinitions(tableAttributes: _*)
.withKeySchema(new KeySchemaElement(USER_ID, KeyType.HASH))
.withGlobalSecondaryIndexes(gsis: _*)
.withProvisionedThroughput(new ProvisionedThroughput(readThroughput, writeThroughput)))
dynamo.describeTable(new DescribeTableRequest(tableName))
}
def getUserById(userId: String): Try[Option[UserAccount]] = {
val key = new util.HashMap[String, AttributeValue]()
key.put(USER_ID, new AttributeValue(userId))
val request = new GetItemRequest()
.withTableName(tableName)
.withKey(key)
Try {
dynamo.getItem(request) match {
case null => None
// Amazon's client java docs state "If there is no matching item, GetItem does not return any data."
// that could mean the item has no data or a null is returned, so we need to handle both cases.
case result: GetItemResult if result.getItem == null => None
case result: GetItemResult if result.getItem.isEmpty => None
case result: GetItemResult =>
Some(DynamoDBUserAccountStore.fromItem(result.getItem, crypto))
}
}
}
def getUserByName(username: String): Try[Option[UserAccount]] = {
val attributeNames = new util.HashMap[String, String]()
attributeNames.put("#uname", USER_NAME)
val attributeValues = new util.HashMap[String, AttributeValue]()
attributeValues.put(":uname", new AttributeValue().withS(username))
val request = new QueryRequest()
.withTableName(tableName)
.withKeyConditionExpression("#uname = :uname")
.withExpressionAttributeNames(attributeNames)
.withExpressionAttributeValues(attributeValues)
.withIndexName(USER_INDEX_NAME)
Try {
dynamo.query(request) match {
case result: QueryResult if result.getCount == 1 =>
Some(DynamoDBUserAccountStore.fromItem(result.getItems.get(0), crypto))
case result: QueryResult if result.getCount == 0 => None
case result: QueryResult if result.getCount >= 2 =>
val prefixString = "!!! INCONSISTENT DATA !!!"
Logger.error(s"$prefixString ${result.getCount} user accounts for ntid $username found!")
for {
item <- result.getItems.asScala
} {
val user = DynamoDBUserAccountStore.fromItem(item, crypto)
Logger.error(s"$prefixString ${user.username} has user ID of ${user.userId}")
}
Some(DynamoDBUserAccountStore.fromItem(result.getItems.get(0), crypto))
}
}
}
def getUserByKey(key: String): Try[Option[UserAccount]] = {
val attributeNames = new util.HashMap[String, String]()
attributeNames.put("#ukey", ACCESS_KEY)
val attributeValues = new util.HashMap[String, AttributeValue]()
attributeValues.put(":ukey", new AttributeValue().withS(key))
val request = new QueryRequest()
.withTableName(tableName)
.withKeyConditionExpression("#ukey = :ukey")
.withExpressionAttributeNames(attributeNames)
.withExpressionAttributeValues(attributeValues)
.withIndexName(ACCESS_KEY_INDEX_NAME)
Try {
dynamo.query(request) match {
case result: QueryResult if result.getCount == 1 =>
Some(DynamoDBUserAccountStore.fromItem(result.getItems.get(0), crypto))
case result: QueryResult if result.getCount == 0 => None
case result: QueryResult if result.getCount >= 2 =>
val prefixString = "!!! INCONSISTENT DATA !!!"
Logger.error(s"$prefixString ${result.getCount} user accounts for access key $key found!")
for {
item <- result.getItems.asScala
} {
val user = DynamoDBUserAccountStore.fromItem(item, crypto)
Logger.error(s"$prefixString ${user.username} has key of ${user.accessKey}")
}
Some(DynamoDBUserAccountStore.fromItem(result.getItems.get(0), crypto))
}
}
}
def storeUser(user: UserAccount): Try[UserAccount] = {
val item = DynamoDBUserAccountStore.toItem(user, crypto)
val request = new PutItemRequest().withItem(item).withTableName(tableName)
Try {
dynamo.putItem(request) match {
case _: PutItemResult => user
}
}
}
}

View File

@@ -1,36 +0,0 @@
/*
* 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 controllers.datastores
import controllers.{ChangeLogMessage, ChangeLogStore, UserChangeMessage}
import scala.collection.mutable
import scala.util.Try
class InMemoryChangeLogStore extends ChangeLogStore {
type InMemoryLog = mutable.MutableList[ChangeLogMessage]
val userChangeLog = new InMemoryLog()
def log(change: ChangeLogMessage): Try[ChangeLogMessage] =
Try {
change match {
case ucm: UserChangeMessage =>
userChangeLog += ucm
ucm
}
}
}

View File

@@ -1,52 +0,0 @@
/*
* 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 controllers.datastores
import controllers.UserAccountStore
import models.UserAccount
import scala.collection.mutable
import scala.util.Try
class InMemoryUserAccountStore extends UserAccountStore {
val users = new mutable.HashMap[String, UserAccount]()
val usersByNameIndex = new mutable.HashMap[String, String]()
val usersByKeyIndex = new mutable.HashMap[String, String]()
def getUserById(userId: String): Try[Option[UserAccount]] =
Try(users.get(userId))
def getUserByName(username: String): Try[Option[UserAccount]] =
Try(usersByNameIndex.get(username)).flatMap {
case Some(userId) => getUserById(userId)
case None => Try(None)
}
def getUserByKey(key: String): Try[Option[UserAccount]] =
Try(usersByKeyIndex.get(key)).flatMap {
case Some(userId) => getUserById(userId)
case None => Try(None)
}
def storeUser(user: UserAccount): Try[UserAccount] =
Try {
users.put(user.userId, user)
usersByNameIndex.put(user.username, user.userId)
usersByKeyIndex.put(user.accessKey, user.userId)
user
}
}

View File

@@ -1,69 +0,0 @@
/*
* 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 models
import java.util.UUID
import org.apache.commons.lang3.RandomStringUtils
import org.joda.time.DateTime
case class UserAccount(
userId: String,
username: String,
firstName: Option[String],
lastName: Option[String],
email: Option[String],
created: DateTime,
accessKey: String,
accessSecret: String,
isSuper: Boolean = false) {
private def generateKey: String = RandomStringUtils.randomAlphanumeric(20)
override def toString() = {
val sb = new StringBuilder
sb.append("UserAccount: [")
sb.append("id=\"").append(userId).append("\"; ")
sb.append("username=\"").append(username).append("\"; ")
sb.append("firstName=\"").append(firstName).append("\"; ")
sb.append("lastName=\"").append(lastName).append("\"; ")
sb.append("email=\"").append(email).append("\"; ")
sb.append("accessKey=\"").append(accessKey).append("\"; ")
sb.append("]")
sb.toString
}
def regenerateCredentials(): UserAccount =
copy(accessKey = generateKey, accessSecret = generateKey)
}
object UserAccount {
private def generateKey: String = RandomStringUtils.randomAlphanumeric(20)
def apply(
username: String,
firstName: Option[String],
lastName: Option[String],
email: Option[String]): UserAccount = {
val userId = UUID.randomUUID().toString
val createdTime = DateTime.now()
val key = generateKey
val secret = generateKey
UserAccount(userId, username, firstName, lastName, email, createdTime, key, secret, false)
}
}

View File

@@ -0,0 +1,108 @@
/*
* 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 modules
/*
* 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.
*/
import cats.effect.IO
import com.google.inject.AbstractModule
import controllers._
import play.api.{Configuration, Environment}
import vinyldns.core.crypto.CryptoAlgebra
import vinyldns.core.domain.membership.{UserChangeRepository, UserRepository}
import vinyldns.dynamodb.repository.{
DynamoDBDataStoreSettings,
DynamoDBRepositorySettings,
DynamoDBUserChangeRepository,
DynamoDBUserRepository
}
class VinylDNSModule(environment: Environment, configuration: Configuration)
extends AbstractModule {
val settings = new Settings(configuration)
def configure(): Unit = {
// Note: Leaving the unsafeRunSync here until we do full dynamic loading of the data store
val crypto = CryptoAlgebra.load(configuration.underlying.getConfig("crypto")).unsafeRunSync()
val dynamoConfig = configuration.get[Configuration]("dynamo")
val dynamoSettings = DynamoDBDataStoreSettings(
key = dynamoConfig.get[String]("key"),
secret = dynamoConfig.get[String]("secret"),
endpoint = dynamoConfig.get[String]("endpoint"),
region = dynamoConfig.get[String]("region")
)
bind(classOf[Authenticator]).toInstance(authenticator())
bind(classOf[UserRepository]).toInstance(userRepository(dynamoSettings, crypto))
bind(classOf[UserChangeRepository]).toInstance(changeLogStore(dynamoSettings, crypto))
}
private def authenticator(): Authenticator =
/**
* Why not load config here you ask? Well, there is some ugliness in the LdapAuthenticator
* that I am not looking to undo at this time. There are private classes
* that do some wrapping. It all seems to work, so I am leaving it alone
* to complete the Play framework upgrade
*/
LdapAuthenticator(settings)
private def userRepository(
dynamoSettings: DynamoDBDataStoreSettings,
crypto: CryptoAlgebra): DynamoDBUserRepository = {
for {
repoSettings <- IO(
DynamoDBRepositorySettings(
tableName = configuration.get[String]("users.tablename"),
provisionedReads = configuration.get[Long]("users.provisionedReadThroughput"),
provisionedWrites = configuration.get[Long]("users.provisionedWriteThroughput")
)
)
repo <- DynamoDBUserRepository(repoSettings, dynamoSettings, crypto)
} yield repo
}.unsafeRunSync()
private def changeLogStore(
dynamoSettings: DynamoDBDataStoreSettings,
crypto: CryptoAlgebra): DynamoDBUserChangeRepository = {
for {
repoSettings <- IO(
DynamoDBRepositorySettings(
tableName = configuration.get[String]("changelog.tablename"),
provisionedReads = configuration.get[Long]("changelog.provisionedReadThroughput"),
provisionedWrites = configuration.get[Long]("changelog.provisionedWriteThroughput")
)
)
repo <- DynamoDBUserChangeRepository(repoSettings, dynamoSettings, crypto)
} yield repo
}.unsafeRunSync()
}

View File

@@ -7,23 +7,23 @@ portal.vinyldns.backend.url = "http://not.real.com"
dynamo { dynamo {
key = "akid goes here" key = "akid goes here"
secret = "secret key goes here" secret = "secret key goes here"
endpoint = "endpoint url goes here" endpoint = "http://foo.bar"
region = "us-east-1" # note: we are always in us-east-1, but this can be overridden region = "us-east-1" # note: we are always in us-east-1, but this can be overridden
test_datastore = true test_datastore = false
} }
users { users {
dummy = true dummy = false
tablename = "userAccounts" tablename = "users-test"
provisionedReadThroughput = 100 provisionedReadThroughput = 100
provisionedWriteThroughput = 100 provisionedWriteThroughput = 100
} }
changelog { changelog {
dummy=true dummy = false
tablename="usersAndGroupChanges" tablename = "usersAndGroupChanges-test"
provisionedReadThroughput=100 provisionedReadThroughput = 100
provisionedWriteThroughput=100 provisionedWriteThroughput = 100
} }
LDAP { LDAP {
@@ -60,3 +60,5 @@ links = [
icon = "" icon = ""
} }
] ]
play.modules.enabled += "modules.VinylDNSModule"

View File

@@ -111,5 +111,7 @@ links = [
} }
] ]
play.modules.enabled += "modules.VinylDNSModule"
// Local.conf has files specific to your environment, for example your own LDAP settings // Local.conf has files specific to your environment, for example your own LDAP settings
include "local.conf" include "local.conf"

View File

@@ -30,7 +30,7 @@ class ConfigSpec extends Specification {
dynamoAKID must beEqualTo("akid goes here") dynamoAKID must beEqualTo("akid goes here")
dynamoSecret must beEqualTo("secret key goes here") dynamoSecret must beEqualTo("secret key goes here")
dynamoEndpoint must beEqualTo("endpoint url goes here") dynamoEndpoint must beEqualTo("http://foo.bar")
region must beEqualTo("us-east-1") region must beEqualTo("us-east-1")
} }
} }

View File

@@ -22,12 +22,8 @@ import org.specs2.mutable.Specification
import org.specs2.runner.JUnitRunner import org.specs2.runner.JUnitRunner
import play.api.test.Helpers._ import play.api.test.Helpers._
import play.api.test._ import play.api.test._
import play.api.inject.guice.GuiceApplicationBuilder
@RunWith(classOf[JUnitRunner]) @RunWith(classOf[JUnitRunner])
class FrontendControllerSpec extends Specification with Mockito { class FrontendControllerSpec extends Specification with Mockito with TestApplicationData {
// this has to be lazy due to how the FrontendController is boot strapped
def app = GuiceApplicationBuilder().build()
"FrontendController" should { "FrontendController" should {
"send 404 on a bad request" in new WithApplication(app) { "send 404 on a bad request" in new WithApplication(app) {

View File

@@ -18,11 +18,11 @@ package controllers
import javax.naming.NamingEnumeration import javax.naming.NamingEnumeration
import javax.naming.directory._ import javax.naming.directory._
import controllers.LdapAuthenticator.{ContextCreator, LdapByDomainAuthenticator} import controllers.LdapAuthenticator.{ContextCreator, LdapByDomainAuthenticator}
import org.specs2.mock.Mockito import org.specs2.mock.Mockito
import org.specs2.mock.mockito.ArgumentCapture import org.specs2.mock.mockito.ArgumentCapture
import org.specs2.mutable.Specification import org.specs2.mutable.Specification
import play.api.{Configuration, Environment}
import play.api.test.WithApplication import play.api.test.WithApplication
import play.api.inject.guice.GuiceApplicationBuilder import play.api.inject.guice.GuiceApplicationBuilder
@@ -63,24 +63,22 @@ class LdapAuthenticatorSpec extends Specification with Mockito {
Mocks(contextCreator, context, searchResults, searchNext, byDomainAuthenticator, mockAttributes) Mocks(contextCreator, context, searchResults, searchNext, byDomainAuthenticator, mockAttributes)
} }
val app = GuiceApplicationBuilder().build()
val testApp = GuiceApplicationBuilder().configure(Map("portal.test_login" -> true)).build()
val settings = new Settings(app.configuration)
val testDomain1 = LdapSearchDomain("someDomain", "DC=test,DC=test,DC=com") val testDomain1 = LdapSearchDomain("someDomain", "DC=test,DC=test,DC=com")
val testDomain2 = LdapSearchDomain("anotherDomain", "DC=test,DC=com") val testDomain2 = LdapSearchDomain("anotherDomain", "DC=test,DC=com")
"LdapAuthenticator" should { "LdapAuthenticator" should {
"apply method must create an LDAP Authenticator" in new WithApplication(app) { "apply method must create an LDAP Authenticator" in {
val underTest = LdapAuthenticator.apply(settings) val testConfig: Configuration =
underTest must haveClass( Configuration.load(Environment.simple()) ++ Configuration.from(
ClassTag( Map("portal.test_login" -> false))
new LdapAuthenticator( val underTest = LdapAuthenticator.apply(new Settings(testConfig))
settings.ldapSearchBase, underTest must beAnInstanceOf[LdapAuthenticator]
LdapByDomainAuthenticator.apply(),
mock[ServiceAccount]).getClass))
} }
"apply method must create a Test Authenticator if selected" in new WithApplication(testApp) { "apply method must create a Test Authenticator if selected" in {
val underTest = LdapAuthenticator.apply(new Settings(app.configuration)) val testConfig: Configuration =
Configuration.load(Environment.simple()) ++ Configuration.from(
Map("portal.test_login" -> true))
val underTest = LdapAuthenticator.apply(new Settings(testConfig))
underTest must beAnInstanceOf[TestAuthenticator] underTest must beAnInstanceOf[TestAuthenticator]
} }
".authenticate" should { ".authenticate" should {

View File

@@ -0,0 +1,171 @@
/*
* 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 controllers
import cats.effect.IO
import org.joda.time.DateTime
import org.specs2.mock.Mockito
import play.api.inject.bind
import play.api.inject.guice.GuiceApplicationBuilder
import play.api.{Application, Configuration, Environment}
import play.api.libs.json.{JsObject, JsValue, Json}
import vinyldns.core.domain.membership._
import scala.util.Success
trait TestApplicationData { this: Mockito =>
val frodoDetails = UserDetails(
"CN=frodo,OU=hobbits,DC=middle,DC=earth",
"frodo",
Some("fbaggins@hobbitmail.me"),
Some("Frodo"),
Some("Baggins"))
val frodoUser = User(
"fbaggins",
"key",
"secret",
Some("Frodo"),
Some("Baggins"),
Some("fbaggins@hobbitmail.me"),
DateTime.now,
"frodo-uuid")
val newFrodoLog = UserChange(
"frodo-uuid",
frodoUser,
"fbaggins",
DateTime.now,
None,
UserChangeType.Create
).toOption.get
val serviceAccountDetails =
UserDetails("CN=frodo,OU=hobbits,DC=middle,DC=earth", "service", None, None, None)
val serviceAccount =
User("service", "key", "secret", None, None, None, DateTime.now, "service-uuid")
val frodoJsonString: String =
s"""{
| "userName": "${frodoUser.userName}",
| "firstName": "${frodoUser.firstName}",
| "lastName": "${frodoUser.lastName}",
| "email": "${frodoUser.email}",
| "created": "${frodoUser.created}",
| "id": "${frodoUser.id}"
|}
""".stripMargin
val samAccount = User(
"sgamgee",
"key",
"secret",
Some("Samwise"),
Some("Gamgee"),
Some("sgamgee@hobbitmail.me"),
DateTime.now,
"sam-uuid")
val samDetails = UserDetails(
"CN=sam,OU=hobbits,DC=middle,DC=earth",
"sam",
Some("sgamgee@hobbitmail.me"),
Some("Sam"),
Some("Gamgee"))
val frodoJson: String =
s"""{
|"name": "${frodoUser.userName}"
|}
""".stripMargin
val hobbitGroupId = "uuid-12345-abcdef"
val hobbitGroup: JsValue = Json.parse(s"""{
| "id": "${hobbitGroupId}",
| "name": "hobbits",
| "email": "hobbitAdmin@shire.me",
| "description": "Hobbits of the shire",
| "members": [ { "id": "${frodoUser.id}" }, { "id": "samwise-userId" } ],
| "admins": [ { "id": "${frodoUser.id}" } ]
| }
""".stripMargin)
val ringbearerGroup: JsValue = Json.parse(
s"""{
| "id": "ringbearer-group-uuid",
| "name": "ringbearers",
| "email": "future-minions@mordor.me",
| "description": "Corruptable folk of middle-earth",
| "members": [ { "id": "${frodoUser.id}" }, { "id": "sauron-userId" } ],
| "admins": [ { "id": "sauron-userId" } ]
| }
""".stripMargin
)
val hobbitGroupRequest: JsValue = Json.parse(s"""{
| "name": "hobbits",
| "email": "hobbitAdmin@shire.me",
| "description": "Hobbits of the shire",
| "members": [ { "id": "${frodoUser.id}" }, { "id": "samwise-userId" } ],
| "admins": [ { "id": "${frodoUser.id}" } ]
| }
""".stripMargin)
val invalidHobbitGroup: JsValue = Json.parse(s"""{
| "name": "hobbits",
| "email": "hobbitAdmin@shire.me",
| "description": "Hobbits of the shire",
| "members": [ { "id": "${frodoUser.id}" }, { "id": "merlin-userId" } ],
| "admins": [ { "id": "${frodoUser.id}" } ]
| }
""".stripMargin)
val hobbitGroupMembers: JsValue = Json.parse(
s"""{
| "members": [ ${frodoJsonString} ],
| "maxItems": 100
|}
""".stripMargin
)
val groupList: JsObject = Json.obj("groups" -> Json.arr(hobbitGroup))
val emptyGroupList: JsObject = Json.obj("groups" -> Json.arr())
val frodoGroupList: JsObject = Json.obj("groups" -> Json.arr(hobbitGroup, ringbearerGroup))
val simulatedBackendPort: Int = 9001
val testConfig: Configuration =
Configuration.load(Environment.simple()) ++ Configuration.from(
Map("portal.vinyldns.backend.url" -> s"http://localhost:$simulatedBackendPort"))
val mockAuth: Authenticator = mock[Authenticator]
val mockUserRepo: UserRepository = mock[UserRepository]
val mockUserChangeRepo: UserChangeRepository = mock[UserChangeRepository]
mockAuth.authenticate("frodo", "secondbreakfast").returns(Success(frodoDetails))
mockUserRepo.getUser(anyString).returns(IO.pure(Some(frodoUser)))
mockUserChangeRepo.save(any[UserChange]).returns(IO.pure(newFrodoLog))
def app: Application =
GuiceApplicationBuilder()
.disable[modules.VinylDNSModule]
.bindings(
bind[Authenticator].to(mockAuth),
bind[UserRepository].to(mockUserRepo),
bind[UserChangeRepository].to(mockUserChangeRepo)
)
.configure(testConfig)
.build()
}

View File

@@ -16,157 +16,78 @@
package controllers package controllers
import models.UserAccount import cats.effect.IO
import org.joda.time.DateTime import org.joda.time.DateTime
import org.specs2.mock.Mockito import org.specs2.mock.Mockito
import org.specs2.mutable.Specification import org.specs2.mutable.Specification
import org.specs2.specification.BeforeEach
import vinyldns.core.domain.membership._
import scala.util.{Failure, Success} class UserAccountAccessorSpec extends Specification with Mockito with BeforeEach {
private val user = User(
"fbaggins",
"key",
"secret",
Some("Frodo"),
Some("Baggins"),
Some("fbaggins@hobbitmail.me"),
DateTime.now,
"frodo-uuid")
private val userLog = UserChange(
"frodo-uuid",
user,
"fbaggins",
DateTime.now,
None,
UserChangeType.Create
).toOption.get
private val mockRepo = mock[UserRepository]
private val mockChangeRepo = mock[UserChangeRepository]
private val underTest = new UserAccountAccessor(mockRepo, mockChangeRepo)
protected def before: Any =
org.mockito.Mockito.reset(mockRepo, mockChangeRepo)
class UserAccountAccessorSpec extends Specification with Mockito {
"User Account Accessor" should { "User Account Accessor" should {
"Return the user when storing a user that does not exist already" in { "Return the user when storing a user that does not exist already" in {
val mockStore = mock[UserAccountStore] mockRepo.save(any[User]).returns(IO.pure(user))
val user = new UserAccount( mockChangeRepo.save(any[UserChange]).returns(IO.pure(userLog))
"uuid", underTest.create(user).unsafeRunSync() must beEqualTo(user)
"fbaggins", there.was(one(mockChangeRepo).save(any[UserChange]))
Some("Frodo"),
Some("Baggins"),
Some("fbaggins@hobbitmail.me"),
DateTime.now(),
"key",
"secret")
mockStore.storeUser(any[UserAccount]).returns(Success(user))
val underTest = new UserAccountAccessor(mockStore)
underTest.put(user) must beASuccessfulTry(user)
} }
"Return the new user when storing a user that already exists in the store" in { "Return the new user when storing a user that already exists in the store" in {
val mockStore = mock[UserAccountStore] val newUser = user.copy(accessKey = "new-key", secretKey = "new-secret")
val oldUser = new UserAccount( mockRepo.save(any[User]).returns(IO.pure(newUser))
"uuid", mockChangeRepo.save(any[UserChange]).returns(IO.pure(userLog))
"fbaggins", underTest.update(newUser, user).unsafeRunSync() must beEqualTo(newUser)
Some("Frodo"), there.was(one(mockChangeRepo).save(any[UserChange]))
Some("Baggins"),
Some("fbaggins@hobbitmail.me"),
DateTime.now(),
"key",
"secret")
val newUser = new UserAccount(
"uuid",
"fbaggins",
Some("Frodo"),
Some("Baggins"),
Some("fbaggins@hobbitmail.me"),
DateTime.now(),
"new-key",
"new-secret")
mockStore.storeUser(any[UserAccount]).returns(Success(newUser))
val underTest = new UserAccountAccessor(mockStore)
underTest.put(newUser) must beASuccessfulTry(newUser)
}
"Return the failure when something goes wrong while storing a user" in {
val mockStore = mock[UserAccountStore]
val ex = new IllegalArgumentException("foobar")
mockStore.storeUser(any[UserAccount]).returns(Failure(ex))
val user = new UserAccount(
"uuid",
"fbaggins",
Some("Frodo"),
Some("Baggins"),
Some("fbaggins@hobbitmail.me"),
DateTime.now(),
"key",
"secret")
val underTest = new UserAccountAccessor(mockStore)
underTest.put(user) must beAFailedTry(ex)
} }
"Return the user when retrieving a user that exists by name" in { "Return the user when retrieving a user that exists by name" in {
val mockStore = mock[UserAccountStore] mockRepo.getUserByName(user.userName).returns(IO.pure(Some(user)))
val user = new UserAccount( mockRepo.getUser(user.userName).returns(IO.pure(None))
"uuid", underTest.get("fbaggins").unsafeRunSync() must beSome(user)
"fbaggins",
Some("Frodo"),
Some("Baggins"),
Some("fbaggins@hobbitmail.me"),
DateTime.now(),
"key",
"secret")
mockStore.getUserByName(user.username).returns(Success(Some(user)))
mockStore.getUserById(user.username).returns(Success(None))
val underTest = new UserAccountAccessor(mockStore)
underTest.get("fbaggins") must beASuccessfulTry[Option[UserAccount]](Some(user))
} }
"Return the user when retrieving a user that exists by user id" in { "Return the user when retrieving a user that exists by user id" in {
val mockStore = mock[UserAccountStore] mockRepo.getUserByName(user.id).returns(IO.pure(None))
val user = new UserAccount( mockRepo.getUser(user.id).returns(IO.pure(Some(user)))
"uuid", underTest.get(user.id).unsafeRunSync() must beSome(user)
"fbaggins",
Some("Frodo"),
Some("Baggins"),
Some("fbaggins@hobbitmail.me"),
DateTime.now(),
"key",
"secret")
mockStore.getUserByName(user.userId).returns(Success(None))
mockStore.getUserById(user.userId).returns(Success(Some(user)))
val underTest = new UserAccountAccessor(mockStore)
underTest.get("uuid") must beASuccessfulTry[Option[UserAccount]](Some(user))
} }
"Return None when the user to be retrieved does not exist" in { "Return None when the user to be retrieved does not exist" in {
val mockStore = mock[UserAccountStore] mockRepo.getUserByName(any[String]).returns(IO.pure(None))
mockStore.getUserByName(any[String]).returns(Success(None)) mockRepo.getUser(any[String]).returns(IO.pure(None))
mockStore.getUserById(any[String]).returns(Success(None)) underTest.get("fbaggins").unsafeRunSync() must beNone
val underTest = new UserAccountAccessor(mockStore)
underTest.get("fbaggins") must beASuccessfulTry[Option[UserAccount]](None)
} }
"Return the failure when the user cannot be looked up via user name" in { "Return the user by access key" in {
val mockStore = mock[UserAccountStore] mockRepo.getUserByAccessKey(user.id).returns(IO.pure(Some(user)))
val user = new UserAccount( underTest.getUserByKey(user.id).unsafeRunSync() must beSome(user)
"uuid",
"fbaggins",
Some("Frodo"),
Some("Baggins"),
Some("fbaggins@hobbitmail.me"),
DateTime.now(),
"key",
"secret")
val ex = new IllegalArgumentException("foobar")
mockStore.getUserByName(user.username).returns(Failure(ex))
mockStore.getUserById(user.username).returns(Success(None))
val underTest = new UserAccountAccessor(mockStore)
underTest.get("fbaggins") must beAFailedTry(ex)
}
"Return the failure when the user cannot be looked up via user id" in {
val mockStore = mock[UserAccountStore]
val user = new UserAccount(
"uuid",
"fbaggins",
Some("Frodo"),
Some("Baggins"),
Some("fbaggins@hobbitmail.me"),
DateTime.now(),
"key",
"secret")
val ex = new IllegalArgumentException("foobar")
mockStore.getUserByName(user.userId).returns(Success(None))
mockStore.getUserById(user.userId).returns(Failure(ex))
val underTest = new UserAccountAccessor(mockStore)
underTest.get("uuid") must beAFailedTry(ex)
} }
} }
} }

File diff suppressed because it is too large Load Diff

View File

@@ -1,66 +0,0 @@
/*
* 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 controllers.datastores
import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClient
import controllers.{ChangeLogMessage, Create, UserChangeMessage}
import models.UserAccount
import org.joda.time.DateTime
import org.specs2.mock.Mockito
import org.specs2.mutable.Specification
import play.api.{Configuration, Environment}
import vinyldns.core.crypto.CryptoAlgebra
class DynamoDBChangeLogStoreSpec extends Specification with Mockito {
private val testCrypto = new CryptoAlgebra {
def encrypt(value: String): String = "encrypted!"
def decrypt(value: String): String = "decrypted!"
}
private val testUserAcc = UserAccount("foo", Some("bar"), Some("baz"), Some("qux"))
private val testMessage =
UserChangeMessage("foo", "bar", DateTime.now, Create, testUserAcc, Some(testUserAcc))
"DynamoDbChangeLogStore" should {
"accept a message and return it upon success" in {
val (client, config) = buildMocks()
val underTest = new DynamoDBChangeLogStore(client, config, testCrypto)
val result = underTest.log(testMessage)
result must beSuccessfulTry[ChangeLogMessage](testMessage)
}
"accept a message and return it upon success when email, last name, and first name are none" in {
val (client, config) = buildMocks()
val underTest = new DynamoDBChangeLogStore(client, config, testCrypto)
val user = testUserAcc.copy(firstName = None, lastName = None, email = None)
val message = testMessage.copy(previousUser = None)
val result = underTest.log(message)
result must beSuccessfulTry[ChangeLogMessage](message)
}
}
def buildMocks(): (AmazonDynamoDBClient, Configuration) = {
val client = mock[AmazonDynamoDBClient]
val config = Configuration.load(Environment.simple())
(client, config)
}
def buildTestStore(
client: AmazonDynamoDBClient = mock[AmazonDynamoDBClient],
config: Configuration = mock[Configuration]): DynamoDBUserAccountStore =
new DynamoDBUserAccountStore(client, config, testCrypto)
}

View File

@@ -1,229 +0,0 @@
/*
* 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 controllers.datastores
import java.util
import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClient
import com.amazonaws.services.dynamodbv2.model._
import models.UserAccount
import org.specs2.mock.Mockito
import org.specs2.mutable.Specification
import play.api.{Configuration, Environment}
import vinyldns.core.crypto.CryptoAlgebra
class DynamoDBUserAccountStoreSpec extends Specification with Mockito {
val testCrypto = new CryptoAlgebra {
def encrypt(value: String): String = "encrypted!"
def decrypt(value: String): String = "decrypted!"
}
"DynamoDBUserAccountStore" should {
"Store a new user when email, first name, and last name are None" in {
val (client, config) = buildMocks()
val user = UserAccount("fbaggins", None, None, None)
val item = DynamoDBUserAccountStore.toItem(user, testCrypto)
val mockResult = mock[PutItemResult]
mockResult.getAttributes.returns(item)
client.putItem(any[PutItemRequest]).returns(mockResult)
val underTest = new DynamoDBUserAccountStore(client, config, testCrypto)
val result = underTest.storeUser(user)
result must beASuccessfulTry
compareUserAccounts(result.get, user)
}
"Store a new user when everything is ok" in {
val (client, config) = buildMocks()
val user =
UserAccount("fbaggins", Some("Frodo"), Some("Baggins"), Some("fbaggins@hobbitmail.me"))
val item = DynamoDBUserAccountStore.toItem(user, testCrypto)
val mockResult = mock[PutItemResult]
mockResult.getAttributes.returns(item)
client.putItem(any[PutItemRequest]).returns(mockResult)
val underTest = new DynamoDBUserAccountStore(client, config, testCrypto)
val result = underTest.storeUser(user)
result must beASuccessfulTry
compareUserAccounts(result.get, user)
}
"Store a user over an existing user returning the new user" in {
val (client, config) = buildMocks()
val oldUser = UserAccount("old", Some("Old"), Some("User"), Some("oldman@mail.me"))
val newUser = oldUser.copy(username = "new")
val mockResult = mock[PutItemResult]
mockResult.getAttributes.returns(DynamoDBUserAccountStore.toItem(newUser, testCrypto))
client.putItem(any[PutItemRequest]).returns(mockResult)
val underTest = new DynamoDBUserAccountStore(client, config, testCrypto)
underTest.storeUser(oldUser)
val result = underTest.storeUser(newUser)
result must beASuccessfulTry
compareUserAccounts(result.get, newUser)
}
"Retrieve a given user based on user-id" in {
val user = UserAccount("fbaggins", Some("Frodo"), Some("Baggins"), Some("fb@hobbitmail.me"))
val (client, config) = buildMocks()
val getResult = mock[GetItemResult]
val resultItem = DynamoDBUserAccountStore.toItem(user, testCrypto)
getResult.getItem.returns(resultItem)
client.getItem(any[GetItemRequest]).returns(getResult)
val underTest = new DynamoDBUserAccountStore(client, config, testCrypto)
val result = underTest.getUserById(user.userId)
result must beASuccessfulTry
result.get must beSome
compareUserAccounts(result.get.get, user)
}
"Retrieve a given user based on username" in {
val user = UserAccount("fbaggins", Some("Frodo"), Some("Baggins"), Some("fb@hobbitmail.me"))
val (client, config) = buildMocks()
val queryResult = mock[QueryResult]
val resultList = new util.ArrayList[util.Map[String, AttributeValue]]()
resultList.add(DynamoDBUserAccountStore.toItem(user, testCrypto))
queryResult.getItems.returns(resultList)
queryResult.getCount.returns(1)
client.query(any[QueryRequest]).returns(queryResult)
val underTest = new DynamoDBUserAccountStore(client, config, testCrypto)
val result = underTest.getUserByName(user.username)
result must beASuccessfulTry
result.get must beSome
compareUserAccounts(result.get.get, user)
}
"Return a successful none if the user is not found by id (empty item)" in {
val user = UserAccount("fbaggins", Some("Frodo"), Some("Baggins"), Some("fb@hobbitmail.me"))
val (client, config) = buildMocks()
val getResult = mock[GetItemResult]
val resultItem = new util.HashMap[String, AttributeValue]()
getResult.getItem.returns(resultItem)
client.getItem(any[GetItemRequest]).returns(getResult)
val underTest = new DynamoDBUserAccountStore(client, config, testCrypto)
val result = underTest.getUserById(user.userId)
result must beASuccessfulTry[Option[UserAccount]](None)
}
"Return a successful none if the user is not found by id (null)" in {
val user = UserAccount("fbaggins", Some("Frodo"), Some("Baggins"), Some("fb@hobbitmail.me"))
val (client, config) = buildMocks()
val getResult = null
client.getItem(any[GetItemRequest]).returns(getResult)
val underTest = new DynamoDBUserAccountStore(client, config, testCrypto)
val result = underTest.getUserById(user.userId)
result must beASuccessfulTry[Option[UserAccount]](None)
}
"Return a successful none if the user is not found by name" in {
val user = UserAccount("fbaggins", Some("Frodo"), Some("Baggins"), Some("fb@hobbitmail.me"))
val (client, config) = buildMocks()
val queryResult = mock[QueryResult]
val resultList = new util.ArrayList[util.Map[String, AttributeValue]]()
queryResult.getItems.returns(resultList)
queryResult.getCount.returns(0)
client.query(any[QueryRequest]).returns(queryResult)
val underTest = new DynamoDBUserAccountStore(client, config, testCrypto)
val result = underTest.getUserByName(user.username)
result must beASuccessfulTry(None)
}
"Return a user based on username when more than one is found" in {
val user = UserAccount("fbaggins", Some("Frodo"), Some("Baggins"), Some("fb@hobbitmail.me"))
val secondUser =
UserAccount("fbaggins", Some("Frodo"), Some("Baggins"), Some("fb@hobbitmail.me"))
val thirdUser =
UserAccount("fbaggins", Some("Frodo"), Some("Baggins"), Some("fb@hobbitmail.me"))
val (client, config) = buildMocks()
val queryResult = mock[QueryResult]
val resultList = new util.ArrayList[util.Map[String, AttributeValue]]()
resultList.add(DynamoDBUserAccountStore.toItem(user, testCrypto))
resultList.add(DynamoDBUserAccountStore.toItem(secondUser, testCrypto))
resultList.add(DynamoDBUserAccountStore.toItem(thirdUser, testCrypto))
queryResult.getItems.returns(resultList)
queryResult.getCount.returns(3)
client.query(any[QueryRequest]).returns(queryResult)
val underTest = new DynamoDBUserAccountStore(client, config, testCrypto)
val result = underTest.getUserByName(user.username)
result must beASuccessfulTry
result.get must beSome
compareUserAccounts(result.get.get, user)
}
"Encrypt the user secret" in {
val user = UserAccount("fbaggins", Some("Frodo"), Some("Baggins"), Some("fb@hobbitmail.me"))
val mockCrypto = mock[CryptoAlgebra]
mockCrypto.encrypt(user.accessSecret).returns("hello")
val item = DynamoDBUserAccountStore.toItem(user, mockCrypto)
item.get("secretkey").getS must beEqualTo("hello")
there.was(one(mockCrypto).encrypt(user.accessSecret))
}
"Decrypt the user secret" in {
val user = UserAccount("fbaggins", Some("Frodo"), Some("Baggins"), Some("fb@hobbitmail.me"))
val mockCrypto = mock[CryptoAlgebra]
mockCrypto.encrypt(user.accessSecret).returns("encrypt")
mockCrypto.decrypt("encrypt").returns("decrypt")
val item = DynamoDBUserAccountStore.toItem(user, mockCrypto)
val u = DynamoDBUserAccountStore.fromItem(item, mockCrypto)
u.accessSecret must beEqualTo("decrypt")
there.was(one(mockCrypto).decrypt(item.get("secretkey").getS))
}
}
def buildMocks(): (AmazonDynamoDBClient, Configuration) = {
val client = mock[AmazonDynamoDBClient]
val config = Configuration.load(Environment.simple())
(client, config)
}
def buildTestStore(
client: AmazonDynamoDBClient = mock[AmazonDynamoDBClient],
config: Configuration = mock[Configuration]): DynamoDBUserAccountStore =
new DynamoDBUserAccountStore(client, config, testCrypto)
def compareUserAccounts(actual: UserAccount, expected: UserAccount) = {
actual.userId must beEqualTo(expected.userId)
actual.created.compareTo(expected.created) must beEqualTo(0)
actual.username must beEqualTo(expected.username)
actual.firstName must beEqualTo(expected.firstName)
actual.lastName must beEqualTo(expected.lastName)
actual.accessKey must beEqualTo(expected.accessKey)
testCrypto.decrypt(actual.accessSecret) must beEqualTo(
testCrypto.decrypt(expected.accessSecret))
}
}

View File

@@ -1,36 +0,0 @@
/*
* 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 controllers.datastores
import controllers.{ChangeLogMessage, Create, UserChangeMessage}
import models.UserAccount
import org.joda.time.DateTime
import org.specs2.mock.Mockito
import org.specs2.mutable.Specification
class InMemoryChangeLogStoreSpec extends Specification with Mockito {
"InMemoryChangeLogStore" should {
"accept a message and return it upon success" in {
val underTest = new InMemoryChangeLogStore
val userAcc = UserAccount("foo", "bar", None, None, None, DateTime.now, "ak", "sk")
val message = UserChangeMessage("foo", "bar", DateTime.now, Create, userAcc, None)
val result = underTest.log(message)
result must beASuccessfulTry[ChangeLogMessage](message)
}
}
}

View File

@@ -1,82 +0,0 @@
/*
* 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 controllers.datastores
import models.UserAccount
import org.specs2.mock.Mockito
import org.specs2.mutable.Specification
class InMemoryUserAccountStoreSpec extends Specification with Mockito {
"InMemoryUserAccountStore" should {
"Store a new user when everything is ok" in {
val underTest = new InMemoryUserAccountStore()
val user = mock[UserAccount]
val result = underTest.storeUser(user)
result must beASuccessfulTry(user)
}
"Store a user over an existing user returning the new user" in {
val underTest = new InMemoryUserAccountStore()
val oldUser = mock[UserAccount]
oldUser.userId.returns("user")
oldUser.username.returns("old")
val newUser = mock[UserAccount]
newUser.userId.returns("user")
newUser.username.returns("new")
underTest.storeUser(oldUser)
val result = underTest.storeUser(newUser)
result must beASuccessfulTry(newUser)
result.get.username must beEqualTo("new")
}
"Retrieve a given user based on user-id" in {
val user = UserAccount("fbaggins", Some("Frodo"), Some("Baggins"), Some("fb@hobbitmail.me"))
val underTest = new InMemoryUserAccountStore()
underTest.storeUser(user)
val result = underTest.getUserById(user.userId)
result must beASuccessfulTry[Option[UserAccount]](Some(user))
}
"Retrieve a given user based on username" in {
val user = UserAccount("fbaggins", Some("Frodo"), Some("Baggins"), Some("fb@hobbitmail.me"))
val underTest = new InMemoryUserAccountStore()
underTest.storeUser(user)
val result = underTest.getUserByName(user.username)
result must beASuccessfulTry[Option[UserAccount]](Some(user))
}
"Return a successful none if the user is not found by id" in {
val user = UserAccount("fbaggins", Some("Frodo"), Some("Baggins"), Some("fb@hobbitmail.me"))
val underTest = new InMemoryUserAccountStore()
val result = underTest.getUserById(user.userId)
result must beASuccessfulTry[Option[UserAccount]](None)
}
"Return a successful none if the user is not found by name" in {
val user = UserAccount("fbaggins", Some("Frodo"), Some("Baggins"), Some("fb@hobbitmail.me"))
val underTest = new InMemoryUserAccountStore()
val result = underTest.getUserByName(user.username)
result must beASuccessfulTry[Option[UserAccount]](None)
}
}
}

View File

@@ -1,65 +0,0 @@
/*
* 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 models
import java.util.UUID
import org.specs2.mock.Mockito
import org.specs2.mutable.Specification
class UserAccountSpec extends Specification with Mockito {
"UserAccount" should {
"Create a UserAccount from username, first name, last name and email" in {
val username = "fbaggins"
val fname = Some("Frodo")
val lname = Some("Baggins")
val email = Some("fb@hobbitmail.me")
val result = UserAccount(username, fname, lname, email)
result must beAnInstanceOf[UserAccount]
UUID.fromString(result.userId) must beAnInstanceOf[UUID]
result.username must beEqualTo(username)
result.firstName must beEqualTo(fname)
result.lastName must beEqualTo(lname)
result.email must beEqualTo(email)
result.accessKey.length must beEqualTo(20)
result.accessSecret.length must beEqualTo(20)
}
"Copy an existing UserAccount with different accessKey and accessSecret" in {
val username = "fbaggins"
val fname = Some("Frodo")
val lname = Some("Baggins")
val email = Some("fb@hobbitmail.me")
val result = UserAccount(username, fname, lname, email)
val newResult = result.regenerateCredentials()
newResult must beAnInstanceOf[UserAccount]
UUID.fromString(newResult.userId) must beEqualTo(UUID.fromString(result.userId))
newResult.username must beEqualTo(username)
newResult.firstName must beEqualTo(fname)
newResult.lastName must beEqualTo(lname)
newResult.email must beEqualTo(email)
newResult.accessKey.length must beEqualTo(20)
newResult.accessSecret.length must beEqualTo(20)
newResult.accessKey mustNotEqual result.accessKey
newResult.accessSecret mustNotEqual result.accessSecret
}
}
}

View File

@@ -32,6 +32,7 @@ object Dependencies {
"dnsjava" % "dnsjava" % "2.1.7", "dnsjava" % "dnsjava" % "2.1.7",
"org.mariadb.jdbc" % "mariadb-java-client" % "2.2.3", "org.mariadb.jdbc" % "mariadb-java-client" % "2.2.3",
"org.apache.commons" % "commons-lang3" % "3.4", "org.apache.commons" % "commons-lang3" % "3.4",
"org.apache.commons" % "commons-text" % "1.4",
"org.flywaydb" % "flyway-core" % "5.1.4", "org.flywaydb" % "flyway-core" % "5.1.4",
"org.json4s" %% "json4s-ext" % "3.5.3", "org.json4s" %% "json4s-ext" % "3.5.3",
"org.json4s" %% "json4s-jackson" % "3.5.3", "org.json4s" %% "json4s-jackson" % "3.5.3",
@@ -55,7 +56,8 @@ object Dependencies {
"com.typesafe" % "config" % configV, "com.typesafe" % "config" % configV,
"joda-time" % "joda-time" % "2.8.1", "joda-time" % "joda-time" % "2.8.1",
"org.scodec" %% "scodec-bits" % scodecV, "org.scodec" %% "scodec-bits" % scodecV,
"nl.grons" %% "metrics-scala" % metricsScalaV "nl.grons" %% "metrics-scala" % metricsScalaV,
"org.apache.commons" % "commons-text" % "1.4"
) )
lazy val dynamoDBDependencies = Seq( lazy val dynamoDBDependencies = Seq(
@@ -84,6 +86,8 @@ object Dependencies {
"com.typesafe.play" %% "play-jdbc" % playV, "com.typesafe.play" %% "play-jdbc" % playV,
"com.typesafe.play" %% "play-guice" % playV, "com.typesafe.play" %% "play-guice" % playV,
"com.typesafe.play" %% "play-ahc-ws" % playV, "com.typesafe.play" %% "play-ahc-ws" % playV,
"com.typesafe.play" %% "play-specs2" % playV % "test" "com.typesafe.play" %% "play-specs2" % playV % "test",
"com.github.pureconfig" %% "pureconfig" % pureConfigV,
"com.github.pureconfig" %% "pureconfig-cats-effect" % pureConfigV
) )
} }