mirror of
https://github.com/VinylDNS/vinyldns
synced 2025-08-22 02:02:14 +00:00
MySql dynamic loader (#151)
* config file updates for mysql loading * dynamic loading for mysql * IT test changes for dynamic load * rebase fixes * move settings to own file * conf cleanup * missing headers * cleanup, some testing * pureconfig cats load * error message fix
This commit is contained in:
parent
42cb7b3e91
commit
ed6144b805
@ -43,23 +43,35 @@ vinyldns {
|
||||
type = "vinyldns.core.crypto.NoOpCrypto"
|
||||
}
|
||||
|
||||
# default settings point to the setup from docker compose
|
||||
db {
|
||||
name = "vinyldns"
|
||||
default {
|
||||
data-stores = ["mysql"]
|
||||
|
||||
mysql {
|
||||
settings {
|
||||
# JDBC Settings, these are all values in scalikejdbc-config, not our own
|
||||
# these must be overridden to use MYSQL for production use
|
||||
# assumes a docker or mysql instance running locally
|
||||
name = "vinyldns"
|
||||
driver = "org.mariadb.jdbc.Driver"
|
||||
migrationUrl = "jdbc:mariadb://vinyldns-mysql:3306/?user=root&password=pass"
|
||||
migrationUrl = ${?JDBC_MIGRATION_URL}
|
||||
migration-url = "jdbc:mariadb://vinyldns-mysql:3306/?user=root&password=pass"
|
||||
migration-url = ${?JDBC_MIGRATION_URL}
|
||||
url = "jdbc:mariadb://vinyldns-mysql:3306/vinyldns?user=root&password=pass"
|
||||
url = ${?JDBC_URL}
|
||||
user = "root"
|
||||
user = ${?JDBC_USER}
|
||||
password = "pass"
|
||||
password = ${?JDBC_PASSWORD}
|
||||
poolInitialSize = 10
|
||||
poolMaxSize = 20
|
||||
connectionTimeoutMillis = 1000
|
||||
maxLifeTime = 600000
|
||||
pool-max-size = 20
|
||||
connection-timeout-millis = 1000
|
||||
max-life-time = 600000
|
||||
}
|
||||
# Repositories that use this data store are listed here
|
||||
repositories {
|
||||
zone {
|
||||
# no additional settings for now
|
||||
},
|
||||
batch-change {
|
||||
# no additional settings for now
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -19,6 +19,15 @@ vinyldns {
|
||||
}
|
||||
}
|
||||
|
||||
mysql.repositories {
|
||||
zone {
|
||||
# no additional settings for now
|
||||
},
|
||||
batch-change {
|
||||
# no additional settings for now
|
||||
}
|
||||
}
|
||||
|
||||
recordSet {
|
||||
# use the dummy store, this should only be used local
|
||||
dummy = true
|
||||
|
@ -30,7 +30,7 @@ import vinyldns.api.domain.record.RecordType._
|
||||
import vinyldns.api.domain.zone.{RecordSetAlreadyExists, Zone, ZoneRepository, ZoneStatus}
|
||||
import vinyldns.api.engine.sqs.TestSqsService
|
||||
import vinyldns.api.repository.dynamodb.{DynamoDBIntegrationSpec, DynamoDBRecordSetRepository}
|
||||
import vinyldns.api.repository.mysql.VinylDNSJDBC
|
||||
import vinyldns.api.repository.mysql.TestMySqlInstance
|
||||
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.ExecutionContext.Implicits.global
|
||||
@ -155,7 +155,7 @@ class RecordSetServiceIntegrationSpec
|
||||
|
||||
def setup(): Unit = {
|
||||
recordSetRepo = new DynamoDBRecordSetRepository(recordSetStoreConfig, dynamoDBHelper)
|
||||
zoneRepo = VinylDNSJDBC.instance.zoneRepository
|
||||
zoneRepo = TestMySqlInstance.zoneRepository
|
||||
|
||||
List(zone, zoneTestNameConflicts, zoneTestAddRecords).map(z => waitForSuccess(zoneRepo.save(z)))
|
||||
|
||||
|
@ -28,7 +28,7 @@ import vinyldns.api.domain.membership.{Group, GroupRepository, User, UserReposit
|
||||
import vinyldns.api.domain.record._
|
||||
import vinyldns.api.engine.sqs.TestSqsService
|
||||
import vinyldns.api.repository.dynamodb.{DynamoDBIntegrationSpec, DynamoDBRecordSetRepository}
|
||||
import vinyldns.api.repository.mysql.VinylDNSJDBC
|
||||
import vinyldns.api.repository.mysql.TestMySqlInstance
|
||||
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.ExecutionContext.Implicits.global
|
||||
@ -104,7 +104,7 @@ class ZoneServiceIntegrationSpec extends DynamoDBIntegrationSpec with MockitoSug
|
||||
|
||||
def setup(): Unit = {
|
||||
recordSetRepo = new DynamoDBRecordSetRepository(recordSetStoreConfig, dynamoDBHelper)
|
||||
zoneRepo = VinylDNSJDBC.instance.zoneRepository
|
||||
zoneRepo = TestMySqlInstance.zoneRepository
|
||||
|
||||
waitForSuccess(zoneRepo.save(zone))
|
||||
// Seeding records in DB
|
||||
|
@ -34,7 +34,7 @@ import vinyldns.api.repository.dynamodb.{
|
||||
DynamoDBRecordSetRepository,
|
||||
DynamoDBZoneChangeRepository
|
||||
}
|
||||
import vinyldns.api.repository.mysql.VinylDNSJDBC
|
||||
import vinyldns.api.repository.mysql.TestMySqlInstance
|
||||
|
||||
import scala.concurrent.duration._
|
||||
import scala.concurrent.{Await, ExecutionContext}
|
||||
@ -148,8 +148,8 @@ class ZoneCommandHandlerIntegrationSpec extends DynamoDBIntegrationSpec with Eve
|
||||
recordChangeRepo = new DynamoDBRecordChangeRepository(recordChangeStoreConfig, dynamoDBHelper)
|
||||
recordSetRepo = new DynamoDBRecordSetRepository(recordSetStoreConfig, dynamoDBHelper)
|
||||
zoneChangeRepo = new DynamoDBZoneChangeRepository(zoneChangeStoreConfig, dynamoDBHelper)
|
||||
zoneRepo = VinylDNSJDBC.instance.zoneRepository
|
||||
batchChangeRepo = VinylDNSJDBC.instance.batchChangeRepository
|
||||
zoneRepo = TestMySqlInstance.zoneRepository
|
||||
batchChangeRepo = TestMySqlInstance.batchChangeRepository
|
||||
sqsConn = SqsConnection()
|
||||
|
||||
//seed items database
|
||||
|
@ -43,7 +43,7 @@ class JdbcBatchChangeRepositoryIntegrationSpec
|
||||
with Inspectors
|
||||
with OptionValues {
|
||||
|
||||
private var repo: JdbcBatchChangeRepository = _
|
||||
private var repo: BatchChangeRepository = _
|
||||
private val timeout = PatienceConfiguration.Timeout(Span(10, Seconds))
|
||||
|
||||
import SingleChangeStatus._
|
||||
@ -148,7 +148,7 @@ class JdbcBatchChangeRepositoryIntegrationSpec
|
||||
import TestData._
|
||||
|
||||
override protected def beforeAll(): Unit =
|
||||
repo = VinylDNSJDBC.instance.batchChangeRepository
|
||||
repo = TestMySqlInstance.batchChangeRepository
|
||||
|
||||
override protected def beforeEach(): Unit =
|
||||
DB.localTx { s =>
|
||||
|
@ -41,11 +41,11 @@ class JdbcZoneRepositoryIntegrationSpec
|
||||
with ScalaFutures
|
||||
with Inspectors {
|
||||
|
||||
private var repo: JdbcZoneRepository = _
|
||||
private var repo: ZoneRepository = _
|
||||
private val timeout = PatienceConfiguration.Timeout(Span(10, Seconds))
|
||||
|
||||
override protected def beforeAll(): Unit =
|
||||
repo = VinylDNSJDBC.instance.zoneRepository
|
||||
repo = TestMySqlInstance.zoneRepository
|
||||
|
||||
override protected def beforeEach(): Unit =
|
||||
DB.localTx { s =>
|
||||
|
@ -0,0 +1,32 @@
|
||||
/*
|
||||
* Copyright 2018 Comcast Cable Communications Management, LLC
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package vinyldns.api.repository.mysql
|
||||
|
||||
import vinyldns.api.VinylDNSConfig
|
||||
import vinyldns.api.domain.batch.BatchChangeRepository
|
||||
import vinyldns.api.domain.zone.ZoneRepository
|
||||
import vinyldns.api.repository.DataStore
|
||||
import vinyldns.api.repository.RepositoryName._
|
||||
|
||||
object TestMySqlInstance {
|
||||
lazy val instance: DataStore =
|
||||
new MySqlDataStoreProvider().load(VinylDNSConfig.mySqlConfig).unsafeRunSync()
|
||||
|
||||
lazy val zoneRepository: ZoneRepository = instance.get[ZoneRepository](zone).get
|
||||
lazy val batchChangeRepository: BatchChangeRepository =
|
||||
instance.get[BatchChangeRepository](batchChange).get
|
||||
}
|
@ -37,6 +37,15 @@ vinyldns {
|
||||
queue-url = "http://localhost:9324/queue/vinyldns-zones" // this is in the docker/elasticmq/custom.conf file
|
||||
}
|
||||
|
||||
mysql.repositories {
|
||||
zone {
|
||||
# no additional settings for now
|
||||
},
|
||||
batch-change {
|
||||
# no additional settings for now
|
||||
}
|
||||
}
|
||||
|
||||
sync-delay = 10000 # 10 second delay for resyncing zone
|
||||
|
||||
batch-change-limit = 20 # Max change limit per batch request
|
||||
|
@ -33,21 +33,29 @@ vinyldns {
|
||||
port = 9000
|
||||
}
|
||||
|
||||
# JDBC Settings, these are all values in scalikejdbc-config, not our own
|
||||
# these must be overridden to use MYSQL for production use
|
||||
# assumes a docker or mysql instance running locally
|
||||
db {
|
||||
name = "vinyldns"
|
||||
default {
|
||||
data-stores = ["mysql"]
|
||||
|
||||
mysql {
|
||||
class-name = "vinyldns.api.repository.mysql.MySqlDataStoreProvider"
|
||||
|
||||
settings {
|
||||
# JDBC Settings, these are all values in scalikejdbc-config, not our own
|
||||
# these must be overridden to use MYSQL for production use
|
||||
# assumes a docker or mysql instance running locally
|
||||
name = "vinyldns"
|
||||
driver = "org.mariadb.jdbc.Driver"
|
||||
migrationUrl = "jdbc:mariadb://localhost:19002/?user=root&password=pass"
|
||||
migration-url = "jdbc:mariadb://localhost:19002/?user=root&password=pass"
|
||||
url = "jdbc:mariadb://localhost:19002/vinyldns?user=root&password=pass"
|
||||
user = "root"
|
||||
password = "pass"
|
||||
poolInitialSize = 10
|
||||
poolMaxSize = 20
|
||||
connectionTimeoutMillis = 1000
|
||||
maxLifeTime = 600000
|
||||
pool-initial-size = 10
|
||||
pool-max-size = 20
|
||||
connection-timeout-millis = 1000
|
||||
max-life-time = 600000
|
||||
}
|
||||
|
||||
repositories {
|
||||
# override with any repos that are running in mysql
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -36,7 +36,8 @@ import vinyldns.api.domain.record.{RecordChangeRepository, RecordSetRepository,
|
||||
import vinyldns.api.domain.zone._
|
||||
import vinyldns.api.engine.ProductionZoneCommandHandler
|
||||
import vinyldns.api.engine.sqs.{SqsCommandBus, SqsConnection}
|
||||
import vinyldns.api.repository.mysql.VinylDNSJDBC
|
||||
import vinyldns.api.repository.{DataStoreStartupError, RepositoryName}
|
||||
import vinyldns.api.repository.mysql.MySqlDataStoreProvider
|
||||
import vinyldns.api.route.{HealthService, VinylDNSService}
|
||||
import vinyldns.core.crypto.Crypto
|
||||
|
||||
@ -64,16 +65,23 @@ object Boot extends App {
|
||||
for {
|
||||
banner <- vinyldnsBanner()
|
||||
_ <- Crypto.loadCrypto(VinylDNSConfig.cryptoConfig) // load crypto
|
||||
_ <- IO(VinylDNSJDBC.instance) // initializes our JDBC repositories
|
||||
// TODO datastore loading will not be hardcoded by type here
|
||||
mySqlDataStore <- new MySqlDataStoreProvider().load(VinylDNSConfig.mySqlConfig)
|
||||
zoneRepo <- IO.fromEither(
|
||||
mySqlDataStore
|
||||
.get[ZoneRepository](RepositoryName.zone)
|
||||
.toRight[Throwable](DataStoreStartupError("Missing zone repository")))
|
||||
batchChangeRepo <- IO.fromEither(
|
||||
mySqlDataStore
|
||||
.get[BatchChangeRepository](RepositoryName.batchChange)
|
||||
.toRight[Throwable](DataStoreStartupError("Missing batch change repository")))
|
||||
userRepo <- IO(UserRepository())
|
||||
groupRepo <- IO(GroupRepository())
|
||||
membershipRepo <- IO(MembershipRepository())
|
||||
zoneRepo <- IO(ZoneRepository())
|
||||
groupChangeRepo <- IO(GroupChangeRepository())
|
||||
recordSetRepo <- IO(RecordSetRepository())
|
||||
recordChangeRepo <- IO(RecordChangeRepository())
|
||||
zoneChangeRepo <- IO(ZoneChangeRepository())
|
||||
batchChangeRepo <- IO(BatchChangeRepository())
|
||||
sqsConfig <- IO(VinylDNSConfig.sqsConfig)
|
||||
sqsConnection <- IO(SqsConnection(sqsConfig))
|
||||
processingDisabled <- IO(VinylDNSConfig.vinyldnsConfig.getBoolean("processing-disabled"))
|
||||
|
@ -18,9 +18,11 @@ package vinyldns.api
|
||||
|
||||
import akka.actor.ActorSystem
|
||||
import com.typesafe.config.{Config, ConfigFactory}
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.util.matching.Regex
|
||||
import vinyldns.api.domain.zone.ZoneConnection
|
||||
import vinyldns.api.repository.DataStoreConfig
|
||||
|
||||
object VinylDNSConfig {
|
||||
|
||||
@ -37,7 +39,8 @@ object VinylDNSConfig {
|
||||
lazy val groupsStoreConfig: Config = vinyldnsConfig.getConfig("groups")
|
||||
lazy val groupChangesStoreConfig: Config = vinyldnsConfig.getConfig("groupChanges")
|
||||
lazy val membershipStoreConfig: Config = vinyldnsConfig.getConfig("membership")
|
||||
lazy val dbConfig: Config = vinyldnsConfig.getConfig("db")
|
||||
lazy val mySqlConfig: DataStoreConfig =
|
||||
pureconfig.loadConfigOrThrow[DataStoreConfig](vinyldnsConfig, "mysql")
|
||||
lazy val sqsConfig: Config = vinyldnsConfig.getConfig("sqs")
|
||||
lazy val cryptoConfig: Config = vinyldnsConfig.getConfig("crypto")
|
||||
lazy val system: ActorSystem = ActorSystem("VinylDNS", VinylDNSConfig.config)
|
||||
|
@ -18,7 +18,6 @@ package vinyldns.api.domain.batch
|
||||
|
||||
import cats.effect._
|
||||
import vinyldns.api.repository.Repository
|
||||
import vinyldns.api.repository.mysql.VinylDNSJDBC
|
||||
|
||||
// $COVERAGE-OFF$
|
||||
trait BatchChangeRepository extends Repository {
|
||||
@ -38,8 +37,4 @@ trait BatchChangeRepository extends Repository {
|
||||
def getSingleChanges(singleChangeIds: List[String]): IO[List[SingleChange]]
|
||||
|
||||
}
|
||||
|
||||
object BatchChangeRepository {
|
||||
def apply(): BatchChangeRepository = VinylDNSJDBC.instance.batchChangeRepository
|
||||
}
|
||||
// $COVERAGE-ON$
|
||||
|
@ -25,7 +25,7 @@ final case class ListGroupChangesResults(
|
||||
changes: Seq[GroupChange],
|
||||
lastEvaluatedTimeStamp: Option[String])
|
||||
|
||||
trait GroupChangeRepository extends Repository {
|
||||
trait GroupChangeRepository extends Repository {
|
||||
def save(groupChange: GroupChange): IO[GroupChange]
|
||||
def getGroupChange(groupChangeId: String): IO[Option[GroupChange]] // For testing
|
||||
def getGroupChanges(
|
||||
|
@ -19,7 +19,6 @@ package vinyldns.api.domain.zone
|
||||
import cats.effect._
|
||||
import vinyldns.api.domain.auth.AuthPrincipal
|
||||
import vinyldns.api.repository.Repository
|
||||
import vinyldns.api.repository.mysql.VinylDNSJDBC
|
||||
|
||||
trait ZoneRepository extends Repository {
|
||||
|
||||
@ -42,7 +41,3 @@ trait ZoneRepository extends Repository {
|
||||
def getZonesByAdminGroupId(adminGroupId: String): IO[List[Zone]]
|
||||
|
||||
}
|
||||
|
||||
object ZoneRepository {
|
||||
def apply(): ZoneRepository = VinylDNSJDBC.instance.zoneRepository
|
||||
}
|
||||
|
@ -27,12 +27,7 @@ case class SqsCredentials(
|
||||
case class SqsConfig(embedded: Boolean, sqsCredentials: SqsCredentials)
|
||||
|
||||
object SqsConfig {
|
||||
import pureconfig._
|
||||
import pureconfig.error.ConfigReaderException
|
||||
import pureconfig.module.catseffect.loadConfigF
|
||||
|
||||
def apply(config: Config, path: String): IO[SqsConfig] =
|
||||
loadConfig[SqsConfig](config, path) match {
|
||||
case Right(c) => IO.pure(c)
|
||||
case Left(e) => IO.raiseError(new ConfigReaderException[SqsConfig](e))
|
||||
}
|
||||
def apply(config: Config, path: String): IO[SqsConfig] = loadConfigF[IO, SqsConfig](config, path)
|
||||
}
|
||||
|
@ -0,0 +1,123 @@
|
||||
/*
|
||||
* Copyright 2018 Comcast Cable Communications Management, LLC
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package vinyldns.api.repository.mysql
|
||||
|
||||
import cats.effect.IO
|
||||
import com.zaxxer.hikari.HikariDataSource
|
||||
import javax.sql.DataSource
|
||||
import org.flywaydb.core.Flyway
|
||||
import org.slf4j.LoggerFactory
|
||||
import pureconfig.module.catseffect.loadConfigF
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
import scalikejdbc.config.DBs
|
||||
import scalikejdbc.{ConnectionPool, DataSourceConnectionPool}
|
||||
import vinyldns.api.repository._
|
||||
|
||||
class MySqlDataStoreProvider extends DataStoreProvider {
|
||||
|
||||
private val logger = LoggerFactory.getLogger("MySqlDataStoreProvider")
|
||||
private val implementedRepositories = Set(RepositoryName.zone, RepositoryName.batchChange)
|
||||
|
||||
def load(config: DataStoreConfig): IO[DataStore] =
|
||||
for {
|
||||
settingsConfig <- loadConfigF[IO, MySqlDataStoreSettings](config.settings)
|
||||
_ <- validateRepos(config.repositories)
|
||||
_ <- runDBMigrations(settingsConfig)
|
||||
_ <- setupDBConnection(settingsConfig)
|
||||
store <- initializeRepos()
|
||||
} yield store
|
||||
|
||||
def validateRepos(reposConfig: RepositoriesConfig): IO[Unit] = {
|
||||
val invalid = reposConfig.keys.diff(implementedRepositories)
|
||||
|
||||
if (invalid.isEmpty) {
|
||||
IO.unit
|
||||
} else {
|
||||
val error = s"Invalid config provided to mysql; unimplemented repos included: $invalid"
|
||||
IO.raiseError(DataStoreStartupError(error))
|
||||
}
|
||||
}
|
||||
|
||||
def initializeRepos(): IO[DataStore] = IO {
|
||||
val zones = Some(new JdbcZoneRepository())
|
||||
val batchChanges = Some(new JdbcBatchChangeRepository())
|
||||
new DataStore(zoneRepository = zones, batchChangeRepository = batchChanges)
|
||||
}
|
||||
|
||||
def runDBMigrations(settings: MySqlDataStoreSettings): IO[Unit] = IO {
|
||||
// Migration needs to happen on the base URL, not the table URL, thus the separate source
|
||||
lazy val migrationDataSource: DataSource = {
|
||||
val ds = new HikariDataSource()
|
||||
ds.setDriverClassName(settings.driver)
|
||||
ds.setJdbcUrl(settings.migrationUrl)
|
||||
ds.setUsername(settings.user)
|
||||
ds.setPassword(settings.password)
|
||||
// migrations happen once on startup; without these settings the default number of connections
|
||||
// will be created and maintained even though this datasource is no longer needed post-migration
|
||||
ds.setMaximumPoolSize(3)
|
||||
ds.setMinimumIdle(0)
|
||||
ds
|
||||
}
|
||||
|
||||
logger.info("Running migrations to ready the databases")
|
||||
|
||||
val migration = new Flyway()
|
||||
migration.setDataSource(migrationDataSource)
|
||||
// flyway changed the default schema table name in v5.0.0
|
||||
// this allows to revert to an old naming convention if needed
|
||||
settings.migrationSchemaTable.foreach { tableName =>
|
||||
migration.setTable(tableName)
|
||||
}
|
||||
|
||||
val placeholders = Map("dbName" -> settings.name)
|
||||
migration.setPlaceholders(placeholders.asJava)
|
||||
migration.setSchemas(settings.name)
|
||||
|
||||
// Runs flyway migrations
|
||||
migration.migrate()
|
||||
logger.info("migrations complete")
|
||||
}
|
||||
|
||||
def setupDBConnection(settings: MySqlDataStoreSettings): IO[Unit] = IO {
|
||||
val dataSource: DataSource = {
|
||||
val ds = new HikariDataSource()
|
||||
ds.setDriverClassName(settings.driver)
|
||||
ds.setJdbcUrl(settings.url)
|
||||
ds.setUsername(settings.user)
|
||||
ds.setPassword(settings.password)
|
||||
ds.setConnectionTimeout(settings.connectionTimeoutMillis)
|
||||
ds.setMaximumPoolSize(settings.poolMaxSize)
|
||||
ds.setMaxLifetime(settings.maxLifeTime)
|
||||
ds.setRegisterMbeans(true)
|
||||
ds
|
||||
}
|
||||
|
||||
logger.info("configuring connection pool")
|
||||
|
||||
// Configure the connection pool
|
||||
ConnectionPool.singleton(new DataSourceConnectionPool(dataSource))
|
||||
|
||||
logger.info("setting up databases")
|
||||
|
||||
// Sets up all databases with scalikejdbc
|
||||
DBs.setupAll()
|
||||
|
||||
logger.info("database init complete")
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,29 @@
|
||||
/*
|
||||
* Copyright 2018 Comcast Cable Communications Management, LLC
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package vinyldns.api.repository.mysql
|
||||
|
||||
case class MySqlDataStoreSettings(
|
||||
name: String,
|
||||
driver: String,
|
||||
migrationUrl: String,
|
||||
url: String,
|
||||
user: String,
|
||||
password: String,
|
||||
poolMaxSize: Int,
|
||||
connectionTimeoutMillis: Long,
|
||||
maxLifeTime: Long,
|
||||
migrationSchemaTable: Option[String])
|
@ -1,99 +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.api.repository.mysql
|
||||
|
||||
import com.typesafe.config.Config
|
||||
import com.zaxxer.hikari.HikariDataSource
|
||||
import javax.sql.DataSource
|
||||
import org.flywaydb.core.Flyway
|
||||
import org.slf4j.LoggerFactory
|
||||
import scala.collection.JavaConverters._
|
||||
import scalikejdbc.config.DBs
|
||||
import scalikejdbc.{ConnectionPool, DataSourceConnectionPool}
|
||||
import vinyldns.api.VinylDNSConfig
|
||||
|
||||
object VinylDNSJDBC {
|
||||
|
||||
val config: Config = VinylDNSConfig.dbConfig
|
||||
lazy val instance: VinylDNSJDBC = new VinylDNSJDBC(config)
|
||||
}
|
||||
|
||||
/* Loads and initializes the MySQL database. Unsafe, will fail if there are any issues and the app won't start */
|
||||
class VinylDNSJDBC(config: Config) {
|
||||
|
||||
private val logger = LoggerFactory.getLogger("VinylDNSJDBC")
|
||||
|
||||
val dataSource: DataSource = {
|
||||
val ds = new HikariDataSource()
|
||||
ds.setDriverClassName(config.getString("default.driver"))
|
||||
ds.setJdbcUrl(config.getString("default.url"))
|
||||
ds.setUsername(config.getString("default.user"))
|
||||
ds.setPassword(config.getString("default.password"))
|
||||
ds.setConnectionTimeout(config.getLong("default.connectionTimeoutMillis"))
|
||||
ds.setMaximumPoolSize(config.getInt("default.poolMaxSize"))
|
||||
ds.setMaxLifetime(config.getLong("default.maxLifeTime"))
|
||||
ds.setRegisterMbeans(true)
|
||||
ds
|
||||
}
|
||||
|
||||
// Migration needs to happen on the base URL, not the table URL, thus the separate source
|
||||
lazy val migrationDataSource: DataSource = {
|
||||
val ds = new HikariDataSource()
|
||||
ds.setDriverClassName(config.getString("default.driver"))
|
||||
ds.setJdbcUrl(config.getString("default.migrationUrl"))
|
||||
ds.setUsername(config.getString("default.user"))
|
||||
ds.setPassword(config.getString("default.password"))
|
||||
// migrations happen once on startup; without these settings the default number of connections
|
||||
// will be created and maintained even though this datasource is no longer needed post-migration
|
||||
ds.setMaximumPoolSize(3)
|
||||
ds.setMinimumIdle(0)
|
||||
ds
|
||||
}
|
||||
|
||||
logger.info("Running migrations to ready the databases")
|
||||
|
||||
val migration = new Flyway()
|
||||
migration.setDataSource(migrationDataSource)
|
||||
// flyway changed the default schema table name in v5.0.0; this allows to revert to an old naming convention if needed
|
||||
if (config.hasPath("default.migrationSchemaTable")) {
|
||||
migration.setTable(config.getString("default.migrationSchemaTable"))
|
||||
}
|
||||
|
||||
val dbName = config.getString("name")
|
||||
val placeholders = Map("dbName" -> dbName)
|
||||
migration.setPlaceholders(placeholders.asJava)
|
||||
migration.setSchemas(dbName)
|
||||
|
||||
// Runs flyway migrations
|
||||
migration.migrate()
|
||||
logger.info("migrations complete")
|
||||
|
||||
logger.info("configuring connection pool")
|
||||
|
||||
// Configure the connection pool
|
||||
ConnectionPool.singleton(new DataSourceConnectionPool(dataSource))
|
||||
|
||||
logger.info("setting up databases")
|
||||
|
||||
// Sets up all databases with scalikejdbc
|
||||
DBs.setupAll()
|
||||
|
||||
logger.info("database init complete")
|
||||
|
||||
val zoneRepository: JdbcZoneRepository = new JdbcZoneRepository()
|
||||
val batchChangeRepository: JdbcBatchChangeRepository = new JdbcBatchChangeRepository()
|
||||
}
|
@ -21,6 +21,15 @@ vinyldns {
|
||||
"some.test.ns."
|
||||
]
|
||||
|
||||
mysql.repositories {
|
||||
zone {
|
||||
# no additional settings for now
|
||||
},
|
||||
batch-change {
|
||||
# no additional settings for now
|
||||
}
|
||||
}
|
||||
|
||||
accounts {
|
||||
dummy = true
|
||||
|
||||
|
@ -0,0 +1,110 @@
|
||||
/*
|
||||
* Copyright 2018 Comcast Cable Communications Management, LLC
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package vinyldns.api.repository.mysql
|
||||
|
||||
import com.typesafe.config.{Config, ConfigFactory}
|
||||
import org.scalatest.{Matchers, WordSpec}
|
||||
import vinyldns.api.repository.{DataStoreConfig, DataStoreStartupError}
|
||||
|
||||
class MySqlDataStoreProviderSpec extends WordSpec with Matchers {
|
||||
|
||||
val mySqlConfig: Config = ConfigFactory.parseString(
|
||||
"""
|
||||
| class-name = "vinyldns.api.repository.mysql.MySqlDataStoreProvider"
|
||||
|
|
||||
| settings {
|
||||
| name = "test-database"
|
||||
| driver = "org.mariadb.jdbc.Driver"
|
||||
| migration-url = "test-url"
|
||||
| url = "test-url"
|
||||
| user = "test-user"
|
||||
| password = "test-pass"
|
||||
| pool-initial-size = 10
|
||||
| pool-max-size = 20
|
||||
| connection-timeout-millis = 1000
|
||||
| max-life-time = 600000
|
||||
| }
|
||||
|
|
||||
| repositories {
|
||||
| zone {},
|
||||
| batch-change {}
|
||||
| }
|
||||
| """.stripMargin)
|
||||
|
||||
val dataStoreSettings: DataStoreConfig =
|
||||
pureconfig.loadConfigOrThrow[DataStoreConfig](mySqlConfig)
|
||||
|
||||
val underTest = new MySqlDataStoreProvider()
|
||||
|
||||
"validateRepos" should {
|
||||
"Return successfully if all configured repos are implemented" in {
|
||||
noException should be thrownBy underTest
|
||||
.validateRepos(dataStoreSettings.repositories)
|
||||
.unsafeRunSync()
|
||||
}
|
||||
"Fail if an unimplemented repo is enabled" in {
|
||||
val placeHolder = ConfigFactory.parseString("test=test")
|
||||
val badRepos = dataStoreSettings.repositories.copy(user = Some(placeHolder))
|
||||
|
||||
val thrown = the[DataStoreStartupError] thrownBy underTest
|
||||
.validateRepos(badRepos)
|
||||
.unsafeRunSync()
|
||||
|
||||
thrown.msg shouldBe "Invalid config provided to mysql; unimplemented repos included: Set(user)"
|
||||
}
|
||||
}
|
||||
"load" should {
|
||||
// Note: success here will actually startup the repos. if the integration tests pass, that is working
|
||||
// as those are calling MySqlDataStoreProvider.load
|
||||
"Fail if a required setting is not included" in {
|
||||
val badConfig = ConfigFactory.parseString(
|
||||
"""
|
||||
| class-name = "vinyldns.api.repository.mysql.MySqlDataStoreProvider"
|
||||
|
|
||||
| settings {
|
||||
| name = "test-database"
|
||||
| driver = "org.mariadb.jdbc.Driver"
|
||||
| migration-url = "test-url"
|
||||
| pool-initial-size = 10
|
||||
| pool-max-size = 20
|
||||
| connection-timeout-millis = 1000
|
||||
| max-life-time = 600000
|
||||
| }
|
||||
|
|
||||
| repositories {
|
||||
| zone {},
|
||||
| batch-change {}
|
||||
| }
|
||||
| """.stripMargin)
|
||||
|
||||
val badSettings = pureconfig.loadConfigOrThrow[DataStoreConfig](badConfig)
|
||||
|
||||
a[pureconfig.error.ConfigReaderException[MySqlDataStoreSettings]] should be thrownBy underTest
|
||||
.load(badSettings)
|
||||
.unsafeRunSync()
|
||||
}
|
||||
"Fail if validateRepos fails" in {
|
||||
val placeHolder = ConfigFactory.parseString("test=test")
|
||||
val badRepos = dataStoreSettings.repositories.copy(user = Some(placeHolder))
|
||||
val badSettings = dataStoreSettings.copy(repositories = badRepos)
|
||||
|
||||
a[DataStoreStartupError] should be thrownBy underTest
|
||||
.load(badSettings)
|
||||
.unsafeRunSync()
|
||||
}
|
||||
}
|
||||
}
|
@ -29,18 +29,26 @@ vinyldns {
|
||||
}
|
||||
|
||||
# default settings point to the setup from docker compose
|
||||
db {
|
||||
name = "vinyldns"
|
||||
default {
|
||||
mysql {
|
||||
settings {
|
||||
name = "vinyldns"
|
||||
driver = "org.mariadb.jdbc.Driver"
|
||||
migrationUrl = "jdbc:mariadb://vinyldns-mysql:3306/?user=root&password=pass"
|
||||
migration-url = "jdbc:mariadb://vinyldns-mysql:3306/?user=root&password=pass"
|
||||
url = "jdbc:mariadb://vinyldns-mysql:3306/vinyldns?user=root&password=pass"
|
||||
user = "root"
|
||||
password = "pass"
|
||||
poolInitialSize = 10
|
||||
poolMaxSize = 20
|
||||
connectionTimeoutMillis = 1000
|
||||
maxLifeTime = 600000
|
||||
pool-initial-size = 10
|
||||
pool-max-size = 20
|
||||
connection-timeout-millis = 1000
|
||||
max-life-time = 600000
|
||||
}
|
||||
repositories {
|
||||
zone {
|
||||
# no additional settings for now
|
||||
},
|
||||
batch-change {
|
||||
# no additional settings for now
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -4,7 +4,7 @@ object Dependencies {
|
||||
lazy val akkaHttpV = "10.1.3"
|
||||
lazy val akkaV = "2.5.12"
|
||||
lazy val jettyV = "8.1.12.v20130726"
|
||||
lazy val pureConfigV = "0.9.0"
|
||||
lazy val pureConfigV = "0.9.2"
|
||||
lazy val metricsScalaV = "3.5.9"
|
||||
lazy val prometheusV = "0.4.0"
|
||||
lazy val catsEffectV = "0.10.1"
|
||||
@ -44,6 +44,7 @@ object Dependencies {
|
||||
"org.slf4j" % "slf4j-api" % "1.7.7",
|
||||
"co.fs2" %% "fs2-core" % "0.10.5",
|
||||
"com.github.pureconfig" %% "pureconfig" % pureConfigV,
|
||||
"com.github.pureconfig" %% "pureconfig-cats-effect" % pureConfigV,
|
||||
"io.prometheus" % "simpleclient_hotspot" % prometheusV,
|
||||
"io.prometheus" % "simpleclient_dropwizard" % prometheusV,
|
||||
"io.prometheus" % "simpleclient_common" % prometheusV,
|
||||
|
Loading…
x
Reference in New Issue
Block a user