diff --git a/modules/portal/app/controllers/LdapAuthenticator.scala b/modules/portal/app/controllers/LdapAuthenticator.scala index ee7aa6cef..5dd251ace 100644 --- a/modules/portal/app/controllers/LdapAuthenticator.scala +++ b/modules/portal/app/controllers/LdapAuthenticator.scala @@ -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) diff --git a/modules/portal/app/controllers/Settings.scala b/modules/portal/app/controllers/Settings.scala index 96b6c2386..b50ad4faa 100644 --- a/modules/portal/app/controllers/Settings.scala +++ b/modules/portal/app/controllers/Settings.scala @@ -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]] = diff --git a/modules/portal/app/controllers/UserAccountAccessor.scala b/modules/portal/app/controllers/UserAccountAccessor.scala index 6355916e6..80314eb37 100644 --- a/modules/portal/app/controllers/UserAccountAccessor.scala +++ b/modules/portal/app/controllers/UserAccountAccessor.scala @@ -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 + } } diff --git a/modules/portal/app/controllers/repository/PortalDataAccessor.scala b/modules/portal/app/controllers/repository/PortalDataAccessor.scala index 935c2ce46..17cb4039c 100644 --- a/modules/portal/app/controllers/repository/PortalDataAccessor.scala +++ b/modules/portal/app/controllers/repository/PortalDataAccessor.scala @@ -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 diff --git a/modules/portal/app/controllers/repository/PortalDataAccessorProvider.scala b/modules/portal/app/controllers/repository/PortalDataAccessorProvider.scala index 5f709d5d4..fd8adacee 100644 --- a/modules/portal/app/controllers/repository/PortalDataAccessorProvider.scala +++ b/modules/portal/app/controllers/repository/PortalDataAccessorProvider.scala @@ -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$ diff --git a/modules/portal/app/modules/VinylDNSModule.scala b/modules/portal/app/modules/VinylDNSModule.scala index df3f92502..4555d1408 100644 --- a/modules/portal/app/modules/VinylDNSModule.scala +++ b/modules/portal/app/modules/VinylDNSModule.scala @@ -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) } diff --git a/modules/portal/app/tasks/UserSyncTask.scala b/modules/portal/app/tasks/UserSyncTask.scala new file mode 100644 index 000000000..ac011d219 --- /dev/null +++ b/modules/portal/app/tasks/UserSyncTask.scala @@ -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 () + } +} diff --git a/modules/portal/conf/application.conf b/modules/portal/conf/application.conf index d91e6808b..c618ce8b3 100644 --- a/modules/portal/conf/application.conf +++ b/modules/portal/conf/application.conf @@ -13,6 +13,9 @@ mysql { user { # no additional settings for now } + task { + # no additional settings for now + } } } diff --git a/modules/portal/test/controllers/LdapAuthenticatorSpec.scala b/modules/portal/test/controllers/LdapAuthenticatorSpec.scala index 0cd19e351..da29ece25 100644 --- a/modules/portal/test/controllers/LdapAuthenticatorSpec.scala +++ b/modules/portal/test/controllers/LdapAuthenticatorSpec.scala @@ -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] diff --git a/modules/portal/test/controllers/UserAccountAccessorSpec.scala b/modules/portal/test/controllers/UserAccountAccessorSpec.scala index f7b1025c2..ae9a64979 100644 --- a/modules/portal/test/controllers/UserAccountAccessorSpec.scala +++ b/modules/portal/test/controllers/UserAccountAccessorSpec.scala @@ -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)) + } } } diff --git a/modules/portal/test/tasks/UserSyncTaskSpec.scala b/modules/portal/test/tasks/UserSyncTaskSpec.scala new file mode 100644 index 000000000..09047e114 --- /dev/null +++ b/modules/portal/test/tasks/UserSyncTaskSpec.scala @@ -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(()) + } + } +} diff --git a/project/Dependencies.scala b/project/Dependencies.scala index 4630e6b6f..855da606a 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -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 ) }