mirror of
https://github.com/VinylDNS/vinyldns
synced 2025-08-22 10:10:12 +00:00
Make dynamo repos startup within IO (#187)
* IO startup for dynamodb stores (rather than unsafe throws) * Update unit and integration tests in the dynamodb module * update api module where dependent on dnamodb
This commit is contained in:
parent
79f2c967b8
commit
475f0f2530
@ -202,7 +202,7 @@ lazy val root = (project in file(".")).enablePlugins(AutomateHeaderPlugin)
|
||||
(scalastyleConfig in Test) := baseDirectory.value / "scalastyle-test-config.xml",
|
||||
(scalastyleConfig in IntegrationTest) := baseDirectory.value / "scalastyle-test-config.xml"
|
||||
)
|
||||
.aggregate(core, api, portal)
|
||||
.aggregate(core, api, portal, dynamodb)
|
||||
|
||||
lazy val coreBuildSettings = Seq(
|
||||
name := "core",
|
||||
|
@ -16,15 +16,10 @@
|
||||
|
||||
package vinyldns.api
|
||||
|
||||
import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClient
|
||||
import org.slf4j.LoggerFactory
|
||||
import vinyldns.dynamodb.repository.{DynamoDBHelper, DynamoDBIntegrationSpec}
|
||||
import vinyldns.dynamodb.repository.{DynamoDBDataStoreSettings, DynamoDBIntegrationSpec}
|
||||
|
||||
trait DynamoDBApiIntegrationSpec extends DynamoDBIntegrationSpec {
|
||||
|
||||
override val dynamoClient: AmazonDynamoDBClient = getDynamoClient(19000)
|
||||
|
||||
override val dynamoDBHelper: DynamoDBHelper =
|
||||
new DynamoDBHelper(dynamoClient, LoggerFactory.getLogger("DynamoDBApiIntegrationSpec"))
|
||||
override val dynamoIntegrationConfig: DynamoDBDataStoreSettings = getDynamoConfig(19000)
|
||||
|
||||
}
|
||||
|
@ -17,7 +17,6 @@
|
||||
package vinyldns.api.domain.record
|
||||
|
||||
import cats.effect._
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import org.joda.time.DateTime
|
||||
import org.scalatest.Matchers
|
||||
import org.scalatest.concurrent.PatienceConfiguration
|
||||
@ -31,7 +30,7 @@ import vinyldns.core.domain.membership.{Group, User, UserRepository}
|
||||
import vinyldns.core.domain.record.RecordType._
|
||||
import vinyldns.core.domain.zone.{Zone, ZoneRepository, ZoneStatus}
|
||||
import vinyldns.api.engine.sqs.TestSqsService
|
||||
import vinyldns.dynamodb.repository.DynamoDBRecordSetRepository
|
||||
import vinyldns.dynamodb.repository.{DynamoDBRecordSetRepository, DynamoDBRepositorySettings}
|
||||
import vinyldns.api.repository.mysql.TestMySqlInstance
|
||||
import vinyldns.core.domain.record._
|
||||
|
||||
@ -48,20 +47,7 @@ class RecordSetServiceIntegrationSpec
|
||||
|
||||
private val recordSetTable = "recordSetTest"
|
||||
|
||||
private val liveTestConfig = ConfigFactory.parseString(s"""
|
||||
| recordSet {
|
||||
| # use the dummy store, this should only be used local
|
||||
| dummy = true
|
||||
|
|
||||
| dynamo {
|
||||
| tableName = "$recordSetTable"
|
||||
| provisionedReads=30
|
||||
| provisionedWrites=30
|
||||
| }
|
||||
| }
|
||||
""".stripMargin)
|
||||
|
||||
private val recordSetStoreConfig = liveTestConfig.getConfig("recordSet")
|
||||
private val recordSetStoreConfig = DynamoDBRepositorySettings(s"$recordSetTable", 30, 30)
|
||||
|
||||
private val timeout = PatienceConfiguration.Timeout(Span(10, Seconds))
|
||||
|
||||
@ -159,7 +145,8 @@ class RecordSetServiceIntegrationSpec
|
||||
adminGroupId = group.id)
|
||||
|
||||
def setup(): Unit = {
|
||||
recordSetRepo = new DynamoDBRecordSetRepository(recordSetStoreConfig, dynamoDBHelper)
|
||||
recordSetRepo =
|
||||
DynamoDBRecordSetRepository(recordSetStoreConfig, dynamoIntegrationConfig).unsafeRunSync()
|
||||
zoneRepo = TestMySqlInstance.zoneRepository
|
||||
|
||||
List(zone, zoneTestNameConflicts, zoneTestAddRecords).map(z => waitForSuccess(zoneRepo.save(z)))
|
||||
|
@ -17,7 +17,6 @@
|
||||
package vinyldns.api.domain.zone
|
||||
|
||||
import cats.effect._
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import org.joda.time.DateTime
|
||||
import org.scalatest.concurrent.PatienceConfiguration
|
||||
import org.scalatest.mockito.MockitoSugar
|
||||
@ -29,7 +28,7 @@ import vinyldns.core.domain.auth.AuthPrincipal
|
||||
import vinyldns.core.domain.membership.{Group, GroupRepository, User, UserRepository}
|
||||
import vinyldns.core.domain.record._
|
||||
import vinyldns.api.engine.sqs.TestSqsService
|
||||
import vinyldns.dynamodb.repository.DynamoDBRecordSetRepository
|
||||
import vinyldns.dynamodb.repository.{DynamoDBRecordSetRepository, DynamoDBRepositorySettings}
|
||||
import vinyldns.api.repository.mysql.TestMySqlInstance
|
||||
import vinyldns.core.domain.zone._
|
||||
|
||||
@ -45,20 +44,7 @@ class ZoneServiceIntegrationSpec
|
||||
|
||||
private val recordSetTable = "recordSetTest"
|
||||
|
||||
private val liveTestConfig = ConfigFactory.parseString(s"""
|
||||
| recordSet {
|
||||
| # use the dummy store, this should only be used local
|
||||
| dummy = true
|
||||
|
|
||||
| dynamo {
|
||||
| tableName = "$recordSetTable"
|
||||
| provisionedReads=30
|
||||
| provisionedWrites=30
|
||||
| }
|
||||
| }
|
||||
""".stripMargin)
|
||||
|
||||
private val recordSetStoreConfig = liveTestConfig.getConfig("recordSet")
|
||||
private val recordSetStoreConfig = DynamoDBRepositorySettings(s"$recordSetTable", 30, 30)
|
||||
|
||||
private val timeout = PatienceConfiguration.Timeout(Span(10, Seconds))
|
||||
|
||||
@ -110,7 +96,8 @@ class ZoneServiceIntegrationSpec
|
||||
private val changeSetA = ChangeSet(RecordSetChangeGenerator.forAdd(testRecordA, zone))
|
||||
|
||||
def setup(): Unit = {
|
||||
recordSetRepo = new DynamoDBRecordSetRepository(recordSetStoreConfig, dynamoDBHelper)
|
||||
recordSetRepo =
|
||||
DynamoDBRecordSetRepository(recordSetStoreConfig, dynamoIntegrationConfig).unsafeRunSync()
|
||||
zoneRepo = TestMySqlInstance.zoneRepository
|
||||
|
||||
waitForSuccess(zoneRepo.save(zone))
|
||||
|
@ -19,7 +19,7 @@ package vinyldns.api.engine
|
||||
import java.util.concurrent.Executors
|
||||
|
||||
import cats.effect.IO
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import cats.implicits._
|
||||
import fs2.{Scheduler, Stream}
|
||||
import org.joda.time.DateTime
|
||||
import org.scalatest.concurrent.Eventually
|
||||
@ -33,6 +33,7 @@ import vinyldns.api.engine.sqs.SqsConnection
|
||||
import vinyldns.dynamodb.repository.{
|
||||
DynamoDBRecordChangeRepository,
|
||||
DynamoDBRecordSetRepository,
|
||||
DynamoDBRepositorySettings,
|
||||
DynamoDBZoneChangeRepository
|
||||
}
|
||||
import vinyldns.api.repository.mysql.TestMySqlInstance
|
||||
@ -56,42 +57,9 @@ class ZoneCommandHandlerIntegrationSpec
|
||||
private val recordSetTable = "recordSetTest"
|
||||
private val recordChangeTable = "recordChangeTest"
|
||||
|
||||
private val liveTestConfig = ConfigFactory.parseString(s"""
|
||||
| zoneChanges {
|
||||
| # use the dummy store, this should only be used local
|
||||
| dummy = true
|
||||
|
|
||||
| dynamo {
|
||||
| tableName = "$zoneChangeTable"
|
||||
| provisionedReads=30
|
||||
| provisionedWrites=30
|
||||
| }
|
||||
| }
|
||||
| recordSet {
|
||||
| # use the dummy store, this should only be used local
|
||||
| dummy = true
|
||||
|
|
||||
| dynamo {
|
||||
| tableName = "$recordSetTable"
|
||||
| provisionedReads=30
|
||||
| provisionedWrites=30
|
||||
| }
|
||||
| }
|
||||
| recordChange {
|
||||
| # use the dummy store, this should only be used local
|
||||
| dummy = true
|
||||
|
|
||||
| dynamo {
|
||||
| tableName = "$recordChangeTable"
|
||||
| provisionedReads=30
|
||||
| provisionedWrites=30
|
||||
| }
|
||||
| }
|
||||
""".stripMargin)
|
||||
|
||||
private val zoneChangeStoreConfig = liveTestConfig.getConfig("zoneChanges")
|
||||
private val recordSetStoreConfig = liveTestConfig.getConfig("recordSet")
|
||||
private val recordChangeStoreConfig = liveTestConfig.getConfig("recordChange")
|
||||
private val zoneChangeStoreConfig = DynamoDBRepositorySettings(s"$zoneChangeTable", 30, 30)
|
||||
private val recordSetStoreConfig = DynamoDBRepositorySettings(s"$recordSetTable", 30, 30)
|
||||
private val recordChangeStoreConfig = DynamoDBRepositorySettings(s"$recordChangeTable", 30, 30)
|
||||
|
||||
private implicit val defaultPatience: PatienceConfig =
|
||||
PatienceConfig(timeout = Span(5, Seconds), interval = Span(500, Millis))
|
||||
@ -151,9 +119,15 @@ class ZoneCommandHandlerIntegrationSpec
|
||||
}
|
||||
|
||||
def setup(): Unit = {
|
||||
recordChangeRepo = new DynamoDBRecordChangeRepository(recordChangeStoreConfig, dynamoDBHelper)
|
||||
recordSetRepo = new DynamoDBRecordSetRepository(recordSetStoreConfig, dynamoDBHelper)
|
||||
zoneChangeRepo = new DynamoDBZoneChangeRepository(zoneChangeStoreConfig, dynamoDBHelper)
|
||||
val repos = (
|
||||
DynamoDBRecordChangeRepository(recordChangeStoreConfig, dynamoIntegrationConfig),
|
||||
DynamoDBRecordSetRepository(recordSetStoreConfig, dynamoIntegrationConfig),
|
||||
DynamoDBZoneChangeRepository(zoneChangeStoreConfig, dynamoIntegrationConfig)
|
||||
).parTupled.unsafeRunSync()
|
||||
|
||||
recordChangeRepo = repos._1
|
||||
recordSetRepo = repos._2
|
||||
zoneChangeRepo = repos._3
|
||||
zoneRepo = TestMySqlInstance.zoneRepository
|
||||
batchChangeRepo = TestMySqlInstance.batchChangeRepository
|
||||
sqsConn = SqsConnection()
|
||||
|
@ -33,13 +33,13 @@ import vinyldns.api.domain.zone._
|
||||
import vinyldns.api.engine.ProductionZoneCommandHandler
|
||||
import vinyldns.api.engine.sqs.{SqsCommandBus, SqsConnection}
|
||||
import vinyldns.dynamodb.repository._
|
||||
import vinyldns.api.repository.{DataStoreStartupError, TestDataLoader}
|
||||
import vinyldns.api.repository.TestDataLoader
|
||||
import vinyldns.api.repository.mysql.MySqlDataStoreProvider
|
||||
import vinyldns.api.route.{HealthService, VinylDNSService}
|
||||
import vinyldns.core.VinylDNSMetrics
|
||||
import vinyldns.core.domain.batch.BatchChangeRepository
|
||||
import vinyldns.core.domain.zone.ZoneRepository
|
||||
import vinyldns.core.repository.RepositoryName
|
||||
import vinyldns.core.repository.{DataStoreStartupError, RepositoryName}
|
||||
|
||||
import scala.concurrent.{ExecutionContext, Future}
|
||||
import scala.io.{Codec, Source}
|
||||
@ -76,30 +76,27 @@ object Boot extends App {
|
||||
.get[BatchChangeRepository](RepositoryName.batchChange)
|
||||
.toRight[Throwable](DataStoreStartupError("Missing zone repository")))
|
||||
// TODO this also will all be removed with dynamic loading
|
||||
userRepo <- IO(
|
||||
DynamoDBUserRepository(VinylDNSConfig.usersStoreConfig, VinylDNSConfig.dynamoConfig))
|
||||
groupRepo <- IO(
|
||||
DynamoDBGroupRepository(VinylDNSConfig.groupsStoreConfig, VinylDNSConfig.dynamoConfig))
|
||||
membershipRepo <- IO(
|
||||
DynamoDBMembershipRepository(
|
||||
userRepo <- DynamoDBUserRepository(
|
||||
VinylDNSConfig.usersStoreConfig,
|
||||
VinylDNSConfig.dynamoConfig)
|
||||
groupRepo <- DynamoDBGroupRepository(
|
||||
VinylDNSConfig.groupsStoreConfig,
|
||||
VinylDNSConfig.dynamoConfig)
|
||||
membershipRepo <- DynamoDBMembershipRepository(
|
||||
VinylDNSConfig.membershipStoreConfig,
|
||||
VinylDNSConfig.dynamoConfig))
|
||||
groupChangeRepo <- IO(
|
||||
DynamoDBGroupChangeRepository(
|
||||
VinylDNSConfig.dynamoConfig)
|
||||
groupChangeRepo <- DynamoDBGroupChangeRepository(
|
||||
VinylDNSConfig.groupChangesStoreConfig,
|
||||
VinylDNSConfig.dynamoConfig))
|
||||
recordSetRepo <- IO(
|
||||
DynamoDBRecordSetRepository(
|
||||
VinylDNSConfig.dynamoConfig)
|
||||
recordSetRepo <- DynamoDBRecordSetRepository(
|
||||
VinylDNSConfig.recordSetStoreConfig,
|
||||
VinylDNSConfig.dynamoConfig))
|
||||
recordChangeRepo <- IO(
|
||||
DynamoDBRecordChangeRepository(
|
||||
VinylDNSConfig.dynamoConfig)
|
||||
recordChangeRepo <- DynamoDBRecordChangeRepository(
|
||||
VinylDNSConfig.recordChangeStoreConfig,
|
||||
VinylDNSConfig.dynamoConfig))
|
||||
zoneChangeRepo <- IO(
|
||||
DynamoDBZoneChangeRepository(
|
||||
VinylDNSConfig.dynamoConfig)
|
||||
zoneChangeRepo <- DynamoDBZoneChangeRepository(
|
||||
VinylDNSConfig.zoneChangeStoreConfig,
|
||||
VinylDNSConfig.dynamoConfig))
|
||||
VinylDNSConfig.dynamoConfig)
|
||||
_ <- TestDataLoader.loadTestData(userRepo)
|
||||
sqsConfig <- IO(VinylDNSConfig.sqsConfig)
|
||||
sqsConnection <- IO(SqsConnection(sqsConfig))
|
||||
|
@ -18,28 +18,34 @@ package vinyldns.api
|
||||
|
||||
import akka.actor.ActorSystem
|
||||
import com.typesafe.config.{Config, ConfigFactory}
|
||||
import pureconfig.{CamelCase, ConfigFieldMapping, ProductHint}
|
||||
import vinyldns.api.VinylDNSConfig.vinyldnsConfig
|
||||
import vinyldns.api.crypto.Crypto
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.util.matching.Regex
|
||||
import vinyldns.core.domain.zone.ZoneConnection
|
||||
import vinyldns.core.repository.DataStoreConfig
|
||||
import vinyldns.dynamodb.repository.{DynamoDBDataStoreSettings, DynamoDBRepositorySettings}
|
||||
|
||||
object VinylDNSConfig {
|
||||
|
||||
lazy val config: Config = ConfigFactory.load()
|
||||
lazy val vinyldnsConfig: Config = config.getConfig("vinyldns")
|
||||
lazy val dynamoConfig: Config = vinyldnsConfig.getConfig("dynamo")
|
||||
|
||||
lazy val dynamoConfig = DynamoConfig.dynamoConfig
|
||||
lazy val zoneChangeStoreConfig: DynamoDBRepositorySettings = DynamoConfig.zoneChangeStoreConfig
|
||||
lazy val recordSetStoreConfig: DynamoDBRepositorySettings = DynamoConfig.recordSetStoreConfig
|
||||
lazy val recordChangeStoreConfig: DynamoDBRepositorySettings =
|
||||
DynamoConfig.recordChangeStoreConfig
|
||||
lazy val usersStoreConfig: DynamoDBRepositorySettings = DynamoConfig.usersStoreConfig
|
||||
lazy val groupsStoreConfig: DynamoDBRepositorySettings = DynamoConfig.groupsStoreConfig
|
||||
lazy val groupChangesStoreConfig: DynamoDBRepositorySettings =
|
||||
DynamoConfig.groupChangesStoreConfig
|
||||
lazy val membershipStoreConfig: DynamoDBRepositorySettings = DynamoConfig.membershipStoreConfig
|
||||
|
||||
lazy val restConfig: Config = vinyldnsConfig.getConfig("rest")
|
||||
lazy val monitoringConfig: Config = vinyldnsConfig.getConfig("monitoring")
|
||||
lazy val accountStoreConfig: Config = vinyldnsConfig.getConfig("accounts")
|
||||
lazy val zoneChangeStoreConfig: Config = vinyldnsConfig.getConfig("zoneChanges")
|
||||
lazy val recordSetStoreConfig: Config = vinyldnsConfig.getConfig("recordSet")
|
||||
lazy val recordChangeStoreConfig: Config = vinyldnsConfig.getConfig("recordChange")
|
||||
lazy val usersStoreConfig: Config = vinyldnsConfig.getConfig("users")
|
||||
lazy val groupsStoreConfig: Config = vinyldnsConfig.getConfig("groups")
|
||||
lazy val groupChangesStoreConfig: Config = vinyldnsConfig.getConfig("groupChanges")
|
||||
lazy val membershipStoreConfig: Config = vinyldnsConfig.getConfig("membership")
|
||||
lazy val mySqlConfig: DataStoreConfig =
|
||||
pureconfig.loadConfigOrThrow[DataStoreConfig](vinyldnsConfig, "mysql")
|
||||
lazy val sqsConfig: Config = vinyldnsConfig.getConfig("sqs")
|
||||
@ -67,3 +73,36 @@ object VinylDNSConfig {
|
||||
ZoneConnection(name, keyName, key, primaryServer).encrypted(Crypto.instance)
|
||||
}
|
||||
}
|
||||
|
||||
object DynamoConfig {
|
||||
/* TODO this whole object will be removed once dynamic loading is in place
|
||||
* I split it out because of the hint - with that, for the moment we can avoid config
|
||||
* changes in dynamo stuff and still use pureconfig
|
||||
*/
|
||||
|
||||
implicit def hint[T]: ProductHint[T] = ProductHint[T](ConfigFieldMapping(CamelCase, CamelCase))
|
||||
lazy val dynamoConfig: DynamoDBDataStoreSettings =
|
||||
pureconfig.loadConfigOrThrow[DynamoDBDataStoreSettings](vinyldnsConfig, "dynamo")
|
||||
|
||||
lazy val zoneChangeStoreConfig: DynamoDBRepositorySettings =
|
||||
pureconfig.loadConfigOrThrow[DynamoDBRepositorySettings](
|
||||
vinyldnsConfig.getConfig("zoneChanges.dynamo"))
|
||||
lazy val recordSetStoreConfig: DynamoDBRepositorySettings =
|
||||
pureconfig.loadConfigOrThrow[DynamoDBRepositorySettings](
|
||||
vinyldnsConfig.getConfig("recordSet.dynamo"))
|
||||
lazy val recordChangeStoreConfig: DynamoDBRepositorySettings =
|
||||
pureconfig.loadConfigOrThrow[DynamoDBRepositorySettings](
|
||||
vinyldnsConfig.getConfig("recordChange.dynamo"))
|
||||
lazy val usersStoreConfig: DynamoDBRepositorySettings =
|
||||
pureconfig.loadConfigOrThrow[DynamoDBRepositorySettings](
|
||||
vinyldnsConfig.getConfig("users.dynamo"))
|
||||
lazy val groupsStoreConfig: DynamoDBRepositorySettings =
|
||||
pureconfig.loadConfigOrThrow[DynamoDBRepositorySettings](
|
||||
vinyldnsConfig.getConfig("groups.dynamo"))
|
||||
lazy val groupChangesStoreConfig: DynamoDBRepositorySettings =
|
||||
pureconfig.loadConfigOrThrow[DynamoDBRepositorySettings](
|
||||
vinyldnsConfig.getConfig("groupChanges.dynamo"))
|
||||
lazy val membershipStoreConfig: DynamoDBRepositorySettings =
|
||||
pureconfig.loadConfigOrThrow[DynamoDBRepositorySettings](
|
||||
vinyldnsConfig.getConfig("membership.dynamo"))
|
||||
}
|
||||
|
@ -56,8 +56,10 @@ object MembershipAuthPrincipalProvider {
|
||||
// TODO this has to be dynamic!!!
|
||||
val userRepository: UserRepository =
|
||||
DynamoDBUserRepository(VinylDNSConfig.usersStoreConfig, VinylDNSConfig.dynamoConfig)
|
||||
.unsafeRunSync()
|
||||
val membershipRepository: MembershipRepository =
|
||||
DynamoDBMembershipRepository(VinylDNSConfig.membershipStoreConfig, VinylDNSConfig.dynamoConfig)
|
||||
.unsafeRunSync()
|
||||
|
||||
def apply(): MembershipAuthPrincipalProvider =
|
||||
new MembershipAuthPrincipalProvider(userRepository, membershipRepository)
|
||||
|
@ -120,5 +120,3 @@ object DataStoreLoader {
|
||||
accessor.toEither.leftMap(errors => DataStoreStartupError(errors.toList.mkString(", ")))
|
||||
}
|
||||
}
|
||||
|
||||
case class DataStoreStartupError(msg: String) extends Throwable(msg)
|
||||
|
@ -26,7 +26,6 @@ import pureconfig.module.catseffect.loadConfigF
|
||||
import scala.collection.JavaConverters._
|
||||
import scalikejdbc.config.DBs
|
||||
import scalikejdbc.{ConnectionPool, DataSourceConnectionPool}
|
||||
import vinyldns.api.repository._
|
||||
import vinyldns.core.repository._
|
||||
|
||||
class MySqlDataStoreProvider extends DataStoreProvider {
|
||||
|
@ -49,7 +49,7 @@ vinyldns {
|
||||
zoneChanges {
|
||||
dynamo {
|
||||
tableName = "zoneChanges"
|
||||
provisionedReads=30
|
||||
provisionedReads=40
|
||||
provisionedWrites=30
|
||||
}
|
||||
}
|
||||
@ -57,7 +57,7 @@ vinyldns {
|
||||
recordSet {
|
||||
dynamo {
|
||||
tableName = "recordSet"
|
||||
provisionedReads=30
|
||||
provisionedReads=40
|
||||
provisionedWrites=30
|
||||
}
|
||||
}
|
||||
@ -65,7 +65,7 @@ vinyldns {
|
||||
recordChange {
|
||||
dynamo {
|
||||
tableName = "recordChange"
|
||||
provisionedReads=30
|
||||
provisionedReads=40
|
||||
provisionedWrites=30
|
||||
}
|
||||
}
|
||||
@ -73,7 +73,7 @@ vinyldns {
|
||||
groups {
|
||||
dynamo {
|
||||
tableName = "groups"
|
||||
provisionedReads=30
|
||||
provisionedReads=40
|
||||
provisionedWrites=30
|
||||
}
|
||||
}
|
||||
@ -81,7 +81,7 @@ vinyldns {
|
||||
groupChanges {
|
||||
dynamo {
|
||||
tableName = "groupChanges"
|
||||
provisionedReads=30
|
||||
provisionedReads=40
|
||||
provisionedWrites=30
|
||||
}
|
||||
}
|
||||
@ -89,7 +89,7 @@ vinyldns {
|
||||
membership {
|
||||
dynamo {
|
||||
tableName = "membership"
|
||||
provisionedReads=30
|
||||
provisionedReads=40
|
||||
provisionedWrites=30
|
||||
}
|
||||
}
|
||||
|
@ -28,51 +28,44 @@ class VinylDNSConfigSpec extends WordSpec with Matchers {
|
||||
|
||||
"load the dynamo config" in {
|
||||
val dynamoConfig = VinylDNSConfig.dynamoConfig
|
||||
dynamoConfig.getString("key") shouldBe "dynamoKey"
|
||||
dynamoConfig.getString("secret") shouldBe "dynamoSecret"
|
||||
dynamoConfig.getString("endpoint") shouldBe "dynamoEndpoint"
|
||||
dynamoConfig.key shouldBe "dynamoKey"
|
||||
dynamoConfig.secret shouldBe "dynamoSecret"
|
||||
dynamoConfig.endpoint shouldBe "dynamoEndpoint"
|
||||
}
|
||||
|
||||
"load the zone change repository config" in {
|
||||
val config = VinylDNSConfig.zoneChangeStoreConfig
|
||||
config.getString("dynamo.tableName") shouldBe "zoneChanges"
|
||||
config.getInt("dynamo.provisionedReads") shouldBe 30
|
||||
config.getInt("dynamo.provisionedWrites") shouldBe 30
|
||||
config.tableName shouldBe "zoneChanges"
|
||||
config.provisionedReads shouldBe 40
|
||||
config.provisionedWrites shouldBe 30
|
||||
}
|
||||
|
||||
"load the record change repository config" in {
|
||||
val config = VinylDNSConfig.recordChangeStoreConfig
|
||||
config.getString("dynamo.tableName") shouldBe "recordChange"
|
||||
config.getInt("dynamo.provisionedReads") shouldBe 30
|
||||
config.getInt("dynamo.provisionedWrites") shouldBe 30
|
||||
config.tableName shouldBe "recordChange"
|
||||
config.provisionedReads shouldBe 40
|
||||
config.provisionedWrites shouldBe 30
|
||||
}
|
||||
|
||||
"load the membership repository config" in {
|
||||
val config = VinylDNSConfig.membershipStoreConfig
|
||||
config.getString("dynamo.tableName") shouldBe "membership"
|
||||
config.getInt("dynamo.provisionedReads") shouldBe 30
|
||||
config.getInt("dynamo.provisionedWrites") shouldBe 30
|
||||
config.tableName shouldBe "membership"
|
||||
config.provisionedReads shouldBe 40
|
||||
config.provisionedWrites shouldBe 30
|
||||
}
|
||||
|
||||
"load the record set repository config" in {
|
||||
val config = VinylDNSConfig.recordSetStoreConfig
|
||||
config.getString("dynamo.tableName") shouldBe "recordSet"
|
||||
config.getInt("dynamo.provisionedReads") shouldBe 30
|
||||
config.getInt("dynamo.provisionedWrites") shouldBe 30
|
||||
}
|
||||
|
||||
"load the account store config" in {
|
||||
val config = VinylDNSConfig.accountStoreConfig
|
||||
config.getString("dynamo.key") shouldBe "dynamoKey"
|
||||
config.getString("dynamo.secret") shouldBe "dynamoSecret"
|
||||
config.getString("dynamo.endpoint") shouldBe "dynamoEndpoint"
|
||||
config.tableName shouldBe "recordSet"
|
||||
config.provisionedReads shouldBe 40
|
||||
config.provisionedWrites shouldBe 30
|
||||
}
|
||||
|
||||
"load the group repository config" in {
|
||||
val config = VinylDNSConfig.groupsStoreConfig
|
||||
config.getString("dynamo.tableName") shouldBe "groups"
|
||||
config.getInt("dynamo.provisionedReads") shouldBe 30
|
||||
config.getInt("dynamo.provisionedWrites") shouldBe 30
|
||||
config.tableName shouldBe "groups"
|
||||
config.provisionedReads shouldBe 40
|
||||
config.provisionedWrites shouldBe 30
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -29,7 +29,7 @@ import vinyldns.core.domain.membership.{
|
||||
import vinyldns.core.domain.batch.BatchChangeRepository
|
||||
import vinyldns.core.domain.record.{RecordChangeRepository, RecordSetRepository}
|
||||
import vinyldns.core.domain.zone.{ZoneChangeRepository, ZoneRepository}
|
||||
import vinyldns.core.repository.{DataAccessor, DataStore, DataStoreConfig, RepositoriesConfig}
|
||||
import vinyldns.core.repository._
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
|
@ -18,8 +18,7 @@ package vinyldns.api.repository.mysql
|
||||
|
||||
import com.typesafe.config.{Config, ConfigFactory}
|
||||
import org.scalatest.{Matchers, WordSpec}
|
||||
import vinyldns.api.repository.DataStoreStartupError
|
||||
import vinyldns.core.repository.DataStoreConfig
|
||||
import vinyldns.core.repository.{DataStoreConfig, DataStoreStartupError}
|
||||
|
||||
class MySqlDataStoreProviderSpec extends WordSpec with Matchers {
|
||||
|
||||
|
@ -0,0 +1,19 @@
|
||||
/*
|
||||
* Copyright 2018 Comcast Cable Communications Management, LLC
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package vinyldns.core.repository
|
||||
|
||||
case class DataStoreStartupError(msg: String) extends Throwable(msg)
|
@ -16,12 +16,8 @@
|
||||
|
||||
package vinyldns.dynamodb.repository
|
||||
|
||||
import java.util
|
||||
import java.util.Collections
|
||||
|
||||
import cats.implicits._
|
||||
import com.amazonaws.services.dynamodbv2.model._
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import org.joda.time.DateTime
|
||||
import vinyldns.core.TestMembershipData._
|
||||
import vinyldns.core.domain.membership.{Group, GroupChange, GroupChangeType}
|
||||
@ -34,13 +30,7 @@ class DynamoDBGroupChangeRepositoryIntegrationSpec extends DynamoDBIntegrationSp
|
||||
|
||||
private val GROUP_CHANGES_TABLE = "group-changes-live"
|
||||
|
||||
private val tableConfig = ConfigFactory.parseString(s"""
|
||||
| dynamo {
|
||||
| tableName = "$GROUP_CHANGES_TABLE"
|
||||
| provisionedReads=30
|
||||
| provisionedWrites=30
|
||||
| }
|
||||
""".stripMargin).withFallback(ConfigFactory.load())
|
||||
private val tableConfig = DynamoDBRepositorySettings(s"$GROUP_CHANGES_TABLE", 30, 30)
|
||||
|
||||
private var repo: DynamoDBGroupChangeRepository = _
|
||||
|
||||
@ -66,10 +56,7 @@ class DynamoDBGroupChangeRepositoryIntegrationSpec extends DynamoDBIntegrationSp
|
||||
listOfDummyGroupChanges ++ listOfRandomTimeGroupChanges
|
||||
|
||||
def setup(): Unit = {
|
||||
repo = new DynamoDBGroupChangeRepository(tableConfig, dynamoDBHelper)
|
||||
waitForRepo(repo.getGroupChange("any"))
|
||||
|
||||
clearGroupChanges()
|
||||
repo = DynamoDBGroupChangeRepository(tableConfig, dynamoIntegrationConfig).unsafeRunSync()
|
||||
|
||||
// Create all the changes
|
||||
val savedGroupChanges = groupChanges.map(repo.save(_)).toList.parSequence
|
||||
@ -80,33 +67,7 @@ class DynamoDBGroupChangeRepositoryIntegrationSpec extends DynamoDBIntegrationSp
|
||||
|
||||
def tearDown(): Unit = {
|
||||
val request = new DeleteTableRequest().withTableName(GROUP_CHANGES_TABLE)
|
||||
dynamoDBHelper.deleteTable(request).unsafeRunSync()
|
||||
}
|
||||
|
||||
private def clearGroupChanges(): Unit = {
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
val scanRequest = new ScanRequest().withTableName(GROUP_CHANGES_TABLE)
|
||||
|
||||
val allGroupChanges = dynamoClient.scan(scanRequest).getItems.asScala.map(repo.fromItem)
|
||||
|
||||
val batchWrites = allGroupChanges
|
||||
.map { groupChange =>
|
||||
val key = new util.HashMap[String, AttributeValue]()
|
||||
key.put("group_change_id", new AttributeValue(groupChange.id))
|
||||
new WriteRequest().withDeleteRequest(new DeleteRequest().withKey(key))
|
||||
}
|
||||
.grouped(25)
|
||||
.map { deleteRequests =>
|
||||
new BatchWriteItemRequest()
|
||||
.withRequestItems(Collections.singletonMap(GROUP_CHANGES_TABLE, deleteRequests.asJava))
|
||||
}
|
||||
.toList
|
||||
|
||||
batchWrites.foreach { batch =>
|
||||
dynamoClient.batchWriteItem(batch)
|
||||
}
|
||||
repo.dynamoDBHelper.deleteTable(request).unsafeRunSync()
|
||||
}
|
||||
|
||||
"DynamoDBGroupChangeRepository" should {
|
||||
|
@ -16,12 +16,8 @@
|
||||
|
||||
package vinyldns.dynamodb.repository
|
||||
|
||||
import java.util
|
||||
import java.util.Collections
|
||||
|
||||
import cats.implicits._
|
||||
import com.amazonaws.services.dynamodbv2.model._
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import vinyldns.core.domain.membership.{Group, GroupStatus}
|
||||
import vinyldns.core.TestMembershipData._
|
||||
|
||||
@ -30,13 +26,7 @@ import scala.concurrent.duration._
|
||||
class DynamoDBGroupRepositoryIntegrationSpec extends DynamoDBIntegrationSpec {
|
||||
private val GROUP_TABLE = "groups-live"
|
||||
|
||||
private val tableConfig = ConfigFactory.parseString(s"""
|
||||
| dynamo {
|
||||
| tableName = "$GROUP_TABLE"
|
||||
| provisionedReads=30
|
||||
| provisionedWrites=30
|
||||
| }
|
||||
""".stripMargin).withFallback(ConfigFactory.load())
|
||||
private val tableConfig = DynamoDBRepositorySettings(s"$GROUP_TABLE", 30, 30)
|
||||
|
||||
private var repo: DynamoDBGroupRepository = _
|
||||
|
||||
@ -64,10 +54,7 @@ class DynamoDBGroupRepositoryIntegrationSpec extends DynamoDBIntegrationSpec {
|
||||
private val groups = activeGroups ++ List(inDbDeletedGroup)
|
||||
|
||||
def setup(): Unit = {
|
||||
repo = new DynamoDBGroupRepository(tableConfig, dynamoDBHelper)
|
||||
waitForRepo(repo.getGroup("any"))
|
||||
|
||||
clearGroups()
|
||||
repo = DynamoDBGroupRepository(tableConfig, dynamoIntegrationConfig).unsafeRunSync()
|
||||
|
||||
// Create all the groups
|
||||
val savedGroups = groups.map(repo.save(_)).toList.parSequence
|
||||
@ -78,36 +65,10 @@ class DynamoDBGroupRepositoryIntegrationSpec extends DynamoDBIntegrationSpec {
|
||||
|
||||
def tearDown(): Unit = {
|
||||
val request = new DeleteTableRequest().withTableName(GROUP_TABLE)
|
||||
val deleteTables = dynamoDBHelper.deleteTable(request)
|
||||
val deleteTables = repo.dynamoDBHelper.deleteTable(request)
|
||||
deleteTables.unsafeRunSync()
|
||||
}
|
||||
|
||||
private def clearGroups(): Unit = {
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
val scanRequest = new ScanRequest().withTableName(GROUP_TABLE)
|
||||
|
||||
val allGroups = dynamoClient.scan(scanRequest).getItems.asScala.map(repo.fromItem)
|
||||
|
||||
val batchWrites = allGroups
|
||||
.map { group =>
|
||||
val key = new util.HashMap[String, AttributeValue]()
|
||||
key.put("group_id", new AttributeValue(group.id))
|
||||
new WriteRequest().withDeleteRequest(new DeleteRequest().withKey(key))
|
||||
}
|
||||
.grouped(25)
|
||||
.map { deleteRequests =>
|
||||
new BatchWriteItemRequest()
|
||||
.withRequestItems(Collections.singletonMap(GROUP_TABLE, deleteRequests.asJava))
|
||||
}
|
||||
.toList
|
||||
|
||||
batchWrites.foreach { batch =>
|
||||
dynamoClient.batchWriteItem(batch)
|
||||
}
|
||||
}
|
||||
|
||||
"DynamoDBGroupRepository" should {
|
||||
"get a group by id" in {
|
||||
val targetGroup = groups.head
|
||||
|
@ -18,13 +18,8 @@ package vinyldns.dynamodb.repository
|
||||
|
||||
import java.util.UUID
|
||||
|
||||
import cats.effect.IO
|
||||
import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClient
|
||||
import com.typesafe.config.{Config, ConfigFactory}
|
||||
import org.scalatest._
|
||||
import org.scalatest.concurrent.ScalaFutures
|
||||
import org.slf4j.LoggerFactory
|
||||
import scala.concurrent.duration._
|
||||
|
||||
trait DynamoDBIntegrationSpec
|
||||
extends WordSpec
|
||||
@ -35,23 +30,15 @@ trait DynamoDBIntegrationSpec
|
||||
with Inspectors {
|
||||
|
||||
// port is defined in the docker/docker-compose.yml file for dynamodb
|
||||
val dynamoClient: AmazonDynamoDBClient = getDynamoClient(19003)
|
||||
val dynamoIntegrationConfig: DynamoDBDataStoreSettings = getDynamoConfig(19003)
|
||||
|
||||
def getDynamoClient(port: Int): AmazonDynamoDBClient = {
|
||||
val endpoint: String = s"http://localhost:$port"
|
||||
val dynamoConfig: Config = ConfigFactory.parseString(s"""
|
||||
| key = "vinyldnsTest"
|
||||
| secret = "notNeededForDynamoDbLocal"
|
||||
| endpoint="$endpoint",
|
||||
| region="us-east-1"
|
||||
""".stripMargin)
|
||||
|
||||
DynamoDBClient(dynamoConfig)
|
||||
def getDynamoConfig(port: Int): DynamoDBDataStoreSettings = {
|
||||
DynamoDBDataStoreSettings("vinyldnsTest",
|
||||
"notNeededForDynamoDbLocal",
|
||||
s"http://localhost:$port",
|
||||
"us-east-1")
|
||||
}
|
||||
|
||||
val dynamoDBHelper: DynamoDBHelper =
|
||||
new DynamoDBHelper(dynamoClient, LoggerFactory.getLogger("DynamoDBIntegrationSpec"))
|
||||
|
||||
override protected def beforeAll(): Unit =
|
||||
setup()
|
||||
|
||||
@ -67,12 +54,4 @@ trait DynamoDBIntegrationSpec
|
||||
/* Generates a random string useful to avoid data collision */
|
||||
def genString: String = UUID.randomUUID().toString
|
||||
|
||||
/* wait until the repo is ready, could take time if the table has to be created */
|
||||
def waitForRepo[A](call: IO[A]): Unit = {
|
||||
var notReady = call.unsafeRunTimed(5.seconds).isEmpty
|
||||
while (notReady) {
|
||||
Thread.sleep(2000)
|
||||
notReady = call.unsafeRunTimed(5.seconds).isEmpty
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -17,19 +17,13 @@
|
||||
package vinyldns.dynamodb.repository
|
||||
|
||||
import cats.implicits._
|
||||
import com.typesafe.config.ConfigFactory
|
||||
|
||||
import scala.concurrent.duration._
|
||||
|
||||
class DynamoDBMembershipRepositoryIntegrationSpec extends DynamoDBIntegrationSpec {
|
||||
private val membershipTable = "membership-live"
|
||||
private val tableConfig = ConfigFactory.parseString(s"""
|
||||
| dynamo {
|
||||
| tableName = "$membershipTable"
|
||||
| provisionedReads=100
|
||||
| provisionedWrites=100
|
||||
| }
|
||||
""".stripMargin).withFallback(ConfigFactory.load())
|
||||
|
||||
private val tableConfig = DynamoDBRepositorySettings(s"$membershipTable", 30, 30)
|
||||
|
||||
private var repo: DynamoDBMembershipRepository = _
|
||||
|
||||
@ -37,8 +31,7 @@ class DynamoDBMembershipRepositoryIntegrationSpec extends DynamoDBIntegrationSpe
|
||||
private val testGroupIds = for (i <- 0 to 5) yield s"test-group-$i"
|
||||
|
||||
def setup(): Unit = {
|
||||
repo = new DynamoDBMembershipRepository(tableConfig, dynamoDBHelper)
|
||||
waitForRepo(repo.getGroupsForUser("any"))
|
||||
repo = DynamoDBMembershipRepository(tableConfig, dynamoIntegrationConfig).unsafeRunSync()
|
||||
|
||||
// Create all the items
|
||||
val results = testGroupIds.map(repo.addMembers(_, testUserIds.toSet)).toList.parSequence
|
||||
|
@ -16,11 +16,10 @@
|
||||
|
||||
package vinyldns.dynamodb.repository
|
||||
|
||||
import java.util
|
||||
import java.util.UUID
|
||||
|
||||
import com.amazonaws.services.dynamodbv2.model.{AttributeValue, DeleteItemRequest, ScanRequest}
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import cats.implicits._
|
||||
import com.amazonaws.services.dynamodbv2.model._
|
||||
import org.joda.time.DateTime
|
||||
import vinyldns.core.domain.record.{ChangeSet, ChangeSetStatus, RecordSetChange}
|
||||
import vinyldns.core.domain.zone.{Zone, ZoneStatus}
|
||||
@ -35,13 +34,7 @@ class DynamoDBRecordChangeRepositoryIntegrationSpec
|
||||
|
||||
private val recordChangeTable = "record-change-live"
|
||||
|
||||
private val tableConfig = ConfigFactory.parseString(s"""
|
||||
| dynamo {
|
||||
| tableName = "$recordChangeTable"
|
||||
| provisionedReads=30
|
||||
| provisionedWrites=30
|
||||
| }
|
||||
""".stripMargin).withFallback(ConfigFactory.load())
|
||||
private val tableConfig = DynamoDBRepositorySettings(s"$recordChangeTable", 30, 30)
|
||||
|
||||
private var repo: DynamoDBRecordChangeRepository = _
|
||||
|
||||
@ -200,11 +193,7 @@ class DynamoDBRecordChangeRepositoryIntegrationSpec
|
||||
.reverse // Changes are retrieved by time stamp in decending order
|
||||
|
||||
def setup(): Unit = {
|
||||
repo = new DynamoDBRecordChangeRepository(tableConfig, dynamoDBHelper)
|
||||
waitForRepo(repo.getRecordSetChange("any", "any"))
|
||||
|
||||
// Clear the table just in case there is some lagging test data
|
||||
clearTable()
|
||||
repo = DynamoDBRecordChangeRepository(tableConfig, dynamoIntegrationConfig).unsafeRunSync()
|
||||
|
||||
changeSets.foreach { changeSet =>
|
||||
// Save the change set
|
||||
@ -223,34 +212,9 @@ class DynamoDBRecordChangeRepositoryIntegrationSpec
|
||||
}
|
||||
}
|
||||
|
||||
def tearDown(): Unit =
|
||||
clearTable()
|
||||
|
||||
private def clearTable(): Unit = {
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
// clear the table that we work with here
|
||||
// NOTE: This is brute force and could be cleaner
|
||||
val scanRequest = new ScanRequest()
|
||||
.withTableName(recordChangeTable)
|
||||
|
||||
val result =
|
||||
dynamoClient.scan(scanRequest).getItems.asScala.map(_.get(repo.RECORD_SET_CHANGE_ID).getS())
|
||||
|
||||
result.foreach(deleteItem)
|
||||
}
|
||||
|
||||
private def deleteItem(recordSetChangeId: String): Unit = {
|
||||
val key = new util.HashMap[String, AttributeValue]()
|
||||
key.put(repo.RECORD_SET_CHANGE_ID, new AttributeValue(recordSetChangeId))
|
||||
val request = new DeleteItemRequest().withTableName(recordChangeTable).withKey(key)
|
||||
try {
|
||||
dynamoClient.deleteItem(request)
|
||||
} catch {
|
||||
case ex: Throwable =>
|
||||
throw new UnexpectedDynamoResponseException(ex.getMessage, ex)
|
||||
}
|
||||
def tearDown(): Unit = {
|
||||
val request = new DeleteTableRequest().withTableName(recordChangeTable)
|
||||
repo.dynamoDBHelper.deleteTable(request).unsafeRunSync()
|
||||
}
|
||||
|
||||
"DynamoDBRepository" should {
|
||||
|
@ -19,8 +19,7 @@ package vinyldns.dynamodb.repository
|
||||
import java.util.UUID
|
||||
|
||||
import cats.implicits._
|
||||
import com.amazonaws.services.dynamodbv2.model.{ScanRequest, ScanResult}
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import com.amazonaws.services.dynamodbv2.model._
|
||||
import org.joda.time.DateTime
|
||||
import vinyldns.core.domain.membership.User
|
||||
import vinyldns.core.domain.zone.{Zone, ZoneStatus}
|
||||
@ -37,17 +36,9 @@ class DynamoDBRecordSetRepositoryIntegrationSpec
|
||||
private val recordSetTable = "record-sets-live"
|
||||
private[repository] val recordSetTableName: String = recordSetTable
|
||||
|
||||
private val tableConfig = ConfigFactory.parseString(s"""
|
||||
| dynamo {
|
||||
| tableName = "$recordSetTable"
|
||||
| provisionedReads=50
|
||||
| provisionedWrites=50
|
||||
| }
|
||||
""".stripMargin).withFallback(ConfigFactory.load())
|
||||
private val tableConfig = DynamoDBRepositorySettings(s"$recordSetTable", 30, 30)
|
||||
|
||||
import dynamoDBHelper._
|
||||
|
||||
private val repo = new DynamoDBRecordSetRepository(tableConfig, dynamoDBHelper)
|
||||
private var repo: DynamoDBRecordSetRepository = _
|
||||
|
||||
private val users = for (i <- 1 to 3)
|
||||
yield User(s"live-test-acct$i", "key", "secret")
|
||||
@ -82,13 +73,7 @@ class DynamoDBRecordSetRepositoryIntegrationSpec
|
||||
)
|
||||
|
||||
def setup(): Unit = {
|
||||
|
||||
// wait until the repo is ready, could take time if the table has to be created
|
||||
val call = repo.listRecordSets(zoneId = "any", startFrom = None, maxItems = None, recordNameFilter = None)
|
||||
waitForRepo(call)
|
||||
|
||||
// Clear the zone just in case there is some lagging test data
|
||||
clearTable()
|
||||
repo = DynamoDBRecordSetRepository(tableConfig, dynamoIntegrationConfig).unsafeRunSync()
|
||||
|
||||
// Create all the items
|
||||
val results = recordSets.map(repo.putRecordSet(_)).toList.parSequence
|
||||
@ -97,45 +82,11 @@ class DynamoDBRecordSetRepositoryIntegrationSpec
|
||||
results.unsafeRunTimed(5.minutes).getOrElse(fail("timeout waiting for data load"))
|
||||
}
|
||||
|
||||
def tearDown(): Unit =
|
||||
clearTable()
|
||||
|
||||
private def clearTable(): Unit = {
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
// clear all the zones from the table that we work with here
|
||||
val scanRequest = new ScanRequest().withTableName(recordSetTable)
|
||||
|
||||
val scanResult = dynamoClient.scan(scanRequest)
|
||||
|
||||
var counter = 0
|
||||
|
||||
def delete(r: ScanResult) {
|
||||
val result = r.getItems.asScala.grouped(25)
|
||||
|
||||
// recurse over the results of the scan, convert each group to a BatchWriteItem with Deletes, and then delete
|
||||
// using a blocking call
|
||||
result.foreach { group =>
|
||||
val recordSetIds = group.map(_.get(DynamoDBRecordSetRepository.RECORD_SET_ID).getS)
|
||||
val deletes = recordSetIds.map(deleteRecordSetFromTable)
|
||||
val batchDelete = toBatchWriteItemRequest(deletes, recordSetTable)
|
||||
|
||||
dynamoClient.batchWriteItem(batchDelete)
|
||||
|
||||
counter = counter + 25
|
||||
def tearDown(): Unit = {
|
||||
val request = new DeleteTableRequest().withTableName(recordSetTable)
|
||||
repo.dynamoDBHelper.deleteTable(request).unsafeRunSync()
|
||||
}
|
||||
|
||||
if (r.getLastEvaluatedKey != null && !r.getLastEvaluatedKey.isEmpty) {
|
||||
val nextScan = new ScanRequest().withTableName(recordSetTable)
|
||||
nextScan.setExclusiveStartKey(scanResult.getLastEvaluatedKey)
|
||||
val nextScanResult = dynamoClient.scan(scanRequest)
|
||||
delete(nextScanResult)
|
||||
}
|
||||
}
|
||||
|
||||
delete(scanResult)
|
||||
}
|
||||
|
||||
"DynamoDBRecordSetRepository" should {
|
||||
"get a record set by id" in {
|
||||
|
@ -18,7 +18,6 @@ package vinyldns.dynamodb.repository
|
||||
|
||||
import cats.implicits._
|
||||
import com.amazonaws.services.dynamodbv2.model.DeleteTableRequest
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import vinyldns.core.domain.membership.User
|
||||
|
||||
import scala.concurrent.duration._
|
||||
@ -27,13 +26,7 @@ class DynamoDBUserRepositoryIntegrationSpec extends DynamoDBIntegrationSpec {
|
||||
|
||||
private val userTable = "users-live"
|
||||
|
||||
private val tableConfig = ConfigFactory.parseString(s"""
|
||||
| dynamo {
|
||||
| tableName = "$userTable"
|
||||
| provisionedReads=100
|
||||
| provisionedWrites=100
|
||||
| }
|
||||
""".stripMargin).withFallback(ConfigFactory.load())
|
||||
private val tableConfig = DynamoDBRepositorySettings(s"$userTable", 30, 30)
|
||||
|
||||
private var repo: DynamoDBUserRepository = _
|
||||
|
||||
@ -43,8 +36,7 @@ class DynamoDBUserRepositoryIntegrationSpec extends DynamoDBIntegrationSpec {
|
||||
}
|
||||
|
||||
def setup(): Unit = {
|
||||
repo = new DynamoDBUserRepository(tableConfig, dynamoDBHelper)
|
||||
waitForRepo(repo.getUser("any"))
|
||||
repo = DynamoDBUserRepository(tableConfig, dynamoIntegrationConfig).unsafeRunSync()
|
||||
|
||||
// Create all the items
|
||||
val results = users.map(repo.save(_)).parSequence
|
||||
@ -55,7 +47,7 @@ class DynamoDBUserRepositoryIntegrationSpec extends DynamoDBIntegrationSpec {
|
||||
|
||||
def tearDown(): Unit = {
|
||||
val request = new DeleteTableRequest().withTableName(userTable)
|
||||
dynamoDBHelper.deleteTable(request).unsafeRunSync()
|
||||
repo.dynamoDBHelper.deleteTable(request).unsafeRunSync()
|
||||
}
|
||||
|
||||
"DynamoDBUserRepository" should {
|
||||
|
@ -16,11 +16,8 @@
|
||||
|
||||
package vinyldns.dynamodb.repository
|
||||
|
||||
import java.util
|
||||
|
||||
import cats.implicits._
|
||||
import com.amazonaws.services.dynamodbv2.model.{AttributeValue, DeleteItemRequest, ScanRequest}
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import com.amazonaws.services.dynamodbv2.model._
|
||||
import org.joda.time.DateTime
|
||||
import vinyldns.core.domain.membership.User
|
||||
import vinyldns.core.domain.zone._
|
||||
@ -34,14 +31,7 @@ class DynamoDBZoneChangeRepositoryIntegrationSpec extends DynamoDBIntegrationSpe
|
||||
|
||||
private val zoneChangeTable = "zone-changes-live"
|
||||
|
||||
private val tableConfig = ConfigFactory.parseString(
|
||||
s"""
|
||||
| dynamo {
|
||||
| tableName = "$zoneChangeTable"
|
||||
| provisionedReads=30
|
||||
| provisionedWrites=30
|
||||
| }
|
||||
""".stripMargin).withFallback(ConfigFactory.load())
|
||||
private val tableConfig = DynamoDBRepositorySettings(s"$zoneChangeTable", 30, 30)
|
||||
|
||||
private var repo: DynamoDBZoneChangeRepository = _
|
||||
|
||||
@ -69,11 +59,7 @@ class DynamoDBZoneChangeRepositoryIntegrationSpec extends DynamoDBIntegrationSpe
|
||||
created = now.minusSeconds(Random.nextInt(1000)))
|
||||
|
||||
def setup(): Unit = {
|
||||
repo = new DynamoDBZoneChangeRepository(tableConfig, dynamoDBHelper)
|
||||
waitForRepo(repo.listZoneChanges("any"))
|
||||
|
||||
// Clear the zone just in case there is some lagging test data
|
||||
clearChanges()
|
||||
repo = DynamoDBZoneChangeRepository(tableConfig, dynamoIntegrationConfig).unsafeRunSync()
|
||||
|
||||
// Create all the zones
|
||||
val results = changes.map(repo.save(_)).toList.parSequence
|
||||
@ -81,39 +67,11 @@ class DynamoDBZoneChangeRepositoryIntegrationSpec extends DynamoDBIntegrationSpe
|
||||
results.unsafeRunTimed(5.minutes).getOrElse(fail("timeout waiting for data load"))
|
||||
}
|
||||
|
||||
def tearDown(): Unit =
|
||||
clearChanges()
|
||||
|
||||
private def clearChanges(): Unit = {
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
// clear all the zones from the table that we work with here
|
||||
// NOTE: This is brute force and could be cleaner
|
||||
val scanRequest = new ScanRequest()
|
||||
.withTableName(zoneChangeTable)
|
||||
|
||||
val result = dynamoClient
|
||||
.scan(scanRequest)
|
||||
.getItems
|
||||
.asScala
|
||||
.map(i => (i.get("zone_id").getS, i.get("change_id").getS))
|
||||
|
||||
result.foreach(Function.tupled(deleteZoneChange))
|
||||
def tearDown(): Unit = {
|
||||
val request = new DeleteTableRequest().withTableName(zoneChangeTable)
|
||||
repo.dynamoDBHelper.deleteTable(request).unsafeRunSync()
|
||||
}
|
||||
|
||||
private def deleteZoneChange(zoneId: String, changeId: String): Unit = {
|
||||
val key = new util.HashMap[String, AttributeValue]()
|
||||
key.put("zone_id", new AttributeValue(zoneId))
|
||||
key.put("change_id", new AttributeValue(changeId))
|
||||
val request = new DeleteItemRequest().withTableName(zoneChangeTable).withKey(key)
|
||||
try {
|
||||
dynamoClient.deleteItem(request)
|
||||
} catch {
|
||||
case ex: Throwable =>
|
||||
throw new UnexpectedDynamoResponseException(ex.getMessage, ex)
|
||||
}
|
||||
}
|
||||
|
||||
"DynamoDBRepository" should {
|
||||
|
||||
|
@ -19,18 +19,15 @@ package vinyldns.dynamodb.repository
|
||||
import com.amazonaws.auth.{AWSStaticCredentialsProvider, BasicAWSCredentials}
|
||||
import com.amazonaws.client.builder.AwsClientBuilder.EndpointConfiguration
|
||||
import com.amazonaws.services.dynamodbv2.{AmazonDynamoDBClient, AmazonDynamoDBClientBuilder}
|
||||
import com.typesafe.config.Config
|
||||
|
||||
object DynamoDBClient {
|
||||
|
||||
def apply(config: Config): AmazonDynamoDBClient = {
|
||||
val dynamoAKID = config.getString("key")
|
||||
val dynamoSecret = config.getString("secret")
|
||||
val dynamoEndpoint = config.getString("endpoint")
|
||||
val dynamoRegion = config.getString("region")
|
||||
def apply(dynamoDBDataStoreSettings: DynamoDBDataStoreSettings): AmazonDynamoDBClient = {
|
||||
val dynamoAKID = dynamoDBDataStoreSettings.key
|
||||
val dynamoSecret = dynamoDBDataStoreSettings.secret
|
||||
val dynamoEndpoint = dynamoDBDataStoreSettings.endpoint
|
||||
val dynamoRegion = dynamoDBDataStoreSettings.region
|
||||
|
||||
// Important! For some reason the basic credentials get lost in Jenkins. Set the aws system properties
|
||||
// just in case
|
||||
System.getProperties.setProperty("aws.accessKeyId", dynamoAKID)
|
||||
System.getProperties.setProperty("aws.secretKey", dynamoSecret)
|
||||
val credentials = new BasicAWSCredentials(dynamoAKID, dynamoSecret)
|
||||
|
@ -0,0 +1,28 @@
|
||||
/*
|
||||
* Copyright 2018 Comcast Cable Communications Management, LLC
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package vinyldns.dynamodb.repository
|
||||
|
||||
final case class DynamoDBDataStoreSettings(
|
||||
key: String,
|
||||
secret: String,
|
||||
endpoint: String,
|
||||
region: String)
|
||||
|
||||
final case class DynamoDBRepositorySettings(
|
||||
tableName: String,
|
||||
provisionedReads: Long,
|
||||
provisionedWrites: Long)
|
@ -20,8 +20,8 @@ import java.nio.ByteBuffer
|
||||
import java.util.HashMap
|
||||
|
||||
import cats.effect._
|
||||
import cats.implicits._
|
||||
import com.amazonaws.services.dynamodbv2.model._
|
||||
import com.typesafe.config.Config
|
||||
import org.joda.time.DateTime
|
||||
import org.slf4j.{Logger, LoggerFactory}
|
||||
import vinyldns.core.domain.membership.{GroupChange, GroupChangeRepository, ListGroupChangesResults}
|
||||
@ -33,39 +33,31 @@ import scala.collection.JavaConverters._
|
||||
|
||||
object DynamoDBGroupChangeRepository {
|
||||
|
||||
def apply(config: Config, dynamoConfig: Config): DynamoDBGroupChangeRepository =
|
||||
new DynamoDBGroupChangeRepository(
|
||||
config,
|
||||
new DynamoDBHelper(
|
||||
DynamoDBClient(dynamoConfig),
|
||||
LoggerFactory.getLogger(classOf[DynamoDBGroupChangeRepository])))
|
||||
}
|
||||
|
||||
class DynamoDBGroupChangeRepository(config: Config, dynamoDBHelper: DynamoDBHelper)
|
||||
extends GroupChangeRepository
|
||||
with Monitored
|
||||
with GroupProtobufConversions {
|
||||
|
||||
val log: Logger = LoggerFactory.getLogger(classOf[DynamoDBGroupChangeRepository])
|
||||
|
||||
private[repository] val GROUP_CHANGE_ID = "group_change_id"
|
||||
private[repository] val GROUP_ID = "group_id"
|
||||
private[repository] val CREATED = "created"
|
||||
private[repository] val GROUP_CHANGE_ATTR = "group_change_blob"
|
||||
|
||||
private val GROUP_ID_AND_CREATED_INDEX = "GROUP_ID_AND_CREATED_INDEX"
|
||||
|
||||
private val dynamoReads = config.getLong("dynamo.provisionedReads")
|
||||
private val dynamoWrites = config.getLong("dynamo.provisionedWrites")
|
||||
private[repository] val GROUP_CHANGE_TABLE = config.getString("dynamo.tableName")
|
||||
def apply(
|
||||
config: DynamoDBRepositorySettings,
|
||||
dynamoConfig: DynamoDBDataStoreSettings): IO[DynamoDBGroupChangeRepository] = {
|
||||
|
||||
private[repository] val tableAttributes = Seq(
|
||||
val dynamoDBHelper = new DynamoDBHelper(
|
||||
DynamoDBClient(dynamoConfig),
|
||||
LoggerFactory.getLogger(classOf[DynamoDBGroupChangeRepository]))
|
||||
|
||||
val dynamoReads = config.provisionedReads
|
||||
val dynamoWrites = config.provisionedWrites
|
||||
val tableName = config.tableName
|
||||
|
||||
val tableAttributes = Seq(
|
||||
new AttributeDefinition(GROUP_ID, "S"),
|
||||
new AttributeDefinition(CREATED, "N"),
|
||||
new AttributeDefinition(GROUP_CHANGE_ID, "S")
|
||||
)
|
||||
|
||||
private[repository] val secondaryIndexes = Seq(
|
||||
val secondaryIndexes = Seq(
|
||||
new GlobalSecondaryIndex()
|
||||
.withIndexName(GROUP_ID_AND_CREATED_INDEX)
|
||||
.withProvisionedThroughput(new ProvisionedThroughput(dynamoReads, dynamoWrites))
|
||||
@ -75,20 +67,35 @@ class DynamoDBGroupChangeRepository(config: Config, dynamoDBHelper: DynamoDBHelp
|
||||
.withProjection(new Projection().withProjectionType("ALL"))
|
||||
)
|
||||
|
||||
dynamoDBHelper.setupTable(
|
||||
val setup = dynamoDBHelper.setupTable(
|
||||
new CreateTableRequest()
|
||||
.withTableName(GROUP_CHANGE_TABLE)
|
||||
.withTableName(tableName)
|
||||
.withAttributeDefinitions(tableAttributes: _*)
|
||||
.withKeySchema(new KeySchemaElement(GROUP_CHANGE_ID, KeyType.HASH))
|
||||
.withGlobalSecondaryIndexes(secondaryIndexes: _*)
|
||||
.withProvisionedThroughput(new ProvisionedThroughput(dynamoReads, dynamoWrites))
|
||||
)
|
||||
|
||||
setup.as(new DynamoDBGroupChangeRepository(tableName, dynamoDBHelper))
|
||||
}
|
||||
}
|
||||
|
||||
class DynamoDBGroupChangeRepository private[repository] (
|
||||
groupChangeTableName: String,
|
||||
val dynamoDBHelper: DynamoDBHelper)
|
||||
extends GroupChangeRepository
|
||||
with Monitored
|
||||
with GroupProtobufConversions {
|
||||
|
||||
import DynamoDBGroupChangeRepository._
|
||||
|
||||
val log: Logger = LoggerFactory.getLogger(classOf[DynamoDBGroupChangeRepository])
|
||||
|
||||
def save(groupChange: GroupChange): IO[GroupChange] =
|
||||
monitor("repo.GroupChange.save") {
|
||||
log.info(s"Saving groupChange ${groupChange.id}.")
|
||||
val item = toItem(groupChange)
|
||||
val request = new PutItemRequest().withTableName(GROUP_CHANGE_TABLE).withItem(item)
|
||||
val request = new PutItemRequest().withTableName(groupChangeTableName).withItem(item)
|
||||
dynamoDBHelper.putItem(request).map(_ => groupChange)
|
||||
}
|
||||
|
||||
@ -97,7 +104,7 @@ class DynamoDBGroupChangeRepository(config: Config, dynamoDBHelper: DynamoDBHelp
|
||||
log.info(s"Getting groupChange $groupChangeId.")
|
||||
val key = new HashMap[String, AttributeValue]()
|
||||
key.put(GROUP_CHANGE_ID, new AttributeValue(groupChangeId))
|
||||
val request = new GetItemRequest().withTableName(GROUP_CHANGE_TABLE).withKey(key)
|
||||
val request = new GetItemRequest().withTableName(groupChangeTableName).withKey(key)
|
||||
|
||||
dynamoDBHelper.getItem(request).map { result =>
|
||||
Option(result.getItem).map(fromItem)
|
||||
@ -126,7 +133,7 @@ class DynamoDBGroupChangeRepository(config: Config, dynamoDBHelper: DynamoDBHelp
|
||||
"#group_id_attribute = :group_id AND #created_attribute < :created"
|
||||
|
||||
val queryRequest = new QueryRequest()
|
||||
.withTableName(GROUP_CHANGE_TABLE)
|
||||
.withTableName(groupChangeTableName)
|
||||
.withIndexName(GROUP_ID_AND_CREATED_INDEX)
|
||||
.withExpressionAttributeNames(expressionAttributeNames)
|
||||
.withExpressionAttributeValues(expressionAttributeValues)
|
||||
|
@ -22,7 +22,6 @@ import java.util.HashMap
|
||||
import cats.effect._
|
||||
import cats.implicits._
|
||||
import com.amazonaws.services.dynamodbv2.model.{CreateTableRequest, Projection, _}
|
||||
import com.typesafe.config.Config
|
||||
import org.joda.time.DateTime
|
||||
import org.slf4j.{Logger, LoggerFactory}
|
||||
import vinyldns.core.domain.membership.GroupStatus.GroupStatus
|
||||
@ -33,20 +32,6 @@ import scala.collection.JavaConverters._
|
||||
|
||||
object DynamoDBGroupRepository {
|
||||
|
||||
def apply(config: Config, dynamoConfig: Config): DynamoDBGroupRepository =
|
||||
new DynamoDBGroupRepository(
|
||||
config,
|
||||
new DynamoDBHelper(
|
||||
DynamoDBClient(dynamoConfig),
|
||||
LoggerFactory.getLogger(classOf[DynamoDBGroupRepository])))
|
||||
}
|
||||
|
||||
class DynamoDBGroupRepository(config: Config, dynamoDBHelper: DynamoDBHelper)
|
||||
extends GroupRepository
|
||||
with Monitored {
|
||||
|
||||
val log: Logger = LoggerFactory.getLogger(classOf[DynamoDBGroupRepository])
|
||||
|
||||
private[repository] val GROUP_ID = "group_id"
|
||||
private val NAME = "name"
|
||||
private val EMAIL = "email"
|
||||
@ -57,16 +42,24 @@ class DynamoDBGroupRepository(config: Config, dynamoDBHelper: DynamoDBHelper)
|
||||
private val ADMIN_IDS = "admin_ids"
|
||||
private val GROUP_NAME_INDEX = "group_name_index"
|
||||
|
||||
private val dynamoReads = config.getLong("dynamo.provisionedReads")
|
||||
private val dynamoWrites = config.getLong("dynamo.provisionedWrites")
|
||||
private[repository] val GROUP_TABLE = config.getString("dynamo.tableName")
|
||||
def apply(
|
||||
config: DynamoDBRepositorySettings,
|
||||
dynamoConfig: DynamoDBDataStoreSettings): IO[DynamoDBGroupRepository] = {
|
||||
|
||||
private[repository] val tableAttributes = Seq(
|
||||
val dynamoDBHelper = new DynamoDBHelper(
|
||||
DynamoDBClient(dynamoConfig),
|
||||
LoggerFactory.getLogger(classOf[DynamoDBGroupRepository]))
|
||||
|
||||
val dynamoReads = config.provisionedReads
|
||||
val dynamoWrites = config.provisionedWrites
|
||||
val tableName = config.tableName
|
||||
|
||||
val tableAttributes = Seq(
|
||||
new AttributeDefinition(GROUP_ID, "S"),
|
||||
new AttributeDefinition(NAME, "S")
|
||||
)
|
||||
|
||||
private[repository] val secondaryIndexes = Seq(
|
||||
val secondaryIndexes = Seq(
|
||||
new GlobalSecondaryIndex()
|
||||
.withIndexName(GROUP_NAME_INDEX)
|
||||
.withProvisionedThroughput(new ProvisionedThroughput(dynamoReads, dynamoWrites))
|
||||
@ -74,20 +67,34 @@ class DynamoDBGroupRepository(config: Config, dynamoDBHelper: DynamoDBHelper)
|
||||
.withProjection(new Projection().withProjectionType("ALL"))
|
||||
)
|
||||
|
||||
dynamoDBHelper.setupTable(
|
||||
val setup = dynamoDBHelper.setupTable(
|
||||
new CreateTableRequest()
|
||||
.withTableName(GROUP_TABLE)
|
||||
.withTableName(tableName)
|
||||
.withAttributeDefinitions(tableAttributes: _*)
|
||||
.withKeySchema(new KeySchemaElement(GROUP_ID, KeyType.HASH))
|
||||
.withGlobalSecondaryIndexes(secondaryIndexes: _*)
|
||||
.withProvisionedThroughput(new ProvisionedThroughput(dynamoReads, dynamoWrites))
|
||||
)
|
||||
|
||||
setup.as(new DynamoDBGroupRepository(tableName, dynamoDBHelper))
|
||||
}
|
||||
}
|
||||
|
||||
class DynamoDBGroupRepository private[repository] (
|
||||
groupTableName: String,
|
||||
val dynamoDBHelper: DynamoDBHelper)
|
||||
extends GroupRepository
|
||||
with Monitored {
|
||||
|
||||
import DynamoDBGroupRepository._
|
||||
|
||||
val log: Logger = LoggerFactory.getLogger(classOf[DynamoDBGroupRepository])
|
||||
|
||||
def save(group: Group): IO[Group] =
|
||||
monitor("repo.Group.save") {
|
||||
log.info(s"Saving group ${group.id} ${group.name}.")
|
||||
val item = toItem(group)
|
||||
val request = new PutItemRequest().withTableName(GROUP_TABLE).withItem(item)
|
||||
val request = new PutItemRequest().withTableName(groupTableName).withItem(item)
|
||||
dynamoDBHelper.putItem(request).map(_ => group)
|
||||
}
|
||||
|
||||
@ -97,7 +104,7 @@ class DynamoDBGroupRepository(config: Config, dynamoDBHelper: DynamoDBHelper)
|
||||
log.info(s"Getting group $groupId.")
|
||||
val key = new HashMap[String, AttributeValue]()
|
||||
key.put(GROUP_ID, new AttributeValue(groupId))
|
||||
val request = new GetItemRequest().withTableName(GROUP_TABLE).withKey(key)
|
||||
val request = new GetItemRequest().withTableName(groupTableName).withKey(key)
|
||||
|
||||
dynamoDBHelper
|
||||
.getItem(request)
|
||||
@ -124,13 +131,13 @@ class DynamoDBGroupRepository(config: Config, dynamoDBHelper: DynamoDBHelper)
|
||||
val keysAndAttributes = new KeysAndAttributes().withKeys(allKeys)
|
||||
|
||||
val request = new util.HashMap[String, KeysAndAttributes]()
|
||||
request.put(GROUP_TABLE, keysAndAttributes)
|
||||
request.put(groupTableName, keysAndAttributes)
|
||||
|
||||
new BatchGetItemRequest().withRequestItems(request)
|
||||
}
|
||||
|
||||
def parseGroups(result: BatchGetItemResult): Set[Group] = {
|
||||
val groupAttributes = result.getResponses.asScala.get(GROUP_TABLE)
|
||||
val groupAttributes = result.getResponses.asScala.get(groupTableName)
|
||||
groupAttributes match {
|
||||
case None =>
|
||||
Set()
|
||||
@ -163,7 +170,7 @@ class DynamoDBGroupRepository(config: Config, dynamoDBHelper: DynamoDBHelper)
|
||||
def getAllGroups(): IO[Set[Group]] =
|
||||
monitor("repo.Group.getAllGroups") {
|
||||
log.info(s"getting all group IDs")
|
||||
val scanRequest = new ScanRequest().withTableName(GROUP_TABLE)
|
||||
val scanRequest = new ScanRequest().withTableName(groupTableName)
|
||||
dynamoDBHelper.scanAll(scanRequest).map { results =>
|
||||
val startTime = System.currentTimeMillis()
|
||||
val groups = results
|
||||
@ -189,7 +196,7 @@ class DynamoDBGroupRepository(config: Config, dynamoDBHelper: DynamoDBHelper)
|
||||
val keyConditionExpression: String = "#name_attribute = :name"
|
||||
|
||||
val queryRequest = new QueryRequest()
|
||||
.withTableName(GROUP_TABLE)
|
||||
.withTableName(groupTableName)
|
||||
.withIndexName(GROUP_NAME_INDEX)
|
||||
.withExpressionAttributeNames(expressionAttributeNames)
|
||||
.withExpressionAttributeValues(expressionAttributeValues)
|
||||
|
@ -39,8 +39,8 @@ class UnexpectedDynamoResponseException(message: String, cause: Throwable)
|
||||
|
||||
trait DynamoUtils {
|
||||
|
||||
def createTableIfNotExists(dynanmoDB: AmazonDynamoDBClient, req: CreateTableRequest): Boolean
|
||||
def waitUntilActive(dynamoDB: AmazonDynamoDBClient, tableName: String): Unit
|
||||
def createTableIfNotExists(dynamoDB: AmazonDynamoDBClient, req: CreateTableRequest): IO[Boolean]
|
||||
def waitUntilActive(dynamoDB: AmazonDynamoDBClient, tableName: String): IO[Unit]
|
||||
}
|
||||
|
||||
/* Used to provide an exponential backoff in the event of a Provisioned Throughput Exception */
|
||||
@ -57,11 +57,13 @@ class DynamoDBHelper(dynamoDB: AmazonDynamoDBClient, log: Logger) {
|
||||
VinylDNSMetrics.metricsRegistry.meter("dynamo.unexpectedFailure")
|
||||
private[repository] val callRateMeter = VinylDNSMetrics.metricsRegistry.meter("dynamo.callRate")
|
||||
private[repository] val dynamoUtils = new DynamoUtils {
|
||||
def waitUntilActive(dynamoDB: AmazonDynamoDBClient, tableName: String): Unit =
|
||||
TableUtils.waitUntilActive(dynamoDB, tableName)
|
||||
def waitUntilActive(dynamoDB: AmazonDynamoDBClient, tableName: String): IO[Unit] =
|
||||
IO(TableUtils.waitUntilActive(dynamoDB, tableName))
|
||||
|
||||
def createTableIfNotExists(dynamoDB: AmazonDynamoDBClient, req: CreateTableRequest): Boolean =
|
||||
TableUtils.createTableIfNotExists(dynamoDB, req)
|
||||
def createTableIfNotExists(
|
||||
dynamoDB: AmazonDynamoDBClient,
|
||||
req: CreateTableRequest): IO[Boolean] =
|
||||
IO(TableUtils.createTableIfNotExists(dynamoDB, req))
|
||||
}
|
||||
|
||||
def shutdown(): Unit = dynamoDB.shutdown()
|
||||
@ -142,12 +144,14 @@ class DynamoDBHelper(dynamoDB: AmazonDynamoDBClient, log: Logger) {
|
||||
}
|
||||
}
|
||||
|
||||
def setupTable(createTableRequest: CreateTableRequest): Unit = {
|
||||
if (!dynamoUtils.createTableIfNotExists(dynamoDB, createTableRequest)) {
|
||||
def setupTable(createTableRequest: CreateTableRequest): IO[Unit] =
|
||||
for {
|
||||
tableCreated <- dynamoUtils.createTableIfNotExists(dynamoDB, createTableRequest)
|
||||
_ = if (!tableCreated) {
|
||||
log.info(s"Table ${createTableRequest.getTableName} already exists")
|
||||
}
|
||||
dynamoUtils.waitUntilActive(dynamoDB, createTableRequest.getTableName())
|
||||
}
|
||||
_ <- dynamoUtils.waitUntilActive(dynamoDB, createTableRequest.getTableName)
|
||||
} yield ()
|
||||
|
||||
def listTables(aws: ListTablesRequest): IO[ListTablesResult] =
|
||||
send[ListTablesRequest, ListTablesResult](aws, dynamoDB.listTables)
|
||||
|
@ -19,8 +19,8 @@ package vinyldns.dynamodb.repository
|
||||
import java.util.{Collections, HashMap}
|
||||
|
||||
import cats.effect._
|
||||
import cats.implicits._
|
||||
import com.amazonaws.services.dynamodbv2.model._
|
||||
import com.typesafe.config.Config
|
||||
import org.slf4j.{Logger, LoggerFactory}
|
||||
import vinyldns.core.domain.membership.MembershipRepository
|
||||
import vinyldns.core.route.Monitored
|
||||
@ -29,36 +29,29 @@ import scala.collection.JavaConverters._
|
||||
|
||||
object DynamoDBMembershipRepository {
|
||||
|
||||
def apply(config: Config, dynamoConfig: Config): DynamoDBMembershipRepository =
|
||||
new DynamoDBMembershipRepository(
|
||||
config,
|
||||
new DynamoDBHelper(
|
||||
DynamoDBClient(dynamoConfig),
|
||||
LoggerFactory.getLogger("DynamoDBMembershipRepository")))
|
||||
}
|
||||
|
||||
class DynamoDBMembershipRepository(config: Config, dynamoDBHelper: DynamoDBHelper)
|
||||
extends MembershipRepository
|
||||
with Monitored {
|
||||
|
||||
val log: Logger = LoggerFactory.getLogger("DynamoDBMembershipRepository")
|
||||
|
||||
private[repository] val USER_ID = "user_id"
|
||||
private[repository] val GROUP_ID = "group_id"
|
||||
|
||||
private val dynamoReads = config.getLong("dynamo.provisionedReads")
|
||||
private val dynamoWrites = config.getLong("dynamo.provisionedWrites")
|
||||
private[repository] val membershipTable = config.getString("dynamo.tableName")
|
||||
def apply(
|
||||
config: DynamoDBRepositorySettings,
|
||||
dynamoConfig: DynamoDBDataStoreSettings): IO[DynamoDBMembershipRepository] = {
|
||||
|
||||
private[repository] val tableAttributes = Seq(
|
||||
val dynamoDBHelper = new DynamoDBHelper(
|
||||
DynamoDBClient(dynamoConfig),
|
||||
LoggerFactory.getLogger("DynamoDBMembershipRepository"))
|
||||
|
||||
val dynamoReads = config.provisionedReads
|
||||
val dynamoWrites = config.provisionedWrites
|
||||
val tableName = config.tableName
|
||||
|
||||
val tableAttributes = Seq(
|
||||
new AttributeDefinition(USER_ID, "S"),
|
||||
new AttributeDefinition(GROUP_ID, "S")
|
||||
)
|
||||
|
||||
private[repository] val secondaryIndexes = Seq()
|
||||
dynamoDBHelper.setupTable(
|
||||
val setup = dynamoDBHelper.setupTable(
|
||||
new CreateTableRequest()
|
||||
.withTableName(membershipTable)
|
||||
.withTableName(tableName)
|
||||
.withAttributeDefinitions(tableAttributes: _*)
|
||||
.withKeySchema(
|
||||
new KeySchemaElement(USER_ID, KeyType.HASH),
|
||||
@ -66,6 +59,20 @@ class DynamoDBMembershipRepository(config: Config, dynamoDBHelper: DynamoDBHelpe
|
||||
.withProvisionedThroughput(new ProvisionedThroughput(dynamoReads, dynamoWrites))
|
||||
)
|
||||
|
||||
setup.as(new DynamoDBMembershipRepository(tableName, dynamoDBHelper))
|
||||
}
|
||||
}
|
||||
|
||||
class DynamoDBMembershipRepository private[repository] (
|
||||
membershipTable: String,
|
||||
dynamoDBHelper: DynamoDBHelper)
|
||||
extends MembershipRepository
|
||||
with Monitored {
|
||||
|
||||
import DynamoDBMembershipRepository._
|
||||
|
||||
val log: Logger = LoggerFactory.getLogger("DynamoDBMembershipRepository")
|
||||
|
||||
def getGroupsForUser(userId: String): IO[Set[String]] =
|
||||
monitor("repo.Membership.getGroupsForUser") {
|
||||
log.info(s"Getting groups by user id $userId")
|
||||
|
@ -22,7 +22,6 @@ import java.util.HashMap
|
||||
import cats.effect._
|
||||
import cats.implicits._
|
||||
import com.amazonaws.services.dynamodbv2.model._
|
||||
import com.typesafe.config.Config
|
||||
import org.joda.time.DateTime
|
||||
import org.slf4j.{Logger, LoggerFactory}
|
||||
import vinyldns.core.domain.record._
|
||||
@ -35,20 +34,6 @@ import scala.collection.JavaConverters._
|
||||
import scala.util.Try
|
||||
|
||||
object DynamoDBRecordChangeRepository {
|
||||
def apply(config: Config, dynamoConfig: Config): DynamoDBRecordChangeRepository =
|
||||
new DynamoDBRecordChangeRepository(
|
||||
config,
|
||||
new DynamoDBHelper(
|
||||
DynamoDBClient(dynamoConfig),
|
||||
LoggerFactory.getLogger("DynamoDBRecordChangeRepository")))
|
||||
}
|
||||
|
||||
class DynamoDBRecordChangeRepository(config: Config, dynamoDBHelper: DynamoDBHelper)
|
||||
extends RecordChangeRepository
|
||||
with ProtobufConversions
|
||||
with Monitored {
|
||||
|
||||
val log: Logger = LoggerFactory.getLogger("DynamoDBRecordChangeRepository")
|
||||
|
||||
private val CHANGE_SET_ID = "change_set_id"
|
||||
private[repository] val RECORD_SET_CHANGE_ID = "record_set_change_id"
|
||||
@ -63,18 +48,28 @@ class DynamoDBRecordChangeRepository(config: Config, dynamoDBHelper: DynamoDBHel
|
||||
private val CHANGE_STATUS_ZONE_ID_INDEX = "change_status_index"
|
||||
private val ZONE_ID_CREATED_INDEX = "zone_id_created_index"
|
||||
|
||||
private val dynamoReads = config.getLong("dynamo.provisionedReads")
|
||||
private val dynamoWrites = config.getLong("dynamo.provisionedWrites")
|
||||
private[repository] val recordChangeTable = config.getString("dynamo.tableName")
|
||||
def apply(
|
||||
config: DynamoDBRepositorySettings,
|
||||
dynamoConfig: DynamoDBDataStoreSettings): IO[DynamoDBRecordChangeRepository] = {
|
||||
|
||||
private[repository] val tableAttributes = Seq(
|
||||
val dynamoDBHelper = new DynamoDBHelper(
|
||||
DynamoDBClient(dynamoConfig),
|
||||
LoggerFactory.getLogger("DynamoDBRecordChangeRepository"))
|
||||
|
||||
val dynamoReads = config.provisionedReads
|
||||
val dynamoWrites = config.provisionedWrites
|
||||
val tableName = config.tableName
|
||||
|
||||
val tableAttributes =
|
||||
Seq(
|
||||
new AttributeDefinition(RECORD_SET_CHANGE_ID, "S"),
|
||||
new AttributeDefinition(CHANGE_SET_STATUS, "N"),
|
||||
new AttributeDefinition(ZONE_ID, "S"),
|
||||
new AttributeDefinition(RECORD_SET_CHANGE_CREATED_TIMESTAMP, "N")
|
||||
)
|
||||
|
||||
private[repository] val secondaryIndexes = Seq(
|
||||
val secondaryIndexes =
|
||||
Seq(
|
||||
new GlobalSecondaryIndex()
|
||||
.withIndexName(ZONE_ID_CHANGE_STATUS_INDEX)
|
||||
.withProvisionedThroughput(new ProvisionedThroughput(dynamoReads, dynamoWrites))
|
||||
@ -105,15 +100,29 @@ class DynamoDBRecordChangeRepository(config: Config, dynamoDBHelper: DynamoDBHel
|
||||
.withProjection(new Projection().withProjectionType("ALL"))
|
||||
)
|
||||
|
||||
dynamoDBHelper.setupTable(
|
||||
val setup = dynamoDBHelper.setupTable(
|
||||
new CreateTableRequest()
|
||||
.withTableName(recordChangeTable)
|
||||
.withTableName(tableName)
|
||||
.withAttributeDefinitions(tableAttributes: _*)
|
||||
.withKeySchema(new KeySchemaElement(RECORD_SET_CHANGE_ID, KeyType.HASH))
|
||||
.withGlobalSecondaryIndexes(secondaryIndexes: _*)
|
||||
.withProvisionedThroughput(new ProvisionedThroughput(dynamoReads, dynamoWrites))
|
||||
)
|
||||
|
||||
setup.as(new DynamoDBRecordChangeRepository(tableName, dynamoDBHelper))
|
||||
}
|
||||
}
|
||||
|
||||
class DynamoDBRecordChangeRepository private[repository] (
|
||||
recordChangeTable: String,
|
||||
val dynamoDBHelper: DynamoDBHelper)
|
||||
extends RecordChangeRepository
|
||||
with ProtobufConversions
|
||||
with Monitored {
|
||||
|
||||
import DynamoDBRecordChangeRepository._
|
||||
val log: Logger = LoggerFactory.getLogger("DynamoDBRecordChangeRepository")
|
||||
|
||||
def toWriteRequest(changeSet: ChangeSet, change: RecordSetChange): WriteRequest =
|
||||
new WriteRequest().withPutRequest(new PutRequest().withItem(toItem(changeSet, change)))
|
||||
|
||||
|
@ -19,8 +19,8 @@ package vinyldns.dynamodb.repository
|
||||
import java.util.HashMap
|
||||
|
||||
import cats.effect._
|
||||
import cats.implicits._
|
||||
import com.amazonaws.services.dynamodbv2.model._
|
||||
import com.typesafe.config.Config
|
||||
import org.slf4j.{Logger, LoggerFactory}
|
||||
import vinyldns.core.domain.DomainHelpers.omitTrailingDot
|
||||
import vinyldns.core.domain.record.RecordType.RecordType
|
||||
@ -36,41 +36,29 @@ object DynamoDBRecordSetRepository extends ProtobufConversions {
|
||||
private[repository] val RECORD_SET_NAME = "record_set_name"
|
||||
private[repository] val RECORD_SET_SORT = "record_set_sort"
|
||||
private[repository] val RECORD_SET_BLOB = "record_set_blob"
|
||||
|
||||
def apply(config: Config, dynamoConfig: Config): DynamoDBRecordSetRepository =
|
||||
new DynamoDBRecordSetRepository(
|
||||
config,
|
||||
new DynamoDBHelper(
|
||||
DynamoDBClient(dynamoConfig),
|
||||
LoggerFactory.getLogger("DynamoDBRecordSetRepository")))
|
||||
|
||||
}
|
||||
|
||||
class DynamoDBRecordSetRepository(config: Config, dynamoDBHelper: DynamoDBHelper)
|
||||
extends RecordSetRepository
|
||||
with DynamoDBRecordSetConversions
|
||||
with Monitored
|
||||
with QueryHelper {
|
||||
|
||||
import DynamoDBRecordSetRepository._
|
||||
|
||||
private val ZONE_ID_RECORD_SET_NAME_INDEX = "zone_id_record_set_name_index"
|
||||
private val ZONE_ID_RECORD_SET_SORT_INDEX = "zone_id_record_set_sort_index"
|
||||
|
||||
private val dynamoReads = config.getLong("dynamo.provisionedReads")
|
||||
private val dynamoWrites = config.getLong("dynamo.provisionedWrites")
|
||||
def apply(
|
||||
config: DynamoDBRepositorySettings,
|
||||
dynamoConfig: DynamoDBDataStoreSettings): IO[DynamoDBRecordSetRepository] = {
|
||||
|
||||
val log: Logger = LoggerFactory.getLogger("DynamoDBRecordSetRepository")
|
||||
private[repository] val recordSetTableName: String = config.getString("dynamo.tableName")
|
||||
val dynamoDBHelper = new DynamoDBHelper(
|
||||
DynamoDBClient(dynamoConfig),
|
||||
LoggerFactory.getLogger("DynamoDBRecordSetRepository"))
|
||||
|
||||
private[repository] val tableAttributes = Seq(
|
||||
val dynamoReads = config.provisionedReads
|
||||
val dynamoWrites = config.provisionedWrites
|
||||
val tableName = config.tableName
|
||||
|
||||
val tableAttributes = Seq(
|
||||
new AttributeDefinition(ZONE_ID, "S"),
|
||||
new AttributeDefinition(RECORD_SET_NAME, "S"),
|
||||
new AttributeDefinition(RECORD_SET_ID, "S"),
|
||||
new AttributeDefinition(RECORD_SET_SORT, "S")
|
||||
)
|
||||
|
||||
private[repository] val secondaryIndexes = Seq(
|
||||
val secondaryIndexes = Seq(
|
||||
new GlobalSecondaryIndex()
|
||||
.withIndexName(ZONE_ID_RECORD_SET_NAME_INDEX)
|
||||
.withProvisionedThroughput(new ProvisionedThroughput(dynamoReads, dynamoWrites))
|
||||
@ -87,15 +75,31 @@ class DynamoDBRecordSetRepository(config: Config, dynamoDBHelper: DynamoDBHelper
|
||||
.withProjection(new Projection().withProjectionType("ALL"))
|
||||
)
|
||||
|
||||
dynamoDBHelper.setupTable(
|
||||
val setup = dynamoDBHelper.setupTable(
|
||||
new CreateTableRequest()
|
||||
.withTableName(recordSetTableName)
|
||||
.withTableName(tableName)
|
||||
.withAttributeDefinitions(tableAttributes: _*)
|
||||
.withKeySchema(new KeySchemaElement(RECORD_SET_ID, KeyType.HASH))
|
||||
.withGlobalSecondaryIndexes(secondaryIndexes: _*)
|
||||
.withProvisionedThroughput(new ProvisionedThroughput(dynamoReads, dynamoWrites))
|
||||
)
|
||||
|
||||
setup.as(new DynamoDBRecordSetRepository(tableName, dynamoDBHelper))
|
||||
}
|
||||
}
|
||||
|
||||
class DynamoDBRecordSetRepository private[repository] (
|
||||
val recordSetTableName: String,
|
||||
val dynamoDBHelper: DynamoDBHelper)
|
||||
extends RecordSetRepository
|
||||
with DynamoDBRecordSetConversions
|
||||
with Monitored
|
||||
with QueryHelper {
|
||||
|
||||
import DynamoDBRecordSetRepository._
|
||||
|
||||
val log: Logger = LoggerFactory.getLogger("DynamoDBRecordSetRepository")
|
||||
|
||||
def apply(changeSet: ChangeSet): IO[ChangeSet] =
|
||||
monitor("repo.RecordSet.apply") {
|
||||
log.info(
|
||||
|
@ -22,7 +22,6 @@ import java.util.HashMap
|
||||
import cats.effect._
|
||||
import cats.implicits._
|
||||
import com.amazonaws.services.dynamodbv2.model._
|
||||
import com.typesafe.config.Config
|
||||
import org.joda.time.DateTime
|
||||
import org.slf4j.{Logger, LoggerFactory}
|
||||
import vinyldns.core.domain.membership.{ListUsersResults, User, UserRepository}
|
||||
@ -32,20 +31,6 @@ import scala.collection.JavaConverters._
|
||||
|
||||
object DynamoDBUserRepository {
|
||||
|
||||
def apply(config: Config, dynamoConfig: Config): DynamoDBUserRepository =
|
||||
new DynamoDBUserRepository(
|
||||
config,
|
||||
new DynamoDBHelper(
|
||||
DynamoDBClient(dynamoConfig),
|
||||
LoggerFactory.getLogger("DynamoDBUserRepository")))
|
||||
}
|
||||
|
||||
class DynamoDBUserRepository(config: Config, dynamoDBHelper: DynamoDBHelper)
|
||||
extends UserRepository
|
||||
with Monitored {
|
||||
|
||||
val log: Logger = LoggerFactory.getLogger(classOf[DynamoDBUserRepository])
|
||||
|
||||
private[repository] val USER_ID = "userid"
|
||||
private[repository] val USER_NAME = "username"
|
||||
private[repository] val FIRST_NAME = "firstname"
|
||||
@ -58,17 +43,25 @@ class DynamoDBUserRepository(config: Config, dynamoDBHelper: DynamoDBHelper)
|
||||
private[repository] val USER_NAME_INDEX_NAME = "username_index"
|
||||
private[repository] val ACCESS_KEY_INDEX_NAME = "access_key_index"
|
||||
|
||||
private val dynamoReads = config.getLong("dynamo.provisionedReads")
|
||||
private val dynamoWrites = config.getLong("dynamo.provisionedWrites")
|
||||
private[repository] val USER_TABLE = config.getString("dynamo.tableName")
|
||||
def apply(
|
||||
config: DynamoDBRepositorySettings,
|
||||
dynamoConfig: DynamoDBDataStoreSettings): IO[DynamoDBUserRepository] = {
|
||||
|
||||
private[repository] val tableAttributes = Seq(
|
||||
val dynamoDBHelper = new DynamoDBHelper(
|
||||
DynamoDBClient(dynamoConfig),
|
||||
LoggerFactory.getLogger("DynamoDBUserRepository"))
|
||||
|
||||
val dynamoReads = config.provisionedReads
|
||||
val dynamoWrites = config.provisionedWrites
|
||||
val tableName = config.tableName
|
||||
|
||||
val tableAttributes = Seq(
|
||||
new AttributeDefinition(USER_ID, "S"),
|
||||
new AttributeDefinition(USER_NAME, "S"),
|
||||
new AttributeDefinition(ACCESS_KEY, "S")
|
||||
)
|
||||
|
||||
private[repository] val secondaryIndexes = Seq(
|
||||
val secondaryIndexes = Seq(
|
||||
new GlobalSecondaryIndex()
|
||||
.withIndexName(USER_NAME_INDEX_NAME)
|
||||
.withProvisionedThroughput(new ProvisionedThroughput(dynamoReads, dynamoWrites))
|
||||
@ -81,22 +74,35 @@ class DynamoDBUserRepository(config: Config, dynamoDBHelper: DynamoDBHelper)
|
||||
.withProjection(new Projection().withProjectionType("ALL"))
|
||||
)
|
||||
|
||||
dynamoDBHelper.setupTable(
|
||||
val setup = dynamoDBHelper.setupTable(
|
||||
new CreateTableRequest()
|
||||
.withTableName(USER_TABLE)
|
||||
.withTableName(tableName)
|
||||
.withAttributeDefinitions(tableAttributes: _*)
|
||||
.withKeySchema(new KeySchemaElement(USER_ID, KeyType.HASH))
|
||||
.withProvisionedThroughput(new ProvisionedThroughput(dynamoReads, dynamoWrites))
|
||||
.withGlobalSecondaryIndexes(secondaryIndexes: _*)
|
||||
)
|
||||
|
||||
setup.as(new DynamoDBUserRepository(tableName, dynamoDBHelper))
|
||||
}
|
||||
}
|
||||
|
||||
class DynamoDBUserRepository private[repository] (
|
||||
userTableName: String,
|
||||
val dynamoDBHelper: DynamoDBHelper)
|
||||
extends UserRepository
|
||||
with Monitored {
|
||||
|
||||
import DynamoDBUserRepository._
|
||||
val log: Logger = LoggerFactory.getLogger(classOf[DynamoDBUserRepository])
|
||||
|
||||
def getUser(userId: String): IO[Option[User]] =
|
||||
monitor("repo.User.getUser") {
|
||||
log.info(s"Getting user by id $userId")
|
||||
|
||||
val key = new HashMap[String, AttributeValue]()
|
||||
key.put(USER_ID, new AttributeValue(userId))
|
||||
val request = new GetItemRequest().withTableName(USER_TABLE).withKey(key)
|
||||
val request = new GetItemRequest().withTableName(userTableName).withKey(key)
|
||||
|
||||
dynamoDBHelper.getItem(request).map(result => Option(result.getItem).map(fromItem))
|
||||
}
|
||||
@ -118,13 +124,13 @@ class DynamoDBUserRepository(config: Config, dynamoDBHelper: DynamoDBHelper)
|
||||
val keysAndAttributes = new KeysAndAttributes().withKeys(allKeys)
|
||||
|
||||
val request = new util.HashMap[String, KeysAndAttributes]()
|
||||
request.put(USER_TABLE, keysAndAttributes)
|
||||
request.put(userTableName, keysAndAttributes)
|
||||
|
||||
new BatchGetItemRequest().withRequestItems(request)
|
||||
}
|
||||
|
||||
def parseUsers(result: BatchGetItemResult): List[User] = {
|
||||
val userAttributes = result.getResponses.asScala.get(USER_TABLE)
|
||||
val userAttributes = result.getResponses.asScala.get(userTableName)
|
||||
userAttributes match {
|
||||
case None =>
|
||||
List()
|
||||
@ -182,7 +188,7 @@ class DynamoDBUserRepository(config: Config, dynamoDBHelper: DynamoDBHelper)
|
||||
val keyConditionExpression: String = "#access_key_attribute = :access_key"
|
||||
|
||||
val queryRequest = new QueryRequest()
|
||||
.withTableName(USER_TABLE)
|
||||
.withTableName(userTableName)
|
||||
.withIndexName(ACCESS_KEY_INDEX_NAME)
|
||||
.withExpressionAttributeNames(expressionAttributeNames)
|
||||
.withExpressionAttributeValues(expressionAttributeValues)
|
||||
@ -198,7 +204,7 @@ class DynamoDBUserRepository(config: Config, dynamoDBHelper: DynamoDBHelper)
|
||||
log.info(s"Saving user id: ${user.id} name: ${user.userName}.")
|
||||
|
||||
val item = toItem(user)
|
||||
val request = new PutItemRequest().withTableName(USER_TABLE).withItem(item)
|
||||
val request = new PutItemRequest().withTableName(userTableName).withItem(item)
|
||||
dynamoDBHelper.putItem(request).map(_ => user)
|
||||
}
|
||||
|
||||
|
@ -20,8 +20,8 @@ import java.nio.ByteBuffer
|
||||
import java.util.HashMap
|
||||
|
||||
import cats.effect._
|
||||
import cats.implicits._
|
||||
import com.amazonaws.services.dynamodbv2.model._
|
||||
import com.typesafe.config.Config
|
||||
import org.joda.time.DateTime
|
||||
import org.slf4j.LoggerFactory
|
||||
import vinyldns.core.domain.zone.ZoneChangeStatus.ZoneChangeStatus
|
||||
@ -39,23 +39,6 @@ import scala.util.Try
|
||||
|
||||
object DynamoDBZoneChangeRepository extends ProtobufConversions {
|
||||
|
||||
def apply(config: Config, dynamoConfig: Config): DynamoDBZoneChangeRepository =
|
||||
new DynamoDBZoneChangeRepository(
|
||||
config,
|
||||
new DynamoDBHelper(
|
||||
DynamoDBClient(dynamoConfig),
|
||||
LoggerFactory.getLogger("DynamoDBZoneChangeRepository")))
|
||||
}
|
||||
|
||||
class DynamoDBZoneChangeRepository(config: Config, dynamoDBHelper: DynamoDBHelper)
|
||||
extends ZoneChangeRepository
|
||||
with ProtobufConversions
|
||||
with Monitored {
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
implicit def dateTimeOrdering: Ordering[DateTime] = Ordering.fromLessThan(_.isAfter(_))
|
||||
|
||||
private[repository] val ZONE_ID = "zone_id"
|
||||
private[repository] val CHANGE_ID = "change_id"
|
||||
private[repository] val STATUS = "status"
|
||||
@ -66,19 +49,26 @@ class DynamoDBZoneChangeRepository(config: Config, dynamoDBHelper: DynamoDBHelpe
|
||||
private val STATUS_INDEX_NAME = "status_zone_id_index"
|
||||
private val ZONE_ID_CREATED_INDEX = "zone_id_created_index"
|
||||
|
||||
val log = LoggerFactory.getLogger(classOf[DynamoDBZoneChangeRepository])
|
||||
private val dynamoReads = config.getLong("dynamo.provisionedReads")
|
||||
private val dynamoWrites = config.getLong("dynamo.provisionedWrites")
|
||||
private[repository] val zoneChangeTable = config.getString("dynamo.tableName")
|
||||
def apply(
|
||||
config: DynamoDBRepositorySettings,
|
||||
dynamoConfig: DynamoDBDataStoreSettings): IO[DynamoDBZoneChangeRepository] = {
|
||||
|
||||
private[repository] val tableAttributes = Seq(
|
||||
val dynamoDBHelper = new DynamoDBHelper(
|
||||
DynamoDBClient(dynamoConfig),
|
||||
LoggerFactory.getLogger("DynamoDBZoneChangeRepository"))
|
||||
|
||||
val dynamoReads = config.provisionedReads
|
||||
val dynamoWrites = config.provisionedWrites
|
||||
val tableName = config.tableName
|
||||
|
||||
val tableAttributes = Seq(
|
||||
new AttributeDefinition(CHANGE_ID, "S"),
|
||||
new AttributeDefinition(ZONE_ID, "S"),
|
||||
new AttributeDefinition(STATUS, "S"),
|
||||
new AttributeDefinition(CREATED, "N")
|
||||
)
|
||||
|
||||
private[repository] val secondaryIndexes = Seq(
|
||||
val secondaryIndexes = Seq(
|
||||
new GlobalSecondaryIndex()
|
||||
.withIndexName(ZONE_ID_STATUS_INDEX_NAME)
|
||||
.withProvisionedThroughput(new ProvisionedThroughput(dynamoReads, dynamoWrites))
|
||||
@ -102,9 +92,9 @@ class DynamoDBZoneChangeRepository(config: Config, dynamoDBHelper: DynamoDBHelpe
|
||||
.withProjection(new Projection().withProjectionType("ALL"))
|
||||
)
|
||||
|
||||
dynamoDBHelper.setupTable(
|
||||
val setup = dynamoDBHelper.setupTable(
|
||||
new CreateTableRequest()
|
||||
.withTableName(zoneChangeTable)
|
||||
.withTableName(tableName)
|
||||
.withAttributeDefinitions(tableAttributes: _*)
|
||||
.withKeySchema(
|
||||
new KeySchemaElement(ZONE_ID, KeyType.HASH),
|
||||
@ -113,6 +103,24 @@ class DynamoDBZoneChangeRepository(config: Config, dynamoDBHelper: DynamoDBHelpe
|
||||
.withProvisionedThroughput(new ProvisionedThroughput(dynamoReads, dynamoWrites))
|
||||
)
|
||||
|
||||
setup.as(new DynamoDBZoneChangeRepository(tableName, dynamoDBHelper))
|
||||
}
|
||||
}
|
||||
|
||||
class DynamoDBZoneChangeRepository private[repository] (
|
||||
zoneChangeTable: String,
|
||||
val dynamoDBHelper: DynamoDBHelper)
|
||||
extends ZoneChangeRepository
|
||||
with ProtobufConversions
|
||||
with Monitored {
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
import DynamoDBZoneChangeRepository._
|
||||
|
||||
implicit def dateTimeOrdering: Ordering[DateTime] = Ordering.fromLessThan(_.isAfter(_))
|
||||
|
||||
val log = LoggerFactory.getLogger(classOf[DynamoDBZoneChangeRepository])
|
||||
|
||||
def save(zoneChange: ZoneChange): IO[ZoneChange] =
|
||||
monitor("repo.ZoneChange.save") {
|
||||
log.info(s"Saving zone change ${zoneChange.id}")
|
||||
|
@ -17,18 +17,39 @@
|
||||
package vinyldns.dynamodb
|
||||
|
||||
import com.typesafe.config.{Config, ConfigFactory}
|
||||
import pureconfig.{CamelCase, ConfigFieldMapping, ProductHint}
|
||||
import vinyldns.dynamodb.repository.{DynamoDBDataStoreSettings, DynamoDBRepositorySettings}
|
||||
|
||||
object DynamoTestConfig {
|
||||
|
||||
lazy val config: Config = ConfigFactory.load()
|
||||
lazy val vinyldnsConfig: Config = config.getConfig("vinyldns")
|
||||
lazy val dynamoConfig: Config = vinyldnsConfig.getConfig("dynamo")
|
||||
lazy val zoneChangeStoreConfig: Config = vinyldnsConfig.getConfig("zoneChanges")
|
||||
lazy val recordSetStoreConfig: Config = vinyldnsConfig.getConfig("recordSet")
|
||||
lazy val recordChangeStoreConfig: Config = vinyldnsConfig.getConfig("recordChange")
|
||||
lazy val usersStoreConfig: Config = vinyldnsConfig.getConfig("users")
|
||||
lazy val groupsStoreConfig: Config = vinyldnsConfig.getConfig("groups")
|
||||
lazy val groupChangesStoreConfig: Config = vinyldnsConfig.getConfig("groupChanges")
|
||||
lazy val membershipStoreConfig: Config = vinyldnsConfig.getConfig("membership")
|
||||
|
||||
lazy val dynamoConfig: DynamoDBDataStoreSettings =
|
||||
pureconfig.loadConfigOrThrow[DynamoDBDataStoreSettings](vinyldnsConfig, "dynamo")
|
||||
|
||||
// TODO these will change when dynamically loaded
|
||||
implicit def hint[T]: ProductHint[T] = ProductHint[T](ConfigFieldMapping(CamelCase, CamelCase))
|
||||
|
||||
lazy val zoneChangeStoreConfig: DynamoDBRepositorySettings =
|
||||
pureconfig.loadConfigOrThrow[DynamoDBRepositorySettings](
|
||||
vinyldnsConfig.getConfig("zoneChanges.dynamo"))
|
||||
lazy val recordSetStoreConfig: DynamoDBRepositorySettings =
|
||||
pureconfig.loadConfigOrThrow[DynamoDBRepositorySettings](
|
||||
vinyldnsConfig.getConfig("recordSet.dynamo"))
|
||||
lazy val recordChangeStoreConfig: DynamoDBRepositorySettings =
|
||||
pureconfig.loadConfigOrThrow[DynamoDBRepositorySettings](
|
||||
vinyldnsConfig.getConfig("recordChange.dynamo"))
|
||||
lazy val usersStoreConfig: DynamoDBRepositorySettings =
|
||||
pureconfig.loadConfigOrThrow[DynamoDBRepositorySettings](
|
||||
vinyldnsConfig.getConfig("users.dynamo"))
|
||||
lazy val groupsStoreConfig: DynamoDBRepositorySettings =
|
||||
pureconfig.loadConfigOrThrow[DynamoDBRepositorySettings](
|
||||
vinyldnsConfig.getConfig("groups.dynamo"))
|
||||
lazy val groupChangesStoreConfig: DynamoDBRepositorySettings =
|
||||
pureconfig.loadConfigOrThrow[DynamoDBRepositorySettings](
|
||||
vinyldnsConfig.getConfig("groupChanges.dynamo"))
|
||||
lazy val membershipStoreConfig: DynamoDBRepositorySettings =
|
||||
pureconfig.loadConfigOrThrow[DynamoDBRepositorySettings](
|
||||
vinyldnsConfig.getConfig("membership.dynamo"))
|
||||
}
|
||||
|
@ -1,42 +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 vinyldns.dynamodb.repository
|
||||
|
||||
import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClient
|
||||
import com.typesafe.config.Config
|
||||
import org.mockito.Mockito._
|
||||
import org.scalatest.mockito.MockitoSugar
|
||||
import org.scalatest.{Matchers, WordSpec}
|
||||
|
||||
class DynamoDBClientSpec extends WordSpec with Matchers with MockitoSugar {
|
||||
|
||||
"DynamoDBClient" should {
|
||||
"create an AmazonDynamoDBClient" in {
|
||||
val mockConfig = mock[Config]
|
||||
doReturn("theKey").when(mockConfig).getString("key")
|
||||
doReturn("theSecret").when(mockConfig).getString("secret")
|
||||
doReturn("http://www.endpoint.com").when(mockConfig).getString("endpoint")
|
||||
|
||||
val client = DynamoDBClient(mockConfig)
|
||||
client shouldBe a[AmazonDynamoDBClient]
|
||||
|
||||
verify(mockConfig).getString("key")
|
||||
verify(mockConfig).getString("secret")
|
||||
verify(mockConfig).getString("endpoint")
|
||||
}
|
||||
}
|
||||
}
|
@ -17,7 +17,6 @@
|
||||
package vinyldns.dynamodb.repository
|
||||
|
||||
import com.amazonaws.services.dynamodbv2.model.{GetItemRequest, ResourceNotFoundException, _}
|
||||
import org.mockito.ArgumentCaptor
|
||||
import org.mockito.Matchers._
|
||||
import org.mockito.Mockito._
|
||||
import org.scalatest.concurrent.ScalaFutures
|
||||
@ -38,43 +37,14 @@ class DynamoDBGroupChangeRepositorySpec
|
||||
|
||||
private val dynamoDBHelper = mock[DynamoDBHelper]
|
||||
private val groupChangeStoreConfig = DynamoTestConfig.groupChangesStoreConfig
|
||||
private val groupChangeTable = groupChangeStoreConfig.getString("dynamo.tableName")
|
||||
private val groupChangeTable = groupChangeStoreConfig.tableName
|
||||
class TestDynamoDBGroupChangeRepository
|
||||
extends DynamoDBGroupChangeRepository(groupChangeStoreConfig, dynamoDBHelper)
|
||||
extends DynamoDBGroupChangeRepository(groupChangeTable, dynamoDBHelper)
|
||||
|
||||
private val underTest = new DynamoDBGroupChangeRepository(groupChangeStoreConfig, dynamoDBHelper)
|
||||
private val underTest = new DynamoDBGroupChangeRepository(groupChangeTable, dynamoDBHelper)
|
||||
|
||||
override def beforeEach(): Unit = {
|
||||
override def beforeEach(): Unit =
|
||||
reset(dynamoDBHelper)
|
||||
doNothing().when(dynamoDBHelper).setupTable(any[CreateTableRequest])
|
||||
}
|
||||
|
||||
"DynamoDBGroupChangeRepository constructor" should {
|
||||
|
||||
"call setup table when it is built" in {
|
||||
val setupTableCaptor = ArgumentCaptor.forClass(classOf[CreateTableRequest])
|
||||
|
||||
new TestDynamoDBGroupChangeRepository
|
||||
verify(dynamoDBHelper).setupTable(setupTableCaptor.capture())
|
||||
|
||||
val createTable = setupTableCaptor.getValue
|
||||
|
||||
createTable.getTableName shouldBe groupChangeTable
|
||||
(createTable.getAttributeDefinitions should contain).only(underTest.tableAttributes: _*)
|
||||
createTable.getKeySchema.get(0).getAttributeName shouldBe underTest.GROUP_CHANGE_ID
|
||||
createTable.getKeySchema.get(0).getKeyType shouldBe KeyType.HASH.toString
|
||||
createTable.getGlobalSecondaryIndexes.toArray() shouldBe underTest.secondaryIndexes.toArray
|
||||
createTable.getProvisionedThroughput.getReadCapacityUnits shouldBe 30L
|
||||
createTable.getProvisionedThroughput.getWriteCapacityUnits shouldBe 30L
|
||||
}
|
||||
|
||||
"fail when an exception is thrown setting up the table" in {
|
||||
|
||||
doThrow(new RuntimeException("fail")).when(dynamoDBHelper).setupTable(any[CreateTableRequest])
|
||||
|
||||
a[RuntimeException] should be thrownBy new TestDynamoDBGroupChangeRepository
|
||||
}
|
||||
}
|
||||
|
||||
"DynamoDBGroupChangeRepository.toItem and fromItem" should {
|
||||
"work with all values set" in {
|
||||
|
@ -17,7 +17,6 @@
|
||||
package vinyldns.dynamodb.repository
|
||||
|
||||
import com.amazonaws.services.dynamodbv2.model.{GetItemRequest, ResourceNotFoundException, _}
|
||||
import org.mockito.ArgumentCaptor
|
||||
import org.mockito.Matchers._
|
||||
import org.mockito.Mockito._
|
||||
import org.scalatest.concurrent.ScalaFutures
|
||||
@ -38,43 +37,12 @@ class DynamoDBGroupRepositorySpec
|
||||
|
||||
private val dynamoDBHelper = mock[DynamoDBHelper]
|
||||
private val groupsStoreConfig = DynamoTestConfig.groupsStoreConfig
|
||||
private val membershipTable = groupsStoreConfig.getString("dynamo.tableName")
|
||||
private val groupsTable = groupsStoreConfig.tableName
|
||||
|
||||
private val underTest = new DynamoDBGroupRepository(groupsStoreConfig, dynamoDBHelper)
|
||||
private val underTest = new DynamoDBGroupRepository(groupsTable, dynamoDBHelper)
|
||||
|
||||
override def beforeEach(): Unit = {
|
||||
override def beforeEach(): Unit =
|
||||
reset(dynamoDBHelper)
|
||||
doNothing().when(dynamoDBHelper).setupTable(any[CreateTableRequest])
|
||||
}
|
||||
|
||||
"DynamoDBGroupRepository constructor" should {
|
||||
|
||||
"call setuptable when it is built" in {
|
||||
val setupTableCaptor = ArgumentCaptor.forClass(classOf[CreateTableRequest])
|
||||
|
||||
new DynamoDBGroupRepository(groupsStoreConfig, dynamoDBHelper)
|
||||
verify(dynamoDBHelper).setupTable(setupTableCaptor.capture())
|
||||
|
||||
val createTable = setupTableCaptor.getValue
|
||||
|
||||
createTable.getTableName shouldBe membershipTable
|
||||
(createTable.getAttributeDefinitions should contain).only(underTest.tableAttributes: _*)
|
||||
createTable.getKeySchema.get(0).getAttributeName shouldBe underTest.GROUP_ID
|
||||
createTable.getKeySchema.get(0).getKeyType shouldBe KeyType.HASH.toString
|
||||
createTable.getGlobalSecondaryIndexes.toArray() shouldBe underTest.secondaryIndexes.toArray
|
||||
createTable.getProvisionedThroughput.getReadCapacityUnits shouldBe 30L
|
||||
createTable.getProvisionedThroughput.getWriteCapacityUnits shouldBe 30L
|
||||
}
|
||||
|
||||
"fail when an exception is thrown setting up the table" in {
|
||||
|
||||
doThrow(new RuntimeException("fail")).when(dynamoDBHelper).setupTable(any[CreateTableRequest])
|
||||
|
||||
a[RuntimeException] should be thrownBy new DynamoDBGroupRepository(
|
||||
groupsStoreConfig,
|
||||
dynamoDBHelper)
|
||||
}
|
||||
}
|
||||
|
||||
"DynamoDBGroupRepository.toItem" should {
|
||||
"set all values correctly" in {
|
||||
@ -220,7 +188,7 @@ class DynamoDBGroupRepositorySpec
|
||||
"return the groups if the id is found" in {
|
||||
val firstResponse = mock[BatchGetItemResult]
|
||||
val firstPage = Map(
|
||||
underTest.GROUP_TABLE -> listOfDummyGroups
|
||||
groupsTable -> listOfDummyGroups
|
||||
.slice(0, 100)
|
||||
.map(underTest.toItem)
|
||||
.asJava).asJava
|
||||
@ -228,7 +196,7 @@ class DynamoDBGroupRepositorySpec
|
||||
|
||||
val secondResponse = mock[BatchGetItemResult]
|
||||
val secondPage = Map(
|
||||
underTest.GROUP_TABLE -> listOfDummyGroups
|
||||
groupsTable -> listOfDummyGroups
|
||||
.slice(100, 200)
|
||||
.map(underTest.toItem)
|
||||
.asJava).asJava
|
||||
@ -249,7 +217,7 @@ class DynamoDBGroupRepositorySpec
|
||||
"not return a group if it is deleted" in {
|
||||
val dynamoResponse = mock[BatchGetItemResult]
|
||||
val expected = underTest.toItem(deletedGroup)
|
||||
val firstPage = Map(underTest.GROUP_TABLE -> List(expected).asJava).asJava
|
||||
val firstPage = Map(groupsTable -> List(expected).asJava).asJava
|
||||
|
||||
doReturn(firstPage).when(dynamoResponse).getResponses
|
||||
doReturn(IO.pure(dynamoResponse))
|
||||
@ -263,7 +231,7 @@ class DynamoDBGroupRepositorySpec
|
||||
|
||||
"return None if no groups found" in {
|
||||
val firstResponse = mock[BatchGetItemResult]
|
||||
val firstPage = Map(underTest.GROUP_TABLE -> List().asJava).asJava
|
||||
val firstPage = Map(groupsTable -> List().asJava).asJava
|
||||
doReturn(firstPage).when(firstResponse).getResponses
|
||||
|
||||
doReturn(IO.pure(firstResponse))
|
||||
|
@ -18,6 +18,7 @@ package vinyldns.dynamodb.repository
|
||||
|
||||
import java.util
|
||||
|
||||
import cats.effect.IO
|
||||
import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClient
|
||||
import com.amazonaws.services.dynamodbv2.model._
|
||||
import com.codahale.metrics.Meter
|
||||
@ -152,11 +153,11 @@ class DynamoDBHelperSpec
|
||||
"return normally when no errors occur" in {
|
||||
val req = new CreateTableRequest().withTableName(testTableName)
|
||||
|
||||
doReturn(true).when(mockDynamoUtils).createTableIfNotExists(mockDynamo, req)
|
||||
doNothing().when(mockDynamoUtils).waitUntilActive(mockDynamo, testTableName)
|
||||
doReturn(IO.pure(true)).when(mockDynamoUtils).createTableIfNotExists(mockDynamo, req)
|
||||
doReturn(IO.unit).when(mockDynamoUtils).waitUntilActive(mockDynamo, testTableName)
|
||||
|
||||
val underTest = new TestDynamoDBHelper
|
||||
underTest.setupTable(req)
|
||||
underTest.setupTable(req).unsafeRunSync()
|
||||
|
||||
verify(mockDynamoUtils).createTableIfNotExists(mockDynamo, req)
|
||||
verify(mockDynamoUtils).waitUntilActive(mockDynamo, testTableName)
|
||||
|
@ -39,32 +39,15 @@ class DynamoDBMembershipRepositorySpec
|
||||
with BeforeAndAfterEach {
|
||||
|
||||
private val membershipStoreConfig = DynamoTestConfig.membershipStoreConfig
|
||||
private val membershipTable = membershipStoreConfig.getString("dynamo.tableName")
|
||||
private val membershipTable = membershipStoreConfig.tableName
|
||||
private val dynamoDBHelper = mock[DynamoDBHelper]
|
||||
class TestDynamoDBMembershipRepository
|
||||
extends DynamoDBMembershipRepository(membershipStoreConfig, dynamoDBHelper) {}
|
||||
extends DynamoDBMembershipRepository(membershipTable, dynamoDBHelper) {}
|
||||
|
||||
private val underTest = new TestDynamoDBMembershipRepository
|
||||
|
||||
override def beforeEach(): Unit = reset(dynamoDBHelper)
|
||||
|
||||
"DynamoDBMembershipRepository.apply" should {
|
||||
|
||||
"call setup table when it is built" in {
|
||||
val setupTableCaptor = ArgumentCaptor.forClass(classOf[CreateTableRequest])
|
||||
new TestDynamoDBMembershipRepository()
|
||||
verify(dynamoDBHelper).setupTable(setupTableCaptor.capture())
|
||||
|
||||
val req = setupTableCaptor.getValue
|
||||
req.getTableName shouldBe membershipTable
|
||||
}
|
||||
|
||||
"fail when an exception is thrown setting up the table" in {
|
||||
doThrow(new RuntimeException("fail")).when(dynamoDBHelper).setupTable(any[CreateTableRequest])
|
||||
a[RuntimeException] should be thrownBy new TestDynamoDBMembershipRepository()
|
||||
}
|
||||
}
|
||||
|
||||
"DynamoDBMembershipRepository.addMembers" should {
|
||||
"add the members in batches and return the members that were added to a group" in {
|
||||
val members = (for (i <- 1 to 60) yield s"member-${i}").toSet
|
||||
@ -90,9 +73,9 @@ class DynamoDBMembershipRepositorySpec
|
||||
// we should have 3 batches
|
||||
val batchWrites = batchCaptor.getAllValues
|
||||
|
||||
batchWrites.get(0).getRequestItems.get(store.membershipTable).size() shouldBe 25
|
||||
batchWrites.get(1).getRequestItems.get(store.membershipTable).size() shouldBe 25
|
||||
batchWrites.get(2).getRequestItems.get(store.membershipTable).size() shouldBe 10
|
||||
batchWrites.get(0).getRequestItems.get(membershipTable).size() shouldBe 25
|
||||
batchWrites.get(1).getRequestItems.get(membershipTable).size() shouldBe 25
|
||||
batchWrites.get(2).getRequestItems.get(membershipTable).size() shouldBe 10
|
||||
|
||||
response should contain theSameElementsAs members
|
||||
}
|
||||
@ -119,7 +102,7 @@ class DynamoDBMembershipRepositorySpec
|
||||
any[FiniteDuration])
|
||||
|
||||
val batchWrites = batchCaptor.getAllValues
|
||||
batchWrites.get(0).getRequestItems.get(store.membershipTable).size() shouldBe 20
|
||||
batchWrites.get(0).getRequestItems.get(membershipTable).size() shouldBe 20
|
||||
response should contain theSameElementsAs members
|
||||
}
|
||||
|
||||
@ -166,9 +149,9 @@ class DynamoDBMembershipRepositorySpec
|
||||
// we should have 3 batches
|
||||
val batchWrites = batchCaptor.getAllValues
|
||||
|
||||
batchWrites.get(0).getRequestItems.get(store.membershipTable).size() shouldBe 25
|
||||
batchWrites.get(1).getRequestItems.get(store.membershipTable).size() shouldBe 25
|
||||
batchWrites.get(2).getRequestItems.get(store.membershipTable).size() shouldBe 10
|
||||
batchWrites.get(0).getRequestItems.get(membershipTable).size() shouldBe 25
|
||||
batchWrites.get(1).getRequestItems.get(membershipTable).size() shouldBe 25
|
||||
batchWrites.get(2).getRequestItems.get(membershipTable).size() shouldBe 10
|
||||
|
||||
response should contain theSameElementsAs members
|
||||
}
|
||||
@ -195,7 +178,7 @@ class DynamoDBMembershipRepositorySpec
|
||||
any[FiniteDuration])
|
||||
|
||||
val batchWrites = batchCaptor.getAllValues
|
||||
batchWrites.get(0).getRequestItems.get(store.membershipTable).size() shouldBe 20
|
||||
batchWrites.get(0).getRequestItems.get(membershipTable).size() shouldBe 20
|
||||
response should contain theSameElementsAs members
|
||||
}
|
||||
|
||||
|
@ -40,40 +40,12 @@ class DynamoDBRecordChangeRepositorySpec
|
||||
|
||||
private val dynamoDBHelper = mock[DynamoDBHelper]
|
||||
private val recordSetConfig = DynamoTestConfig.recordChangeStoreConfig
|
||||
private val recordChangeTable = recordSetConfig.getString("dynamo.tableName")
|
||||
private val recordChangeTable = recordSetConfig.tableName
|
||||
|
||||
class TestRepo extends DynamoDBRecordChangeRepository(recordSetConfig, dynamoDBHelper)
|
||||
class TestRepo extends DynamoDBRecordChangeRepository(recordChangeTable, dynamoDBHelper)
|
||||
|
||||
override def beforeEach(): Unit = {
|
||||
override def beforeEach(): Unit =
|
||||
reset(dynamoDBHelper)
|
||||
doNothing().when(dynamoDBHelper).setupTable(any[CreateTableRequest])
|
||||
}
|
||||
|
||||
"DynamoDBRecordChangeRepository.apply" should {
|
||||
"call setup table when it is built" in {
|
||||
val setupTableCaptor = ArgumentCaptor.forClass(classOf[CreateTableRequest])
|
||||
|
||||
val store = new TestRepo
|
||||
verify(dynamoDBHelper).setupTable(setupTableCaptor.capture())
|
||||
|
||||
val createTable = setupTableCaptor.getValue
|
||||
|
||||
createTable.getTableName shouldBe recordChangeTable
|
||||
(createTable.getAttributeDefinitions should contain).only(store.tableAttributes: _*)
|
||||
createTable.getKeySchema.get(0).getAttributeName shouldBe store.RECORD_SET_CHANGE_ID
|
||||
createTable.getKeySchema.get(0).getKeyType shouldBe KeyType.HASH.toString
|
||||
createTable.getGlobalSecondaryIndexes.toArray() shouldBe store.secondaryIndexes.toArray
|
||||
createTable.getProvisionedThroughput.getReadCapacityUnits shouldBe 30L
|
||||
createTable.getProvisionedThroughput.getWriteCapacityUnits shouldBe 30L
|
||||
}
|
||||
|
||||
"fail when an exception is thrown setting up the table" in {
|
||||
|
||||
doThrow(new RuntimeException("fail")).when(dynamoDBHelper).setupTable(any[CreateTableRequest])
|
||||
|
||||
a[RuntimeException] should be thrownBy new TestRepo
|
||||
}
|
||||
}
|
||||
|
||||
"DynamoDBRecordChangeRepository.save" should {
|
||||
"group change sets into batch writes with 25 in each" in {
|
||||
@ -180,7 +152,7 @@ class DynamoDBRecordChangeRepositorySpec
|
||||
.thenReturn(new java.util.ArrayList[java.util.Map[String, AttributeValue]]())
|
||||
when(dynamoDBHelper.query(any[QueryRequest])).thenReturn(IO.pure(dynamoResponse))
|
||||
|
||||
val store = new DynamoDBRecordChangeRepository(recordSetConfig, dynamoDBHelper)
|
||||
val store = new DynamoDBRecordChangeRepository(recordChangeTable, dynamoDBHelper)
|
||||
val response = store.getRecordSetChange(zoneActive.id, pendingCreateAAAA.id).unsafeRunSync()
|
||||
|
||||
verify(dynamoDBHelper).query(any[QueryRequest])
|
||||
|
@ -39,43 +39,14 @@ class DynamoDBRecordSetRepositorySpec
|
||||
with ScalaFutures
|
||||
with BeforeAndAfterEach {
|
||||
|
||||
import DynamoDBRecordSetRepository._
|
||||
|
||||
private val dynamoDBHelper = mock[DynamoDBHelper]
|
||||
private val recordChangeConfig = DynamoTestConfig.recordChangeStoreConfig
|
||||
|
||||
class TestDynamoRecordSetRepo
|
||||
extends DynamoDBRecordSetRepository(recordChangeConfig, dynamoDBHelper)
|
||||
extends DynamoDBRecordSetRepository(recordChangeConfig.tableName, dynamoDBHelper)
|
||||
|
||||
override def beforeEach(): Unit = {
|
||||
override def beforeEach(): Unit =
|
||||
reset(dynamoDBHelper)
|
||||
doNothing().when(dynamoDBHelper).setupTable(any[CreateTableRequest])
|
||||
}
|
||||
|
||||
"DynamoDBRecordRepository.apply" should {
|
||||
"call setup table" in {
|
||||
val setupTableCaptor = ArgumentCaptor.forClass(classOf[CreateTableRequest])
|
||||
|
||||
val store = new TestDynamoRecordSetRepo
|
||||
verify(dynamoDBHelper).setupTable(setupTableCaptor.capture())
|
||||
|
||||
val createTable = setupTableCaptor.getValue
|
||||
|
||||
(createTable.getAttributeDefinitions should contain).only(store.tableAttributes: _*)
|
||||
createTable.getKeySchema.get(0).getAttributeName shouldBe RECORD_SET_ID
|
||||
createTable.getKeySchema.get(0).getKeyType shouldBe KeyType.HASH.toString
|
||||
createTable.getGlobalSecondaryIndexes.toArray() shouldBe store.secondaryIndexes.toArray
|
||||
createTable.getProvisionedThroughput.getReadCapacityUnits shouldBe 30L
|
||||
createTable.getProvisionedThroughput.getWriteCapacityUnits shouldBe 30L
|
||||
}
|
||||
|
||||
"fail when an exception is thrown setting up the table" in {
|
||||
|
||||
doThrow(new RuntimeException("fail")).when(dynamoDBHelper).setupTable(any[CreateTableRequest])
|
||||
|
||||
a[RuntimeException] should be thrownBy new TestDynamoRecordSetRepo()
|
||||
}
|
||||
}
|
||||
|
||||
"DynamoDBRecordSetRepository.applyChangeSet" should {
|
||||
"return the ChangeSet" in {
|
||||
@ -184,7 +155,7 @@ class DynamoDBRecordSetRepositorySpec
|
||||
when(dynamoDBHelper.getItem(any[GetItemRequest]))
|
||||
.thenReturn(IO.pure(dynamoResponse))
|
||||
|
||||
val store = new DynamoDBRecordSetRepository(recordChangeConfig, dynamoDBHelper)
|
||||
val store = new DynamoDBRecordSetRepository(recordChangeConfig.tableName, dynamoDBHelper)
|
||||
val response = store.getRecordSet(rsOk.zoneId, rsOk.id).unsafeRunSync()
|
||||
|
||||
verify(dynamoDBHelper).getItem(any[GetItemRequest])
|
||||
@ -196,7 +167,7 @@ class DynamoDBRecordSetRepositorySpec
|
||||
"DynamoDBRecordSetRepository.listRecordSets(zoneId)" should {
|
||||
"returns empty if no record set exist" in {
|
||||
|
||||
val store = new DynamoDBRecordSetRepository(recordChangeConfig, dynamoDBHelper)
|
||||
val store = new DynamoDBRecordSetRepository(recordChangeConfig.tableName, dynamoDBHelper)
|
||||
|
||||
val dynamoResponse = mock[QueryResult]
|
||||
val expectedItems = new util.ArrayList[util.HashMap[String, AttributeValue]]()
|
||||
@ -324,7 +295,7 @@ class DynamoDBRecordSetRepositorySpec
|
||||
}
|
||||
"throw exception when query returns an unexpected response" in {
|
||||
when(dynamoDBHelper.query(any[QueryRequest])).thenThrow(new ResourceNotFoundException("fail"))
|
||||
val store = new DynamoDBRecordSetRepository(recordChangeConfig, dynamoDBHelper)
|
||||
val store = new DynamoDBRecordSetRepository(recordChangeConfig.tableName, dynamoDBHelper)
|
||||
|
||||
a[ResourceNotFoundException] should be thrownBy store.getRecordSets(
|
||||
rsOk.zoneId,
|
||||
|
@ -42,80 +42,49 @@ class DynamoDBUserRepositorySpec
|
||||
private val mockPutItemResult = mock[PutItemResult] // User repo is initialized with dummy users
|
||||
doReturn(IO.pure(mockPutItemResult)).when(dynamoDBHelper).putItem(any[PutItemRequest])
|
||||
private val usersStoreConfig = DynamoTestConfig.usersStoreConfig
|
||||
private val userTable = usersStoreConfig.getString("dynamo.tableName")
|
||||
private val userTable = usersStoreConfig.tableName
|
||||
|
||||
class TestDynamoDBUserRepository extends DynamoDBUserRepository(usersStoreConfig, dynamoDBHelper)
|
||||
class TestDynamoDBUserRepository extends DynamoDBUserRepository(userTable, dynamoDBHelper)
|
||||
|
||||
private val underTest = new DynamoDBUserRepository(usersStoreConfig, dynamoDBHelper)
|
||||
private val underTest = new DynamoDBUserRepository(userTable, dynamoDBHelper)
|
||||
|
||||
override def beforeEach(): Unit = {
|
||||
override def beforeEach(): Unit =
|
||||
reset(dynamoDBHelper)
|
||||
doNothing().when(dynamoDBHelper).setupTable(any[CreateTableRequest])
|
||||
}
|
||||
|
||||
"DynamoDBUserRepository constructor" should {
|
||||
"call setuptable when it is built" in {
|
||||
val mockPutItemResult = mock[PutItemResult] // User repo is initialized with dummy users
|
||||
doReturn(IO.pure(mockPutItemResult))
|
||||
.when(dynamoDBHelper)
|
||||
.putItem(any[PutItemRequest])
|
||||
|
||||
val setupTableCaptor = ArgumentCaptor.forClass(classOf[CreateTableRequest])
|
||||
|
||||
new TestDynamoDBUserRepository
|
||||
verify(dynamoDBHelper).setupTable(setupTableCaptor.capture())
|
||||
|
||||
val createTable = setupTableCaptor.getValue
|
||||
|
||||
createTable.getTableName shouldBe userTable
|
||||
(createTable.getAttributeDefinitions should contain).only(underTest.tableAttributes: _*)
|
||||
createTable.getKeySchema.get(0).getAttributeName shouldBe underTest.USER_ID
|
||||
createTable.getKeySchema.get(0).getKeyType shouldBe KeyType.HASH.toString
|
||||
createTable.getGlobalSecondaryIndexes.toArray() shouldBe underTest.secondaryIndexes.toArray
|
||||
createTable.getProvisionedThroughput.getReadCapacityUnits shouldBe 30L
|
||||
createTable.getProvisionedThroughput.getWriteCapacityUnits shouldBe 30L
|
||||
}
|
||||
|
||||
"fail when an exception is thrown setting up the table" in {
|
||||
|
||||
doThrow(new RuntimeException("fail")).when(dynamoDBHelper).setupTable(any[CreateTableRequest])
|
||||
|
||||
a[RuntimeException] should be thrownBy new TestDynamoDBUserRepository
|
||||
}
|
||||
}
|
||||
import DynamoDBUserRepository._
|
||||
|
||||
"DynamoDBUserRepository.toItem" should {
|
||||
"set all values correctly" in {
|
||||
val items = underTest.toItem(okUser)
|
||||
items.get(underTest.USER_ID).getS shouldBe okUser.id
|
||||
items.get(underTest.USER_NAME).getS shouldBe okUser.userName
|
||||
items.get(underTest.ACCESS_KEY).getS shouldBe okUser.accessKey
|
||||
items.get(underTest.SECRET_KEY).getS shouldBe okUser.secretKey
|
||||
items.get(underTest.FIRST_NAME).getS shouldBe okUser.firstName.get
|
||||
items.get(underTest.LAST_NAME).getS shouldBe okUser.lastName.get
|
||||
items.get(underTest.EMAIL).getS shouldBe okUser.email.get
|
||||
items.get(underTest.CREATED).getN shouldBe okUser.created.getMillis.toString
|
||||
items.get(USER_ID).getS shouldBe okUser.id
|
||||
items.get(USER_NAME).getS shouldBe okUser.userName
|
||||
items.get(ACCESS_KEY).getS shouldBe okUser.accessKey
|
||||
items.get(SECRET_KEY).getS shouldBe okUser.secretKey
|
||||
items.get(FIRST_NAME).getS shouldBe okUser.firstName.get
|
||||
items.get(LAST_NAME).getS shouldBe okUser.lastName.get
|
||||
items.get(EMAIL).getS shouldBe okUser.email.get
|
||||
items.get(CREATED).getN shouldBe okUser.created.getMillis.toString
|
||||
}
|
||||
"set the first name to null if it is not present" in {
|
||||
val emptyFirstName = okUser.copy(firstName = None)
|
||||
|
||||
val items = underTest.toItem(emptyFirstName)
|
||||
Option(items.get(underTest.FIRST_NAME).getS) shouldBe None
|
||||
items.get(underTest.FIRST_NAME).getNULL shouldBe true
|
||||
Option(items.get(DynamoDBUserRepository.FIRST_NAME).getS) shouldBe None
|
||||
items.get(DynamoDBUserRepository.FIRST_NAME).getNULL shouldBe true
|
||||
}
|
||||
"set the last name to null if it is not present" in {
|
||||
val emptyLastName = okUser.copy(lastName = None)
|
||||
|
||||
val items = underTest.toItem(emptyLastName)
|
||||
Option(items.get(underTest.LAST_NAME).getS) shouldBe None
|
||||
items.get(underTest.LAST_NAME).getNULL shouldBe true
|
||||
Option(items.get(LAST_NAME).getS) shouldBe None
|
||||
items.get(LAST_NAME).getNULL shouldBe true
|
||||
}
|
||||
"set the email to null if it is not present" in {
|
||||
val emptyEmail = okUser.copy(email = None)
|
||||
|
||||
val items = underTest.toItem(emptyEmail)
|
||||
Option(items.get(underTest.EMAIL).getS) shouldBe None
|
||||
items.get(underTest.EMAIL).getNULL shouldBe true
|
||||
Option(items.get(EMAIL).getS) shouldBe None
|
||||
items.get(EMAIL).getNULL shouldBe true
|
||||
}
|
||||
}
|
||||
|
||||
@ -149,11 +118,11 @@ class DynamoDBUserRepositorySpec
|
||||
}
|
||||
"sets empty values correctly if key is not present in item" in {
|
||||
val item = new java.util.HashMap[String, AttributeValue]()
|
||||
item.put(underTest.USER_ID, new AttributeValue("ok"))
|
||||
item.put(underTest.USER_NAME, new AttributeValue("ok"))
|
||||
item.put(underTest.CREATED, new AttributeValue().withN("0"))
|
||||
item.put(underTest.ACCESS_KEY, new AttributeValue("accessKey"))
|
||||
item.put(underTest.SECRET_KEY, new AttributeValue("secretkey"))
|
||||
item.put(USER_ID, new AttributeValue("ok"))
|
||||
item.put(USER_NAME, new AttributeValue("ok"))
|
||||
item.put(CREATED, new AttributeValue().withN("0"))
|
||||
item.put(ACCESS_KEY, new AttributeValue("accessKey"))
|
||||
item.put(SECRET_KEY, new AttributeValue("secretkey"))
|
||||
val user = underTest.fromItem(item)
|
||||
|
||||
user.firstName shouldBe None
|
||||
@ -169,11 +138,11 @@ class DynamoDBUserRepositorySpec
|
||||
}
|
||||
"sets the isSuper flag correctly if the key is not present in the item" in {
|
||||
val item = new java.util.HashMap[String, AttributeValue]()
|
||||
item.put(underTest.USER_ID, new AttributeValue("ok"))
|
||||
item.put(underTest.USER_NAME, new AttributeValue("ok"))
|
||||
item.put(underTest.CREATED, new AttributeValue().withN("0"))
|
||||
item.put(underTest.ACCESS_KEY, new AttributeValue("accesskey"))
|
||||
item.put(underTest.SECRET_KEY, new AttributeValue("secretkey"))
|
||||
item.put(USER_ID, new AttributeValue("ok"))
|
||||
item.put(USER_NAME, new AttributeValue("ok"))
|
||||
item.put(CREATED, new AttributeValue().withN("0"))
|
||||
item.put(ACCESS_KEY, new AttributeValue("accesskey"))
|
||||
item.put(SECRET_KEY, new AttributeValue("secretkey"))
|
||||
val user = underTest.fromItem(item)
|
||||
|
||||
user.isSuper shouldBe false
|
||||
@ -218,13 +187,13 @@ class DynamoDBUserRepositorySpec
|
||||
"DynamoDBUserRepository.getUsers" should {
|
||||
"return the users if the id is found" in {
|
||||
val firstResponse = mock[BatchGetItemResult]
|
||||
val firstPage = Map(
|
||||
underTest.USER_TABLE -> listOfDummyUsers.slice(0, 100).map(underTest.toItem).asJava).asJava
|
||||
val firstPage =
|
||||
Map(userTable -> listOfDummyUsers.slice(0, 100).map(underTest.toItem).asJava).asJava
|
||||
doReturn(firstPage).when(firstResponse).getResponses
|
||||
|
||||
val secondResponse = mock[BatchGetItemResult]
|
||||
val secondPage = Map(
|
||||
underTest.USER_TABLE -> listOfDummyUsers
|
||||
userTable -> listOfDummyUsers
|
||||
.slice(100, 200)
|
||||
.map(underTest.toItem)
|
||||
.asJava).asJava
|
||||
@ -245,7 +214,7 @@ class DynamoDBUserRepositorySpec
|
||||
}
|
||||
"return None if no users found" in {
|
||||
val firstResponse = mock[BatchGetItemResult]
|
||||
val firstPage = Map(underTest.USER_TABLE -> List().asJava).asJava
|
||||
val firstPage = Map(userTable -> List().asJava).asJava
|
||||
doReturn(firstPage).when(firstResponse).getResponses
|
||||
|
||||
doReturn(IO.pure(firstResponse))
|
||||
@ -281,21 +250,21 @@ class DynamoDBUserRepositorySpec
|
||||
|
||||
for { userId <- userIds } {
|
||||
val key = new util.HashMap[String, AttributeValue]()
|
||||
key.put(underTest.USER_ID, new AttributeValue(userId))
|
||||
key.put(USER_ID, new AttributeValue(userId))
|
||||
allKeys.add(key)
|
||||
}
|
||||
|
||||
val keysAndAttributes = new KeysAndAttributes().withKeys(allKeys)
|
||||
|
||||
val request = new util.HashMap[String, KeysAndAttributes]()
|
||||
request.put(underTest.USER_TABLE, keysAndAttributes)
|
||||
request.put(userTable, keysAndAttributes)
|
||||
|
||||
new BatchGetItemRequest().withRequestItems(request)
|
||||
}
|
||||
|
||||
val firstResponse = mock[BatchGetItemResult]
|
||||
val firstPage = Map(
|
||||
underTest.USER_TABLE -> listOfDummyUsers
|
||||
userTable -> listOfDummyUsers
|
||||
.slice(151, 200)
|
||||
.map(underTest.toItem)
|
||||
.asJava).asJava
|
||||
@ -322,8 +291,8 @@ class DynamoDBUserRepositorySpec
|
||||
}
|
||||
"truncates the response to only return limit items" in {
|
||||
val firstResponse = mock[BatchGetItemResult]
|
||||
val firstPage = Map(
|
||||
underTest.USER_TABLE -> listOfDummyUsers.slice(0, 50).map(underTest.toItem).asJava).asJava
|
||||
val firstPage =
|
||||
Map(userTable -> listOfDummyUsers.slice(0, 50).map(underTest.toItem).asJava).asJava
|
||||
doReturn(firstPage).when(firstResponse).getResponses
|
||||
|
||||
doReturn(IO.pure(firstResponse))
|
||||
|
@ -17,7 +17,6 @@
|
||||
package vinyldns.dynamodb.repository
|
||||
|
||||
import com.amazonaws.services.dynamodbv2.model._
|
||||
import org.mockito.ArgumentCaptor
|
||||
import org.mockito.Matchers._
|
||||
import org.mockito.Mockito._
|
||||
import org.scalatest.concurrent.ScalaFutures
|
||||
@ -37,10 +36,10 @@ class DynamoDBZoneChangeRepositorySpec
|
||||
|
||||
private val dynamoDBHelper = mock[TestDynamoDBHelper]
|
||||
private val zoneChangeStoreConfig = DynamoTestConfig.zoneChangeStoreConfig
|
||||
private val zoneChangeTable = zoneChangeStoreConfig.getString("dynamo.tableName")
|
||||
private val zoneChangeTable = zoneChangeStoreConfig.tableName
|
||||
|
||||
class TestDynamoDBZoneChangeRepository
|
||||
extends DynamoDBZoneChangeRepository(zoneChangeStoreConfig, dynamoDBHelper)
|
||||
extends DynamoDBZoneChangeRepository(zoneChangeTable, dynamoDBHelper)
|
||||
|
||||
private val underTest = new TestDynamoDBZoneChangeRepository
|
||||
|
||||
@ -53,32 +52,6 @@ class DynamoDBZoneChangeRepositorySpec
|
||||
val zoneChangeFailed: ZoneChange =
|
||||
ZoneChange(okZone, "ok", ZoneChangeType.Update, ZoneChangeStatus.Failed)
|
||||
|
||||
"DynamoDBZoneChangeRepository.apply" should {
|
||||
"call setuptable when it is built" in {
|
||||
val setupTableCaptor = ArgumentCaptor.forClass(classOf[CreateTableRequest])
|
||||
|
||||
val store = new TestDynamoDBZoneChangeRepository
|
||||
verify(dynamoDBHelper).setupTable(setupTableCaptor.capture())
|
||||
|
||||
val createTable = setupTableCaptor.getValue
|
||||
createTable.getTableName shouldBe zoneChangeTable
|
||||
|
||||
(createTable.getAttributeDefinitions should contain).only(store.tableAttributes: _*)
|
||||
createTable.getKeySchema.get(0).getAttributeName shouldBe store.ZONE_ID
|
||||
createTable.getKeySchema.get(0).getKeyType shouldBe KeyType.HASH.toString
|
||||
createTable.getKeySchema.get(1).getAttributeName shouldBe store.CHANGE_ID
|
||||
createTable.getKeySchema.get(1).getKeyType shouldBe KeyType.RANGE.toString
|
||||
createTable.getGlobalSecondaryIndexes.toArray() shouldBe store.secondaryIndexes.toArray
|
||||
createTable.getProvisionedThroughput.getReadCapacityUnits shouldBe 30L
|
||||
createTable.getProvisionedThroughput.getWriteCapacityUnits shouldBe 30L
|
||||
}
|
||||
|
||||
"fail when an exception is thrown setting up the table" in {
|
||||
doThrow(new RuntimeException("fail")).when(dynamoDBHelper).setupTable(any[CreateTableRequest])
|
||||
a[RuntimeException] should be thrownBy new TestDynamoDBZoneChangeRepository
|
||||
}
|
||||
}
|
||||
|
||||
"DynamoDBZoneChangeRepository.save" should {
|
||||
"call DynamoDBClient.putItem when creating a zone change" in {
|
||||
val putItemResult = mock[PutItemResult]
|
||||
|
@ -60,7 +60,9 @@ object Dependencies {
|
||||
|
||||
lazy val dynamoDBDependencies = Seq(
|
||||
"com.amazonaws" % "aws-java-sdk-core" % awsV withSources(),
|
||||
"com.amazonaws" % "aws-java-sdk-dynamodb" % awsV withSources()
|
||||
"com.amazonaws" % "aws-java-sdk-dynamodb" % awsV withSources(),
|
||||
"com.github.pureconfig" %% "pureconfig" % pureConfigV,
|
||||
"com.github.pureconfig" %% "pureconfig-cats-effect" % pureConfigV
|
||||
)
|
||||
|
||||
lazy val commonTestDependencies = Seq(
|
||||
|
Loading…
x
Reference in New Issue
Block a user