mirror of
https://github.com/VinylDNS/vinyldns
synced 2025-08-22 02:02:14 +00:00
Ldap Syncer using Generic TaskScheduler (#718)
* Add task and task handler. * Update tests. * Updates. * Updates based on feedback (rebstar6). * Update tests. * Updates based on feedback (rebstar6). * Add log for sync error. * Change handleError to handleErrorWith. * WIP * WIP * Use new TaskScheduler * Fixing unit test * Cleanup errant change
This commit is contained in:
parent
2150451f52
commit
44acfde265
@ -18,12 +18,13 @@ package controllers
|
||||
|
||||
import java.util
|
||||
|
||||
import cats.effect.IO
|
||||
import cats.effect.{ContextShift, IO}
|
||||
import cats.implicits._
|
||||
import controllers.LdapAuthenticator.LdapByDomainAuthenticator
|
||||
import controllers.VinylDNS.UserDetails
|
||||
import javax.naming.directory._
|
||||
import javax.naming.Context
|
||||
import vinyldns.core.domain.membership.User
|
||||
import vinyldns.core.health.HealthCheck._
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
@ -92,8 +93,6 @@ object LdapAuthenticator {
|
||||
private[controllers] object LdapByDomainAuthenticator {
|
||||
def apply(settings: Settings): LdapByDomainAuthenticator =
|
||||
new LdapByDomainAuthenticator(settings, createContext(settings))
|
||||
|
||||
def apply(): LdapByDomainAuthenticator = LdapByDomainAuthenticator(Settings)
|
||||
}
|
||||
|
||||
/**
|
||||
@ -178,6 +177,8 @@ final case class LdapServiceException(errorMessage: String)
|
||||
extends LdapException(s"Encountered error communicating with LDAP service: $errorMessage")
|
||||
final case class InvalidCredentials(username: String)
|
||||
extends LdapException(s"Provided credentials were invalid for user [$username].")
|
||||
final case class NoLdapSearchDomainsConfigured()
|
||||
extends LdapException("No LDAP search domains were configured so user lookup is impossible.")
|
||||
|
||||
/**
|
||||
* Top level ldap authenticator that tries authenticating on multiple domains. Authentication is
|
||||
@ -191,6 +192,9 @@ class LdapAuthenticator(
|
||||
serviceAccount: ServiceAccount)
|
||||
extends Authenticator {
|
||||
|
||||
private implicit val cs: ContextShift[IO] =
|
||||
IO.contextShift(scala.concurrent.ExecutionContext.global)
|
||||
|
||||
/**
|
||||
* Attempts to search for user in specified LDAP domains. Attempts all LDAP domains that are specified in order; in
|
||||
* the event that user details are not found in any of the domains, returns an error based on whether all
|
||||
@ -226,10 +230,16 @@ class LdapAuthenticator(
|
||||
}
|
||||
|
||||
def authenticate(username: String, password: String): Either[LdapException, LdapUserDetails] =
|
||||
findUserDetails(searchBase, username, authenticator.authenticate(_, username, password), true)
|
||||
// Need to check domains here due to recursive nature of findUserDetails
|
||||
if (searchBase.isEmpty) Left(NoLdapSearchDomainsConfigured())
|
||||
else
|
||||
findUserDetails(searchBase, username, authenticator.authenticate(_, username, password), true)
|
||||
|
||||
def lookup(username: String): Either[LdapException, LdapUserDetails] =
|
||||
findUserDetails(searchBase, username, authenticator.lookup(_, username, serviceAccount), true)
|
||||
// Need to check domains here due to recursive nature of findUserDetails
|
||||
if (searchBase.isEmpty) Left(NoLdapSearchDomainsConfigured())
|
||||
else
|
||||
findUserDetails(searchBase, username, authenticator.lookup(_, username, serviceAccount), true)
|
||||
|
||||
def healthCheck(): HealthCheck =
|
||||
IO {
|
||||
@ -240,11 +250,24 @@ class LdapAuthenticator(
|
||||
case _ => ().asRight
|
||||
}
|
||||
}.asHealthCheck
|
||||
|
||||
// List[User] => List[Either[LdapException, LdapUserDetails]] => List[User]
|
||||
def getUsersNotInLdap(users: List[User]): IO[List[User]] =
|
||||
users
|
||||
.map { u =>
|
||||
IO(lookup(u.userName)).map {
|
||||
case Left(_: UserDoesNotExistException) => Some(u) // Only grab users that do not exist
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
.parSequence
|
||||
.map(_.flatten)
|
||||
}
|
||||
|
||||
trait Authenticator {
|
||||
def authenticate(username: String, password: String): Either[LdapException, LdapUserDetails]
|
||||
def lookup(username: String): Either[LdapException, LdapUserDetails]
|
||||
def getUsersNotInLdap(usernames: List[User]): IO[List[User]]
|
||||
def healthCheck(): HealthCheck
|
||||
}
|
||||
|
||||
@ -282,6 +305,9 @@ class TestAuthenticator(authenticator: Authenticator) extends Authenticator {
|
||||
}
|
||||
|
||||
def healthCheck(): HealthCheck = authenticator.healthCheck()
|
||||
|
||||
def getUsersNotInLdap(users: List[User]): IO[List[User]] =
|
||||
authenticator.getUsersNotInLdap(users)
|
||||
}
|
||||
|
||||
case class LdapSearchDomain(organization: String, domainName: String)
|
||||
|
@ -26,6 +26,7 @@ import pureconfig.module.catseffect.loadConfigF
|
||||
import vinyldns.core.repository.DataStoreConfig
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.concurrent.duration._
|
||||
|
||||
// $COVERAGE-OFF$
|
||||
class Settings(private val config: Configuration) {
|
||||
@ -42,6 +43,13 @@ class Settings(private val config: Configuration) {
|
||||
val ldapSecurityAuthentication: String = config.get[String]("LDAP.context.securityAuthentication")
|
||||
val ldapProviderUrl: URI = new URI(config.get[String]("LDAP.context.providerUrl"))
|
||||
|
||||
val ldapSyncEnabled: Boolean =
|
||||
config.getOptional[Boolean]("LDAP.user-sync.enabled").getOrElse(false)
|
||||
val ldapSyncPollingInterval: FiniteDuration = config
|
||||
.getOptional[Int]("LDAP.user-sync.hours-polling-interval")
|
||||
.getOrElse(24)
|
||||
.hours
|
||||
|
||||
val portalTestLogin: Boolean = config.getOptional[Boolean]("portal.test_login").getOrElse(false)
|
||||
|
||||
val dataStoreConfigs: IO[List[DataStoreConfig]] =
|
||||
|
@ -16,14 +16,17 @@
|
||||
|
||||
package controllers
|
||||
|
||||
import cats.effect.IO
|
||||
import cats.effect.{ContextShift, IO}
|
||||
import cats.implicits._
|
||||
import javax.inject.{Inject, Singleton}
|
||||
import org.joda.time.DateTime
|
||||
import vinyldns.core.domain.membership.{User, UserChange, UserChangeRepository, UserRepository}
|
||||
import vinyldns.core.domain.membership._
|
||||
|
||||
@Singleton
|
||||
class UserAccountAccessor @Inject()(users: UserRepository, changes: UserChangeRepository) {
|
||||
|
||||
implicit val cs: ContextShift[IO] = IO.contextShift(scala.concurrent.ExecutionContext.global)
|
||||
|
||||
/**
|
||||
* Lookup a user in the store. Using identifier as the user id and/or name
|
||||
*
|
||||
@ -53,4 +56,21 @@ class UserAccountAccessor @Inject()(users: UserRepository, changes: UserChangeRe
|
||||
|
||||
def getUserByKey(key: String): IO[Option[User]] =
|
||||
users.getUserByAccessKey(key)
|
||||
|
||||
def getAllUsers: IO[List[User]] =
|
||||
users.getAllUsers
|
||||
|
||||
def lockUsers(usersToLock: List[User]): IO[List[User]] = {
|
||||
val currentTime = DateTime.now
|
||||
for {
|
||||
lockedUsers <- users.save(usersToLock.map(_.copy(lockStatus = LockStatus.Locked)))
|
||||
_ <- usersToLock
|
||||
.zip(lockedUsers)
|
||||
.map {
|
||||
case (oldUser, newUser) =>
|
||||
changes.save(UserChange.UpdateUser(newUser, "system", currentTime, oldUser))
|
||||
}
|
||||
.parSequence
|
||||
} yield lockedUsers
|
||||
}
|
||||
}
|
||||
|
@ -18,8 +18,10 @@ package controllers.repository
|
||||
|
||||
import vinyldns.core.domain.membership.{UserChangeRepository, UserRepository}
|
||||
import vinyldns.core.repository.DataAccessor
|
||||
import vinyldns.core.task.TaskRepository
|
||||
|
||||
final case class PortalDataAccessor(
|
||||
userRepository: UserRepository,
|
||||
userChangeRepository: UserChangeRepository)
|
||||
userChangeRepository: UserChangeRepository,
|
||||
taskRepository: TaskRepository)
|
||||
extends DataAccessor
|
||||
|
@ -22,6 +22,7 @@ import vinyldns.core.domain.membership.{UserChangeRepository, UserRepository}
|
||||
import vinyldns.core.repository.DataStoreLoader.getRepoOf
|
||||
import vinyldns.core.repository.RepositoryName._
|
||||
import vinyldns.core.repository.{DataAccessorProvider, DataStore, DataStoreConfig}
|
||||
import vinyldns.core.task.TaskRepository
|
||||
|
||||
// $COVERAGE-OFF$
|
||||
object PortalDataAccessorProvider extends DataAccessorProvider[PortalDataAccessor] {
|
||||
@ -32,7 +33,8 @@ object PortalDataAccessorProvider extends DataAccessorProvider[PortalDataAccesso
|
||||
dataStores: List[(DataStoreConfig, DataStore)]): ValidatedNel[String, PortalDataAccessor] =
|
||||
(
|
||||
getRepoOf[UserRepository](dataStores, user),
|
||||
getRepoOf[UserChangeRepository](dataStores, userChange)
|
||||
getRepoOf[UserChangeRepository](dataStores, userChange),
|
||||
getRepoOf[TaskRepository](dataStores, task)
|
||||
).mapN(PortalDataAccessor)
|
||||
}
|
||||
// $COVERAGE-ON$
|
||||
|
@ -32,20 +32,26 @@ package modules
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import cats.effect.{ContextShift, IO, Timer}
|
||||
import com.google.inject.AbstractModule
|
||||
import controllers._
|
||||
import controllers.repository.{PortalDataAccessor, PortalDataAccessorProvider}
|
||||
import play.api.{Configuration, Environment}
|
||||
import tasks.UserSyncTask
|
||||
import vinyldns.core.crypto.CryptoAlgebra
|
||||
import vinyldns.core.domain.membership.{UserChangeRepository, UserRepository}
|
||||
import vinyldns.core.health.HealthService
|
||||
import vinyldns.core.repository.DataStoreLoader
|
||||
import vinyldns.core.task.{TaskRepository, TaskScheduler}
|
||||
|
||||
// $COVERAGE-OFF$
|
||||
class VinylDNSModule(environment: Environment, configuration: Configuration)
|
||||
extends AbstractModule {
|
||||
|
||||
val settings = new Settings(configuration)
|
||||
implicit val t: Timer[IO] = IO.timer(scala.concurrent.ExecutionContext.global)
|
||||
implicit val cs: ContextShift[IO] =
|
||||
IO.contextShift(scala.concurrent.ExecutionContext.Implicits.global)
|
||||
|
||||
def configure(): Unit = {
|
||||
val startApp = for {
|
||||
@ -56,12 +62,28 @@ class VinylDNSModule(environment: Environment, configuration: Configuration)
|
||||
.loadAll[PortalDataAccessor](repoConfigs, crypto, PortalDataAccessorProvider)
|
||||
auth = authenticator()
|
||||
healthService = new HealthService(auth.healthCheck() :: loaderResponse.healthChecks)
|
||||
repositories = loaderResponse.accessor
|
||||
_ <- if (settings.ldapSyncEnabled) {
|
||||
TaskScheduler
|
||||
.schedule(
|
||||
new UserSyncTask(
|
||||
new UserAccountAccessor(
|
||||
repositories.userRepository,
|
||||
repositories.userChangeRepository),
|
||||
auth,
|
||||
settings.ldapSyncPollingInterval),
|
||||
repositories.taskRepository
|
||||
)
|
||||
.compile
|
||||
.drain
|
||||
.start
|
||||
} else IO.unit
|
||||
} yield {
|
||||
val repositories = loaderResponse.accessor
|
||||
bind(classOf[CryptoAlgebra]).toInstance(crypto)
|
||||
bind(classOf[Authenticator]).toInstance(auth)
|
||||
bind(classOf[UserRepository]).toInstance(repositories.userRepository)
|
||||
bind(classOf[UserChangeRepository]).toInstance(repositories.userChangeRepository)
|
||||
bind(classOf[TaskRepository]).toInstance(repositories.taskRepository)
|
||||
bind(classOf[HealthService]).toInstance(healthService)
|
||||
}
|
||||
|
||||
|
47
modules/portal/app/tasks/UserSyncTask.scala
Normal file
47
modules/portal/app/tasks/UserSyncTask.scala
Normal file
@ -0,0 +1,47 @@
|
||||
/*
|
||||
* 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 tasks
|
||||
|
||||
import cats.effect.IO
|
||||
import controllers.{Authenticator, UserAccountAccessor}
|
||||
import org.slf4j.{Logger, LoggerFactory}
|
||||
import vinyldns.core.domain.membership.LockStatus
|
||||
import vinyldns.core.task.Task
|
||||
|
||||
import scala.concurrent.duration._
|
||||
|
||||
class UserSyncTask(
|
||||
userAccountAccessor: UserAccountAccessor,
|
||||
authenticator: Authenticator,
|
||||
val runEvery: FiniteDuration = 24.hours,
|
||||
val timeout: FiniteDuration = 24.hours)
|
||||
extends Task {
|
||||
val name: String = "user_sync"
|
||||
private val logger: Logger = LoggerFactory.getLogger("UserSyncTask")
|
||||
|
||||
def run(): IO[Unit] = {
|
||||
logger.error("Initiating user sync")
|
||||
for {
|
||||
allUsers <- userAccountAccessor.getAllUsers
|
||||
activeUsers = allUsers.filter(u => u.lockStatus != LockStatus.Locked && !u.isTest)
|
||||
nonActiveUsers <- authenticator.getUsersNotInLdap(activeUsers)
|
||||
lockedUsers <- userAccountAccessor.lockUsers(nonActiveUsers)
|
||||
_ <- IO(logger.error(s"""usersLocked="${lockedUsers
|
||||
.map(_.userName)}"; userLockCount="${lockedUsers.size}" """))
|
||||
} yield ()
|
||||
}
|
||||
}
|
@ -13,6 +13,9 @@ mysql {
|
||||
user {
|
||||
# no additional settings for now
|
||||
}
|
||||
task {
|
||||
# no additional settings for now
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -25,6 +25,8 @@ import org.specs2.mock.mockito.ArgumentCapture
|
||||
import org.specs2.mutable.Specification
|
||||
import play.api.{Configuration, Environment}
|
||||
import vinyldns.core.health.HealthCheck._
|
||||
import vinyldns.core.domain.membership.User
|
||||
import vinyldns.core.health.HealthCheck.HealthCheckError
|
||||
|
||||
class LdapAuthenticatorSpec extends Specification with Mockito {
|
||||
|
||||
@ -62,6 +64,7 @@ class LdapAuthenticatorSpec extends Specification with Mockito {
|
||||
|
||||
val testDomain1 = LdapSearchDomain("someDomain", "DC=test,DC=test,DC=com")
|
||||
val testDomain2 = LdapSearchDomain("anotherDomain", "DC=test,DC=com")
|
||||
val nonexistentUser = User("does-not-exist", "accessKey", "secretKey")
|
||||
|
||||
"LdapAuthenticator" should {
|
||||
"apply method must create an LDAP Authenticator" in {
|
||||
@ -164,6 +167,13 @@ class LdapAuthenticatorSpec extends Specification with Mockito {
|
||||
response must beLeft
|
||||
}
|
||||
}
|
||||
|
||||
"return an error if no LDAP search domains are provided" in {
|
||||
val noDomainsLdapAuthenticator =
|
||||
new LdapAuthenticator(List(), mock[LdapByDomainAuthenticator], mock[ServiceAccount])
|
||||
|
||||
noDomainsLdapAuthenticator.authenticate("someUserName", "somePassword") must beLeft
|
||||
}
|
||||
}
|
||||
".lookup" should {
|
||||
"lookup first with 1st domain" in {
|
||||
@ -186,6 +196,13 @@ class LdapAuthenticatorSpec extends Specification with Mockito {
|
||||
"and return details if authenticated" in {
|
||||
response must beRight
|
||||
}
|
||||
|
||||
"return an error if no LDAP search domains are provided" in {
|
||||
val noDomainsLdapAuthenticator =
|
||||
new LdapAuthenticator(List(), mock[LdapByDomainAuthenticator], mock[ServiceAccount])
|
||||
|
||||
noDomainsLdapAuthenticator.lookup("someUserName") must beLeft
|
||||
}
|
||||
}
|
||||
|
||||
"lookup with 2nd domain if 1st fails with UserDoesNotExistException" in {
|
||||
@ -316,6 +333,45 @@ class LdapAuthenticatorSpec extends Specification with Mockito {
|
||||
response should beRight[Unit]
|
||||
}
|
||||
}
|
||||
".getUsersNotInLdap" should {
|
||||
"return a list of users not found in LDAP" in {
|
||||
val byDomainAuthenticator = mock[LdapByDomainAuthenticator]
|
||||
val serviceAccount = mock[ServiceAccount]
|
||||
byDomainAuthenticator
|
||||
.lookup(testDomain1, "does-not-exist", serviceAccount)
|
||||
.returns(Left(UserDoesNotExistException("does not exist")))
|
||||
byDomainAuthenticator
|
||||
.lookup(testDomain2, "does-not-exist", serviceAccount)
|
||||
.returns(Left(UserDoesNotExistException("does not exist")))
|
||||
byDomainAuthenticator
|
||||
.lookup(testDomain1, "existing-user", serviceAccount)
|
||||
.returns(Right(LdapUserDetails("", "", None, None, None)))
|
||||
byDomainAuthenticator
|
||||
.lookup(testDomain1, "another-existing-user", serviceAccount)
|
||||
.returns(Left(UserDoesNotExistException("does not exist")))
|
||||
byDomainAuthenticator
|
||||
.lookup(testDomain2, "another-existing-user", serviceAccount)
|
||||
.returns(Right(LdapUserDetails("", "", None, None, None)))
|
||||
byDomainAuthenticator
|
||||
.lookup(testDomain1, serviceAccount.name, serviceAccount)
|
||||
.returns(Right(LdapUserDetails("", "", None, None, None)))
|
||||
val authenticator =
|
||||
new LdapAuthenticator(
|
||||
List(testDomain1, testDomain2),
|
||||
byDomainAuthenticator,
|
||||
serviceAccount
|
||||
)
|
||||
|
||||
authenticator
|
||||
.getUsersNotInLdap(
|
||||
List(
|
||||
nonexistentUser,
|
||||
nonexistentUser.copy(userName = "existing-user"),
|
||||
nonexistentUser.copy(userName = "another-existing-user")))
|
||||
.unsafeRunSync() must
|
||||
beEqualTo(List(nonexistentUser))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
"LdapByDomainAuthenticator" should {
|
||||
@ -481,15 +537,25 @@ class LdapAuthenticatorSpec extends Specification with Mockito {
|
||||
.returns(Left(UserDoesNotExistException("should not be here")))
|
||||
|
||||
val underTest = new TestAuthenticator(mockLdapAuth)
|
||||
val result = underTest.authenticate("testuser", "testpassword")
|
||||
val testUserLookup = underTest.authenticate("testuser", "testpassword")
|
||||
|
||||
result must beRight(
|
||||
testUserLookup must beRight(
|
||||
LdapUserDetails(
|
||||
"O=test,OU=testdata,CN=testuser",
|
||||
"testuser",
|
||||
Some("test@test.test"),
|
||||
Some("Test"),
|
||||
Some("User")))
|
||||
|
||||
val recordPagingUserLookup = underTest.lookup("recordPagingTestUser")
|
||||
|
||||
recordPagingUserLookup must beRight(
|
||||
LdapUserDetails(
|
||||
"O=test,OU=testdata,CN=recordPagingTestUser",
|
||||
"recordPagingTestUser",
|
||||
Some("test@test.test"),
|
||||
Some("Test"),
|
||||
Some("User")))
|
||||
there.were(noCallsTo(mockLdapAuth))
|
||||
}
|
||||
"authenticate the record paging test user" in {
|
||||
@ -529,15 +595,26 @@ class LdapAuthenticatorSpec extends Specification with Mockito {
|
||||
.returns(Left(UserDoesNotExistException("should not be here")))
|
||||
|
||||
val underTest = new TestAuthenticator(mockLdapAuth)
|
||||
val result = underTest.lookup("testuser")
|
||||
val testUserLookup = underTest.lookup("testuser")
|
||||
|
||||
result must beRight(
|
||||
testUserLookup must beRight(
|
||||
LdapUserDetails(
|
||||
"O=test,OU=testdata,CN=testuser",
|
||||
"testuser",
|
||||
Some("test@test.test"),
|
||||
Some("Test"),
|
||||
Some("User")))
|
||||
|
||||
val recordPagingUserLookup = underTest.lookup("recordPagingTestUser")
|
||||
|
||||
recordPagingUserLookup must beRight(
|
||||
LdapUserDetails(
|
||||
"O=test,OU=testdata,CN=recordPagingTestUser",
|
||||
"recordPagingTestUser",
|
||||
Some("test@test.test"),
|
||||
Some("Test"),
|
||||
Some("User")))
|
||||
|
||||
there.were(noCallsTo(mockLdapAuth))
|
||||
}
|
||||
"lookup the record paging test user" in {
|
||||
@ -570,6 +647,23 @@ class LdapAuthenticatorSpec extends Specification with Mockito {
|
||||
result must beRight(userDetails)
|
||||
there.was(one(mockLdapAuth).lookup("foo"))
|
||||
}
|
||||
"find non-existent users" in {
|
||||
val mockLdapAuth = mock[LdapAuthenticator]
|
||||
mockLdapAuth
|
||||
.getUsersNotInLdap(List(nonexistentUser))
|
||||
.returns(IO(List(nonexistentUser)))
|
||||
|
||||
val underTest = new TestAuthenticator(mockLdapAuth)
|
||||
underTest.getUsersNotInLdap(List(nonexistentUser)).unsafeRunSync() must beEqualTo(
|
||||
List(nonexistentUser))
|
||||
}
|
||||
"perform a health check" in {
|
||||
val mockLdapAuth = mock[LdapAuthenticator]
|
||||
mockLdapAuth.healthCheck().returns(IO(Right(())))
|
||||
val underTest = new TestAuthenticator(mockLdapAuth)
|
||||
|
||||
underTest.healthCheck().unsafeRunSync() must beRight(())
|
||||
}
|
||||
}
|
||||
"return a successful health check" in {
|
||||
val mockLdapAuth = mock[LdapAuthenticator]
|
||||
|
@ -51,15 +51,15 @@ class UserAccountAccessorSpec extends Specification with Mockito with BeforeEach
|
||||
protected def before: Any =
|
||||
org.mockito.Mockito.reset(mockRepo, mockChangeRepo)
|
||||
|
||||
"User Account Accessor" should {
|
||||
"Return the user when storing a user that does not exist already" in {
|
||||
"UserAccountAccessor" should {
|
||||
"return the user when storing a user that does not exist already" in {
|
||||
mockRepo.save(any[User]).returns(IO.pure(user))
|
||||
mockChangeRepo.save(any[UserChange]).returns(IO.pure(userLog))
|
||||
underTest.create(user).unsafeRunSync() must beEqualTo(user)
|
||||
there.was(one(mockChangeRepo).save(any[UserChange]))
|
||||
}
|
||||
|
||||
"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 newUser = user.copy(accessKey = "new-key", secretKey = "new-secret")
|
||||
mockRepo.save(any[User]).returns(IO.pure(newUser))
|
||||
mockChangeRepo.save(any[UserChange]).returns(IO.pure(userLog))
|
||||
@ -67,27 +67,45 @@ class UserAccountAccessorSpec extends Specification with Mockito with BeforeEach
|
||||
there.was(one(mockChangeRepo).save(any[UserChange]))
|
||||
}
|
||||
|
||||
"Return the user when retrieving a user that exists by name" in {
|
||||
"return the user when retrieving a user that exists by name" in {
|
||||
mockRepo.getUserByName(user.userName).returns(IO.pure(Some(user)))
|
||||
mockRepo.getUser(user.userName).returns(IO.pure(None))
|
||||
underTest.get("fbaggins").unsafeRunSync() must beSome(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 {
|
||||
mockRepo.getUserByName(user.id).returns(IO.pure(None))
|
||||
mockRepo.getUser(user.id).returns(IO.pure(Some(user)))
|
||||
underTest.get(user.id).unsafeRunSync() must beSome(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 {
|
||||
mockRepo.getUserByName(any[String]).returns(IO.pure(None))
|
||||
mockRepo.getUser(any[String]).returns(IO.pure(None))
|
||||
underTest.get("fbaggins").unsafeRunSync() must beNone
|
||||
}
|
||||
|
||||
"Return the user by access key" in {
|
||||
"return the user by access key" in {
|
||||
mockRepo.getUserByAccessKey(user.id).returns(IO.pure(Some(user)))
|
||||
underTest.getUserByKey(user.id).unsafeRunSync() must beSome(user)
|
||||
}
|
||||
|
||||
"return all users" in {
|
||||
val userList = List(user, user.copy(id = "user2", userName = "user2"))
|
||||
mockRepo.getAllUsers.returns(IO.pure(userList))
|
||||
underTest.getAllUsers.unsafeRunSync() must beEqualTo(userList)
|
||||
}
|
||||
|
||||
"lock specified users" in {
|
||||
val lockedUser = user.copy(lockStatus = LockStatus.Locked)
|
||||
val lockedUserChange = UserChange.UpdateUser(
|
||||
user.copy(lockStatus = LockStatus.Locked),
|
||||
"system",
|
||||
DateTime.now,
|
||||
user)
|
||||
mockRepo.save(List(lockedUser)).returns(IO(List(lockedUser)))
|
||||
mockChangeRepo.save(any[UserChange]).returns(IO(lockedUserChange))
|
||||
underTest.lockUsers(List(user)).unsafeRunSync() must beEqualTo(List(lockedUser))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
68
modules/portal/test/tasks/UserSyncTaskSpec.scala
Normal file
68
modules/portal/test/tasks/UserSyncTaskSpec.scala
Normal file
@ -0,0 +1,68 @@
|
||||
/*
|
||||
* 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 tasks
|
||||
|
||||
import cats.effect.IO
|
||||
import controllers.{Authenticator, UserAccountAccessor}
|
||||
import org.specs2.mock.Mockito
|
||||
import org.specs2.mutable.Specification
|
||||
import vinyldns.core.domain.membership._
|
||||
|
||||
class UserSyncTaskSpec extends Specification with Mockito {
|
||||
val notAuthUser: User = User("not-authorized", "accessKey", "secretKey")
|
||||
val lockedNotAuthUser: User = notAuthUser.copy(lockStatus = LockStatus.Locked)
|
||||
val mockAuthenticator: Authenticator = {
|
||||
val mockObject = mock[Authenticator]
|
||||
mockObject.getUsersNotInLdap(List(notAuthUser)).returns(IO(List(notAuthUser)))
|
||||
mockObject
|
||||
}
|
||||
|
||||
val mockUserAccountAccessor: UserAccountAccessor = {
|
||||
val mockObject = mock[UserAccountAccessor]
|
||||
mockObject.getAllUsers.returns(IO(List(notAuthUser)))
|
||||
mockObject
|
||||
.lockUsers(List(notAuthUser))
|
||||
.returns(IO(List(lockedNotAuthUser)))
|
||||
mockObject
|
||||
}
|
||||
|
||||
"SyncUserTask" should {
|
||||
"successfully lock unauthorized, non-test users" in {
|
||||
new UserSyncTask(mockUserAccountAccessor, mockAuthenticator)
|
||||
.run()
|
||||
.unsafeRunSync() must beEqualTo(())
|
||||
|
||||
there.was(one(mockUserAccountAccessor).lockUsers(List(notAuthUser)))
|
||||
}
|
||||
|
||||
"successfully process if no users are found" in {
|
||||
val mockAuth: Authenticator = mock[Authenticator]
|
||||
mockAuth.getUsersNotInLdap(List(notAuthUser)).returns(IO(Nil))
|
||||
|
||||
val mockUsers = mock[UserAccountAccessor]
|
||||
mockUsers
|
||||
.lockUsers(Nil)
|
||||
.returns(IO(Nil))
|
||||
|
||||
mockUsers.getAllUsers.returns(IO(List(notAuthUser)))
|
||||
|
||||
new UserSyncTask(mockUsers, mockAuth)
|
||||
.run()
|
||||
.unsafeRunSync() must beEqualTo(())
|
||||
}
|
||||
}
|
||||
}
|
@ -16,6 +16,7 @@ object Dependencies {
|
||||
lazy val awsV = "1.11.423"
|
||||
lazy val jaxbV = "2.3.0"
|
||||
lazy val ip4sV = "1.1.1"
|
||||
lazy val fs2V = "1.0.0"
|
||||
|
||||
lazy val apiDependencies = Seq(
|
||||
"com.typesafe.akka" %% "akka-http" % akkaHttpV,
|
||||
@ -38,7 +39,7 @@ object Dependencies {
|
||||
"org.scalikejdbc" %% "scalikejdbc-config" % scalikejdbcV,
|
||||
"org.scodec" %% "scodec-bits" % scodecV,
|
||||
"org.slf4j" % "slf4j-api" % "1.7.25",
|
||||
"co.fs2" %% "fs2-core" % "1.0.0",
|
||||
"co.fs2" %% "fs2-core" % fs2V,
|
||||
"com.github.pureconfig" %% "pureconfig" % pureConfigV,
|
||||
"com.github.pureconfig" %% "pureconfig-cats-effect" % pureConfigV,
|
||||
"io.prometheus" % "simpleclient_hotspot" % prometheusV,
|
||||
@ -110,6 +111,7 @@ object Dependencies {
|
||||
"com.typesafe.play" %% "play-ahc-ws" % playV,
|
||||
"com.typesafe.play" %% "play-specs2" % playV % "test",
|
||||
"com.nimbusds" % "oauth2-oidc-sdk" % "6.5",
|
||||
"com.nimbusds" % "nimbus-jose-jwt" % "7.0"
|
||||
"com.nimbusds" % "nimbus-jose-jwt" % "7.0",
|
||||
"co.fs2" %% "fs2-core" % fs2V
|
||||
)
|
||||
}
|
||||
|
Loading…
x
Reference in New Issue
Block a user